From 86713399ccd08f14d986391f40d32273268e4fb0 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 11 Apr 2024 17:59:28 +0800 Subject: [PATCH 01/64] feat(rise-ctl): add fix dirty upstream fragment ids for streaming job (#16258) --- src/ctl/src/cmd_impl/debug.rs | 2 + .../src/cmd_impl/debug/fix_table_fragments.rs | 76 +++++++++++++++++++ src/ctl/src/lib.rs | 17 +++++ 3 files changed, 95 insertions(+) create mode 100644 src/ctl/src/cmd_impl/debug/fix_table_fragments.rs diff --git a/src/ctl/src/cmd_impl/debug.rs b/src/ctl/src/cmd_impl/debug.rs index fed9a564da71..1b7d7db5b2aa 100644 --- a/src/ctl/src/cmd_impl/debug.rs +++ b/src/ctl/src/cmd_impl/debug.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod fix_table_fragments; mod meta_store; +pub use fix_table_fragments::*; pub use meta_store::*; diff --git a/src/ctl/src/cmd_impl/debug/fix_table_fragments.rs b/src/ctl/src/cmd_impl/debug/fix_table_fragments.rs new file mode 100644 index 000000000000..627545ac178e --- /dev/null +++ b/src/ctl/src/cmd_impl/debug/fix_table_fragments.rs @@ -0,0 +1,76 @@ +// 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 etcd_client::ConnectOptions; +use risingwave_common::util::stream_graph_visitor::visit_stream_node_cont; +use risingwave_meta::model::{MetadataModel, TableFragments}; +use risingwave_meta::storage::{EtcdMetaStore, WrappedEtcdClient}; +use risingwave_pb::stream_plan::stream_node::NodeBody; + +use crate::DebugCommon; + +pub async fn fix_table_fragments( + common: DebugCommon, + table_id: u32, + dirty_fragment_ids: Vec, +) -> anyhow::Result<()> { + let DebugCommon { + etcd_endpoints, + etcd_username, + etcd_password, + enable_etcd_auth, + .. + } = common; + + let client = if enable_etcd_auth { + let options = ConnectOptions::default().with_user( + etcd_username.clone().unwrap_or_default(), + etcd_password.clone().unwrap_or_default(), + ); + WrappedEtcdClient::connect(etcd_endpoints.clone(), Some(options), true).await? + } else { + WrappedEtcdClient::connect(etcd_endpoints.clone(), None, false).await? + }; + + let meta_store = EtcdMetaStore::new(client); + + let mut table_fragments = TableFragments::select(&meta_store, &table_id) + .await? + .expect("table fragments not found"); + + for fragment in table_fragments.fragments.values_mut() { + fragment + .upstream_fragment_ids + .retain(|id| !dirty_fragment_ids.contains(id)); + for actor in &mut fragment.actors { + visit_stream_node_cont(actor.nodes.as_mut().unwrap(), |node| { + if let Some(NodeBody::Union(_)) = node.node_body { + node.input.retain_mut(|input| { + if let Some(NodeBody::Merge(merge_node)) = &mut input.node_body + && dirty_fragment_ids.contains(&merge_node.upstream_fragment_id) + { + false + } else { + true + } + }); + } + true + }) + } + } + + table_fragments.insert(&meta_store).await?; + Ok(()) +} diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index c41dec52aaad..a1aaa8f48c5f 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -141,6 +141,18 @@ pub enum DebugCommands { #[command(flatten)] common: DebugCommon, }, + /// Fix table fragments by cleaning up some un-exist fragments, which happens when the upstream + /// streaming job is failed to create and the fragments are not cleaned up due to some unidentified issues. + FixDirtyUpstreams { + #[command(flatten)] + common: DebugCommon, + + #[clap(long)] + table_id: u32, + + #[clap(long, value_delimiter = ',')] + dirty_fragment_ids: Vec, + }, } #[derive(Subcommand)] @@ -853,6 +865,11 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { .await? } Commands::Debug(DebugCommands::Dump { common }) => cmd_impl::debug::dump(common).await?, + Commands::Debug(DebugCommands::FixDirtyUpstreams { + common, + table_id, + dirty_fragment_ids, + }) => cmd_impl::debug::fix_table_fragments(common, table_id, dirty_fragment_ids).await?, Commands::Throttle(ThrottleCommands::Source(args)) => { apply_throttle(context, risingwave_pb::meta::PbThrottleTarget::Source, args).await? } From 0e8d9c0f5cc0e48e2198ae5b56b51aec6e6e9729 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Thu, 11 Apr 2024 18:25:40 +0800 Subject: [PATCH 02/64] refactor(source): rename `update_with_offset(start_offset)` to `update_offset(last_read_offset)` (#16256) Signed-off-by: Richard Chien --- src/connector/src/source/base.rs | 10 +++---- src/connector/src/source/cdc/split.rs | 28 +++++++++---------- src/connector/src/source/datagen/split.rs | 4 +-- .../src/source/filesystem/file_common.rs | 8 +++--- .../src/source/google_pubsub/split.rs | 4 +-- src/connector/src/source/iceberg/mod.rs | 2 +- src/connector/src/source/kafka/split.rs | 4 +-- src/connector/src/source/kinesis/split.rs | 6 ++-- src/connector/src/source/mqtt/split.rs | 2 +- src/connector/src/source/nats/split.rs | 6 ++-- src/connector/src/source/nexmark/split.rs | 4 +-- src/connector/src/source/pulsar/split.rs | 6 ++-- src/connector/src/source/test_source.rs | 4 +-- src/meta/src/stream/source_manager.rs | 2 +- 14 files changed, 45 insertions(+), 45 deletions(-) diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index b3c8c4010a9c..7b6c6a71897c 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -450,12 +450,12 @@ impl SplitMetaData for SplitImpl { Self::restore_from_json_inner(&split_type, inner_value.into()) } - fn update_with_offset(&mut self, start_offset: String) -> Result<()> { + fn update_offset(&mut self, last_seen_offset: String) -> Result<()> { dispatch_split_impl!( self, inner, IgnoreType, - inner.update_with_offset(start_offset) + inner.update_offset(last_seen_offset) ) } } @@ -467,9 +467,9 @@ impl SplitImpl { }) } - pub fn update_in_place(&mut self, start_offset: String) -> Result<()> { + pub fn update_in_place(&mut self, last_seen_offset: String) -> Result<()> { dispatch_split_impl!(self, inner, IgnoreType, { - inner.update_with_offset(start_offset)? + inner.update_offset(last_seen_offset)? }); Ok(()) } @@ -540,7 +540,7 @@ pub trait SplitMetaData: Sized { fn encode_to_json(&self) -> JsonbVal; fn restore_from_json(value: JsonbVal) -> Result; - fn update_with_offset(&mut self, start_offset: String) -> crate::error::ConnectorResult<()>; + fn update_offset(&mut self, last_seen_offset: String) -> crate::error::ConnectorResult<()>; } /// [`ConnectorState`] maintains the consuming splits' info. In specific split readers, diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index 82c36a29892d..d38200910f7e 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -46,7 +46,7 @@ trait CdcSplitTrait: Send + Sync { fn split_id(&self) -> u32; fn start_offset(&self) -> &Option; fn is_snapshot_done(&self) -> bool; - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()>; + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()>; // MySQL and MongoDB shares the same logic to extract the snapshot flag fn extract_snapshot_flag(&self, start_offset: &str) -> ConnectorResult { @@ -116,10 +116,10 @@ impl CdcSplitTrait for MySqlCdcSplit { self.inner.snapshot_done } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { // if snapshot_done is already true, it won't be updated - self.inner.snapshot_done = self.extract_snapshot_flag(start_offset.as_str())?; - self.inner.start_offset = Some(start_offset); + self.inner.snapshot_done = self.extract_snapshot_flag(last_seen_offset.as_str())?; + self.inner.start_offset = Some(last_seen_offset); Ok(()) } } @@ -151,9 +151,9 @@ impl CdcSplitTrait for PostgresCdcSplit { self.inner.snapshot_done } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - self.inner.snapshot_done = self.extract_snapshot_flag(start_offset.as_str())?; - self.inner.start_offset = Some(start_offset); + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + self.inner.snapshot_done = self.extract_snapshot_flag(last_seen_offset.as_str())?; + self.inner.start_offset = Some(last_seen_offset); Ok(()) } @@ -206,10 +206,10 @@ impl CdcSplitTrait for MongoDbCdcSplit { self.inner.snapshot_done } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { // if snapshot_done is already true, it will remain true - self.inner.snapshot_done = self.extract_snapshot_flag(start_offset.as_str())?; - self.inner.start_offset = Some(start_offset); + self.inner.snapshot_done = self.extract_snapshot_flag(last_seen_offset.as_str())?; + self.inner.start_offset = Some(last_seen_offset); Ok(()) } } @@ -268,8 +268,8 @@ impl SplitMetaData for DebeziumCdcSplit { serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - self.update_with_offset(start_offset) + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + self.update_offset_inner(last_seen_offset) } } @@ -318,8 +318,8 @@ impl DebeziumCdcSplit { dispatch_cdc_split!(self, ref, is_snapshot_done()) } - pub fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - dispatch_cdc_split!(self, mut, update_with_offset(start_offset)?); + pub fn update_offset_inner(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + dispatch_cdc_split!(self, mut, update_offset(last_seen_offset)?); Ok(()) } } diff --git a/src/connector/src/source/datagen/split.rs b/src/connector/src/source/datagen/split.rs index 6d51cfa7d47a..7a21b320f0d9 100644 --- a/src/connector/src/source/datagen/split.rs +++ b/src/connector/src/source/datagen/split.rs @@ -40,8 +40,8 @@ impl SplitMetaData for DatagenSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - self.start_offset = Some(start_offset.as_str().parse::().unwrap()); + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + self.start_offset = Some(last_seen_offset.as_str().parse::().unwrap()); Ok(()) } } diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index ccff7315491b..b2c5d5ccdb4d 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -55,8 +55,8 @@ impl SplitMetaData for FsSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - let offset = start_offset.parse().unwrap(); + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + let offset = last_seen_offset.parse().unwrap(); self.offset = offset; Ok(()) } @@ -106,8 +106,8 @@ impl SplitMetaData for OpendalFsSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - let offset = start_offset.parse().unwrap(); + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + let offset = last_seen_offset.parse().unwrap(); self.offset = offset; Ok(()) } diff --git a/src/connector/src/source/google_pubsub/split.rs b/src/connector/src/source/google_pubsub/split.rs index 91253afd825f..14c40150488a 100644 --- a/src/connector/src/source/google_pubsub/split.rs +++ b/src/connector/src/source/google_pubsub/split.rs @@ -48,8 +48,8 @@ impl SplitMetaData for PubsubSplit { format!("{}-{}", self.subscription, self.index).into() } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - self.start_offset = Some(start_offset); + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + self.start_offset = Some(last_seen_offset); Ok(()) } } diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 5f477f9b9668..e7e4971dd42c 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -126,7 +126,7 @@ impl SplitMetaData for IcebergSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, _start_offset: String) -> ConnectorResult<()> { + fn update_offset(&mut self, _last_seen_offset: String) -> ConnectorResult<()> { unimplemented!() } } diff --git a/src/connector/src/source/kafka/split.rs b/src/connector/src/source/kafka/split.rs index a98707eb4ab3..a649a729c039 100644 --- a/src/connector/src/source/kafka/split.rs +++ b/src/connector/src/source/kafka/split.rs @@ -40,8 +40,8 @@ impl SplitMetaData for KafkaSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - self.start_offset = Some(start_offset.as_str().parse::().unwrap()); + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + self.start_offset = Some(last_seen_offset.as_str().parse::().unwrap()); Ok(()) } } diff --git a/src/connector/src/source/kinesis/split.rs b/src/connector/src/source/kinesis/split.rs index 1c7bea61f874..441846f00a78 100644 --- a/src/connector/src/source/kinesis/split.rs +++ b/src/connector/src/source/kinesis/split.rs @@ -47,11 +47,11 @@ impl SplitMetaData for KinesisSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - let start_offset = if start_offset.is_empty() { + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + let start_offset = if last_seen_offset.is_empty() { KinesisOffset::Earliest } else { - KinesisOffset::SequenceNumber(start_offset) + KinesisOffset::SequenceNumber(last_seen_offset) }; self.start_position = start_offset; diff --git a/src/connector/src/source/mqtt/split.rs b/src/connector/src/source/mqtt/split.rs index b86bde6097ae..85ad7394518f 100644 --- a/src/connector/src/source/mqtt/split.rs +++ b/src/connector/src/source/mqtt/split.rs @@ -38,7 +38,7 @@ impl SplitMetaData for MqttSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, _start_sequence: String) -> ConnectorResult<()> { + fn update_offset(&mut self, _last_seen_offset: String) -> ConnectorResult<()> { Ok(()) } } diff --git a/src/connector/src/source/nats/split.rs b/src/connector/src/source/nats/split.rs index d3b4ded01901..7a07c481f9bc 100644 --- a/src/connector/src/source/nats/split.rs +++ b/src/connector/src/source/nats/split.rs @@ -51,11 +51,11 @@ impl SplitMetaData for NatsSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_sequence: String) -> ConnectorResult<()> { - let start_sequence = if start_sequence.is_empty() { + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + let start_sequence = if last_seen_offset.is_empty() { NatsOffset::Earliest } else { - NatsOffset::SequenceNumber(start_sequence) + NatsOffset::SequenceNumber(last_seen_offset) }; self.start_sequence = start_sequence; Ok(()) diff --git a/src/connector/src/source/nexmark/split.rs b/src/connector/src/source/nexmark/split.rs index 5150f1b6a1e1..56fd8491c3e2 100644 --- a/src/connector/src/source/nexmark/split.rs +++ b/src/connector/src/source/nexmark/split.rs @@ -39,8 +39,8 @@ impl SplitMetaData for NexmarkSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - self.start_offset = Some(start_offset.as_str().parse::().unwrap()); + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + self.start_offset = Some(last_seen_offset.as_str().parse::().unwrap()); Ok(()) } } diff --git a/src/connector/src/source/pulsar/split.rs b/src/connector/src/source/pulsar/split.rs index bf9b63d99d74..d7651e6ed10b 100644 --- a/src/connector/src/source/pulsar/split.rs +++ b/src/connector/src/source/pulsar/split.rs @@ -40,11 +40,11 @@ impl SplitMetaData for PulsarSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - let start_offset = if start_offset.is_empty() { + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + let start_offset = if last_seen_offset.is_empty() { PulsarEnumeratorOffset::Earliest } else { - PulsarEnumeratorOffset::MessageId(start_offset) + PulsarEnumeratorOffset::MessageId(last_seen_offset) }; self.start_offset = start_offset; diff --git a/src/connector/src/source/test_source.rs b/src/connector/src/source/test_source.rs index 6d224593d7a2..e8379fafeeb7 100644 --- a/src/connector/src/source/test_source.rs +++ b/src/connector/src/source/test_source.rs @@ -154,8 +154,8 @@ impl SplitMetaData for TestSourceSplit { serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { - self.offset = start_offset; + fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> { + self.offset = last_seen_offset; Ok(()) } } diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index d950820889e1..cda40ff8fdce 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -1163,7 +1163,7 @@ mod tests { serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, _start_offset: String) -> ConnectorResult<()> { + fn update_offset(&mut self, _last_read_offset: String) -> ConnectorResult<()> { Ok(()) } } From a6aa6208d24d13f4e30cc2fb22ca9c180b9272a5 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 11 Apr 2024 19:06:13 +0800 Subject: [PATCH 03/64] fix(sql-backend): specify the initialized self-increment id to avoid the effect of having set the self-increment step (#16261) --- src/ctl/src/cmd_impl/meta/migration.rs | 4 +- .../migration/src/m20230908_072257_init.rs | 60 ++++++++++++++++--- 2 files changed, 54 insertions(+), 10 deletions(-) diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index 5e3806e93466..29461d42b9f8 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -815,14 +815,14 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an .conn .execute(Statement::from_string( DatabaseBackend::Postgres, - "SELECT setval('object_oid_seq', (SELECT MAX(oid) FROM object) + 1);", + "SELECT setval('object_oid_seq', SELECT MAX(oid) FROM object);", )) .await?; meta_store_sql .conn .execute(Statement::from_string( DatabaseBackend::Postgres, - "SELECT setval('user_user_id_seq', (SELECT MAX(user_id) FROM \"user\") + 1);", + "SELECT setval('user_user_id_seq', SELECT MAX(user_id) FROM \"user\");", )) .await?; } diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index b01e684cf4bf..14b79b33f26d 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -1,6 +1,6 @@ use sea_orm_migration::prelude::{Index as MigrationIndex, Table as MigrationTable, *}; -use crate::sea_orm::DbBackend; +use crate::sea_orm::{DatabaseBackend, DbBackend, Statement}; use crate::{assert_not_has_tables, drop_tables}; #[derive(DeriveMigrationName)] @@ -869,6 +869,7 @@ impl MigrationTrait for Migration { let insert_sys_users = Query::insert() .into_table(User::Table) .columns([ + User::UserId, User::Name, User::IsSuper, User::CanCreateUser, @@ -876,6 +877,7 @@ impl MigrationTrait for Migration { User::CanLogin, ]) .values_panic([ + 1.into(), "root".into(), true.into(), true.into(), @@ -883,6 +885,7 @@ impl MigrationTrait for Migration { true.into(), ]) .values_panic([ + 2.into(), "postgres".into(), true.into(), true.into(), @@ -894,15 +897,19 @@ impl MigrationTrait for Migration { // Since User table is newly created, we assume that the initial user id of `root` is 1 and `postgres` is 2. let insert_objects = Query::insert() .into_table(Object::Table) - .columns([Object::ObjType, Object::OwnerId, Object::DatabaseId]) - .values_panic(["DATABASE".into(), 1.into(), None::.into()]) - .values_panic(["SCHEMA".into(), 1.into(), 1.into()]) // public - .values_panic(["SCHEMA".into(), 1.into(), 1.into()]) // pg_catalog - .values_panic(["SCHEMA".into(), 1.into(), 1.into()]) // information_schema - .values_panic(["SCHEMA".into(), 1.into(), 1.into()]) // rw_catalog + .columns([ + Object::Oid, + Object::ObjType, + Object::OwnerId, + Object::DatabaseId, + ]) + .values_panic([1.into(), "DATABASE".into(), 1.into(), None::.into()]) + .values_panic([2.into(), "SCHEMA".into(), 1.into(), 1.into()]) // public + .values_panic([3.into(), "SCHEMA".into(), 1.into(), 1.into()]) // pg_catalog + .values_panic([4.into(), "SCHEMA".into(), 1.into(), 1.into()]) // information_schema + .values_panic([5.into(), "SCHEMA".into(), 1.into(), 1.into()]) // rw_catalog .to_owned(); - // Since all tables are newly created, we assume that the initial object id of `dev` is 1 and the schemas' ids are 2, 3, 4, 5. let insert_sys_database = Query::insert() .into_table(Database::Table) .columns([Database::DatabaseId, Database::Name]) @@ -923,6 +930,43 @@ impl MigrationTrait for Migration { manager.exec_stmt(insert_sys_database).await?; manager.exec_stmt(insert_sys_schemas).await?; + // Rest auto increment offset + match manager.get_database_backend() { + DbBackend::MySql => { + manager + .get_connection() + .execute(Statement::from_string( + DatabaseBackend::MySql, + "ALTER TABLE object AUTO_INCREMENT = 6", + )) + .await?; + manager + .get_connection() + .execute(Statement::from_string( + DatabaseBackend::MySql, + "ALTER TABLE user AUTO_INCREMENT = 3", + )) + .await?; + } + DbBackend::Postgres => { + manager + .get_connection() + .execute(Statement::from_string( + DatabaseBackend::Postgres, + "SELECT setval('object_oid_seq', 5)", + )) + .await?; + manager + .get_connection() + .execute(Statement::from_string( + DatabaseBackend::Postgres, + "SELECT setval('user_user_id_seq', 2)", + )) + .await?; + } + DbBackend::Sqlite => {} + } + Ok(()) } From 5f36b8a78afe0c0558118b7b661564cb9d1ac1cb Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 12 Apr 2024 12:30:40 +0800 Subject: [PATCH 04/64] chore(dependabot): don't update arrow in dependabot (#16270) Signed-off-by: xxchan --- .github/dependabot.yml | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 0dc1ea876851..0523e534c5d8 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -7,16 +7,19 @@ updates: open-pull-requests-limit: 10 # Disable auto rebase to reduce cost. Use `@dependabot rebase` manually instead. rebase-strategy: "disabled" - # Ignore patch to reduce spam. Manually run `cargo update` regularly instead. ignore: + # Ignore patch to reduce spam. Manually run `cargo update` regularly instead. - dependency-name: "*" update-types: ["version-update:semver-patch"] + # Ignore arrow crates. It does major releases frequently: https://github.com/apache/arrow-rs/issues/5368 + # We depend on arrow directly, and also many other crates depending on arrow, including icelake, deltalake, arrow-udf, ... + # It will always need human intervention, and we'd better be the last one to update arrow. + - dependency-name: "arrow*" + update-types: ["version-update:semver-minor", "version-update:semver-major"] + - dependency-name: "parquet" + update-types: ["version-update:semver-minor", "version-update:semver-major"] # Create a group of dependencies to be updated together in one pull request groups: - arrow: - patterns: - - "arrow*" - - "parquet" aws: patterns: - "aws*" From 7845a654eda6eb6a2656f3e1aa679757123eecb6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 12 Apr 2024 05:01:22 +0000 Subject: [PATCH 05/64] chore(deps): Bump sqllogictest from 0.19.1 to 0.20.0 (#16267) Signed-off-by: dependabot[bot] Signed-off-by: xxchan Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: xxchan --- Cargo.lock | 8 ++++---- src/tests/simulation/Cargo.toml | 2 +- src/tests/simulation/src/client.rs | 4 +--- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 58a157173e81..c835d484fb1e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1652,7 +1652,7 @@ dependencies = [ "bitflags 2.5.0", "cexpr", "clang-sys", - "itertools 0.10.5", + "itertools 0.12.1", "lazy_static", "lazycell", "log", @@ -8448,7 +8448,7 @@ dependencies = [ "indoc", "libc", "memoffset", - "parking_lot 0.11.2", + "parking_lot 0.12.1", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -12076,9 +12076,9 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.19.1" +version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "004809e4710d743a13eb2975fb6a1498da76157e19ad059d595d032e6a50e30b" +checksum = "b8518892e5e36bfa90163e53c4e4f36a388e0afa1cd6a3de0614253b3c9029c7" dependencies = [ "async-trait", "educe 0.4.23", diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 59fee3a7c77f..fc55e6356f4c 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -50,7 +50,7 @@ risingwave_sqlsmith = { workspace = true } serde = "1.0.188" serde_derive = "1.0.188" serde_json = "1.0.107" -sqllogictest = "0.19" +sqllogictest = "0.20" tempfile = "3" tikv-jemallocator = { workspace = true } tokio = { version = "0.2.24", package = "madsim-tokio" } diff --git a/src/tests/simulation/src/client.rs b/src/tests/simulation/src/client.rs index df5bf1e1a052..84717583eb42 100644 --- a/src/tests/simulation/src/client.rs +++ b/src/tests/simulation/src/client.rs @@ -220,9 +220,7 @@ impl sqllogictest::AsyncDB for RisingWave { tokio::time::sleep(dur).await } - async fn run_command( - _command: std::process::Command, - ) -> std::io::Result { + async fn run_command(_command: std::process::Command) -> std::io::Result { unimplemented!("spawning process is not supported in simulation mode") } } From 4ac029cde91eed09f988cae4c9ae1340869ef684 Mon Sep 17 00:00:00 2001 From: August Date: Fri, 12 Apr 2024 14:42:55 +0800 Subject: [PATCH 06/64] feat(sql-backend): change json column type to blob to allow proto field rename (#16090) --- src/ctl/src/cmd_impl/meta/migration.rs | 13 +- src/meta/model_v2/migration/README.md | 11 +- .../migration/src/m20230908_072257_init.rs | 57 +++---- .../migration/src/m20231008_020431_hummock.rs | 12 +- .../src/m20240304_074901_subscription.rs | 12 +- src/meta/model_v2/src/actor_dispatcher.rs | 4 +- src/meta/model_v2/src/compaction_config.rs | 3 +- src/meta/model_v2/src/compaction_status.rs | 4 +- src/meta/model_v2/src/compaction_task.rs | 5 +- src/meta/model_v2/src/connection.rs | 2 +- src/meta/model_v2/src/fragment.rs | 33 +--- src/meta/model_v2/src/function.rs | 4 +- .../model_v2/src/hummock_version_delta.rs | 5 +- src/meta/model_v2/src/lib.rs | 155 +++++++++++++++--- src/meta/model_v2/src/sink.rs | 2 +- src/meta/model_v2/src/source.rs | 6 +- src/meta/model_v2/src/table.rs | 4 +- src/meta/model_v2/src/user.rs | 4 +- src/meta/model_v2/src/view.rs | 2 +- src/meta/src/controller/catalog.rs | 20 +-- src/meta/src/controller/fragment.rs | 74 ++++----- src/meta/src/controller/mod.rs | 39 +++-- src/meta/src/controller/streaming_job.rs | 39 +++-- src/meta/src/controller/user.rs | 2 +- src/meta/src/controller/utils.rs | 13 +- src/meta/src/hummock/model/ext/hummock.rs | 14 +- src/meta/src/rpc/ddl_controller_v2.rs | 2 +- src/meta/src/stream/source_manager.rs | 2 +- 28 files changed, 305 insertions(+), 238 deletions(-) diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index 29461d42b9f8..db3e166c07a6 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -38,7 +38,6 @@ use risingwave_meta::stream::TableRevision; use risingwave_meta_model_migration::{Migrator, MigratorTrait}; use risingwave_meta_model_v2::catalog_version::VersionCategory; use risingwave_meta_model_v2::compaction_status::LevelHandlers; -use risingwave_meta_model_v2::fragment::StreamNode; use risingwave_meta_model_v2::hummock_sequence::{ COMPACTION_GROUP_ID, COMPACTION_TASK_ID, META_BACKUP_ID, SSTABLE_OBJECT_ID, }; @@ -435,7 +434,7 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an table.schema_id = *schema_rewrite.get(&table.schema_id).unwrap(); }); let mut fragment = fragment.into_active_model(); - fragment.stream_node = Set(StreamNode::from_protobuf(&stream_node)); + fragment.stream_node = Set((&stream_node).into()); Fragment::insert(fragment) .exec(&meta_store_sql.conn) .await?; @@ -683,7 +682,7 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an max_committed_epoch: Set(vd.max_committed_epoch as _), safe_epoch: Set(vd.safe_epoch as _), trivial_move: Set(vd.trivial_move), - full_version_delta: Set(vd.to_protobuf().into()), + full_version_delta: Set((&vd.to_protobuf()).into()), }) .collect_vec(), ) @@ -716,7 +715,7 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an .into_iter() .map(|cg| compaction_config::ActiveModel { compaction_group_id: Set(cg.group_id as _), - config: Set((*cg.compaction_config).clone().into()), + config: Set((&*cg.compaction_config).into()), }) .collect_vec(), ) @@ -733,7 +732,9 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an .into_iter() .map(|cs| compaction_status::ActiveModel { compaction_group_id: Set(cs.compaction_group_id as _), - status: Set(LevelHandlers(cs.level_handlers.iter().map_into().collect())), + status: Set(LevelHandlers::from( + cs.level_handlers.iter().map_into().collect_vec(), + )), }) .collect_vec(), ) @@ -751,7 +752,7 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an compaction_task::ActiveModel { id: Set(task.task_id as _), context_id: Set(context_id as _), - task: Set(task.into()), + task: Set((&task).into()), } })) .exec(&meta_store_sql.conn) diff --git a/src/meta/model_v2/migration/README.md b/src/meta/model_v2/migration/README.md index 3b438d89e31c..527d4ba53807 100644 --- a/src/meta/model_v2/migration/README.md +++ b/src/meta/model_v2/migration/README.md @@ -1,10 +1,19 @@ # Running Migrator CLI +> **WARNING:** Migration files are used to define schema changes for the database. Each migration file contains an up and down function, +> which are used to define upgrade and downgrade operations for the schema. +> +> When you need to make schema changes to the system catalog, you need to generate a new migration file and then apply it to the database. +> Note that each migration file can only be applied once and will be recorded in a system table, so for new schema changes, you need to +> generate a new migration file. Unless you are sure the modification of the migration file has not been included in any released version yet, +> **DO NOT** modify already published migration files. + +## How to run the migrator CLI - Generate a new migration file ```sh cargo run -- generate MIGRATION_NAME ``` -- Apply all pending migrations +- Apply all pending migrations for test purposes, `DATABASE_URL` required. ```sh cargo run ``` diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index 14b79b33f26d..260344a8b4fd 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -140,7 +140,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(User::CanCreateDb).boolean().not_null()) .col(ColumnDef::new(User::CanCreateUser).boolean().not_null()) .col(ColumnDef::new(User::CanLogin).boolean().not_null()) - .col(ColumnDef::new(User::AuthInfo).json_binary()) + .col(ColumnDef::new(User::AuthInfo).binary()) .to_owned(), ) .await?; @@ -381,11 +381,7 @@ impl MigrationTrait for Migration { .blob(BlobSize::Long) .not_null(), ) - .col( - ColumnDef::new(Fragment::VnodeMapping) - .json_binary() - .not_null(), - ) + .col(ColumnDef::new(Fragment::VnodeMapping).binary().not_null()) .col(ColumnDef::new(Fragment::StateTableIds).json_binary()) .col(ColumnDef::new(Fragment::UpstreamFragmentId).json_binary()) .foreign_key( @@ -411,12 +407,12 @@ impl MigrationTrait for Migration { ) .col(ColumnDef::new(Actor::FragmentId).integer().not_null()) .col(ColumnDef::new(Actor::Status).string().not_null()) - .col(ColumnDef::new(Actor::Splits).json_binary()) + .col(ColumnDef::new(Actor::Splits).binary()) .col(ColumnDef::new(Actor::ParallelUnitId).integer().not_null()) .col(ColumnDef::new(Actor::WorkerId).integer().not_null()) .col(ColumnDef::new(Actor::UpstreamActorIds).json_binary()) - .col(ColumnDef::new(Actor::VnodeBitmap).json_binary()) - .col(ColumnDef::new(Actor::ExprContext).json_binary().not_null()) + .col(ColumnDef::new(Actor::VnodeBitmap).binary()) + .col(ColumnDef::new(Actor::ExprContext).binary().not_null()) .foreign_key( &mut ForeignKey::create() .name("FK_actor_fragment_id") @@ -458,7 +454,7 @@ impl MigrationTrait for Migration { .json_binary() .not_null(), ) - .col(ColumnDef::new(ActorDispatcher::HashMapping).json_binary()) + .col(ColumnDef::new(ActorDispatcher::HashMapping).binary()) .col( ColumnDef::new(ActorDispatcher::DispatcherId) .integer() @@ -499,7 +495,7 @@ impl MigrationTrait for Migration { .primary_key(), ) .col(ColumnDef::new(Connection::Name).string().not_null()) - .col(ColumnDef::new(Connection::Info).json_binary().not_null()) + .col(ColumnDef::new(Connection::Info).binary().not_null()) .foreign_key( &mut ForeignKey::create() .name("FK_connection_object_id") @@ -518,7 +514,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Source::SourceId).integer().primary_key()) .col(ColumnDef::new(Source::Name).string().not_null()) .col(ColumnDef::new(Source::RowIdIndex).integer()) - .col(ColumnDef::new(Source::Columns).json_binary().not_null()) + .col(ColumnDef::new(Source::Columns).binary().not_null()) .col(ColumnDef::new(Source::PkColumnIds).json_binary().not_null()) .col( ColumnDef::new(Source::WithProperties) @@ -526,12 +522,8 @@ impl MigrationTrait for Migration { .not_null(), ) .col(ColumnDef::new(Source::Definition).text().not_null()) - .col(ColumnDef::new(Source::SourceInfo).json_binary()) - .col( - ColumnDef::new(Source::WatermarkDescs) - .json_binary() - .not_null(), - ) + .col(ColumnDef::new(Source::SourceInfo).binary()) + .col(ColumnDef::new(Source::WatermarkDescs).binary().not_null()) .col(ColumnDef::new(Source::OptionalAssociatedTableId).integer()) .col(ColumnDef::new(Source::ConnectionId).integer()) .col(ColumnDef::new(Source::Version).big_integer().not_null()) @@ -570,8 +562,8 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Table::OptionalAssociatedSourceId).integer()) .col(ColumnDef::new(Table::TableType).string().not_null()) .col(ColumnDef::new(Table::BelongsToJobId).integer()) - .col(ColumnDef::new(Table::Columns).json_binary().not_null()) - .col(ColumnDef::new(Table::Pk).json_binary().not_null()) + .col(ColumnDef::new(Table::Columns).binary().not_null()) + .col(ColumnDef::new(Table::Pk).binary().not_null()) .col( ColumnDef::new(Table::DistributionKey) .json_binary() @@ -601,14 +593,14 @@ impl MigrationTrait for Migration { ) .col(ColumnDef::new(Table::DistKeyInPk).json_binary().not_null()) .col(ColumnDef::new(Table::DmlFragmentId).integer()) - .col(ColumnDef::new(Table::Cardinality).json_binary()) + .col(ColumnDef::new(Table::Cardinality).binary()) .col( ColumnDef::new(Table::CleanedByWatermark) .boolean() .not_null(), ) .col(ColumnDef::new(Table::Description).string()) - .col(ColumnDef::new(Table::Version).json_binary()) + .col(ColumnDef::new(Table::Version).binary()) .col(ColumnDef::new(Table::RetentionSeconds).integer()) .col( ColumnDef::new(Table::IncomingSinks) @@ -650,7 +642,8 @@ impl MigrationTrait for Migration { &mut ForeignKey::create() .name("FK_table_optional_associated_source_id") .from(Table::Table, Table::OptionalAssociatedSourceId) - .to(Source::Table, Source::SourceId) + .to(Object::Table, Object::Oid) + .on_delete(ForeignKeyAction::Cascade) .to_owned(), ) .to_owned(), @@ -662,8 +655,8 @@ impl MigrationTrait for Migration { .table(Sink::Table) .col(ColumnDef::new(Sink::SinkId).integer().primary_key()) .col(ColumnDef::new(Sink::Name).string().not_null()) - .col(ColumnDef::new(Sink::Columns).json_binary().not_null()) - .col(ColumnDef::new(Sink::PlanPk).json_binary().not_null()) + .col(ColumnDef::new(Sink::Columns).binary().not_null()) + .col(ColumnDef::new(Sink::PlanPk).binary().not_null()) .col( ColumnDef::new(Sink::DistributionKey) .json_binary() @@ -676,7 +669,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Sink::ConnectionId).integer()) .col(ColumnDef::new(Sink::DbName).string().not_null()) .col(ColumnDef::new(Sink::SinkFromName).string().not_null()) - .col(ColumnDef::new(Sink::SinkFormatDesc).json_binary()) + .col(ColumnDef::new(Sink::SinkFormatDesc).binary()) .col(ColumnDef::new(Sink::TargetTable).integer()) .foreign_key( &mut ForeignKey::create() @@ -711,7 +704,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(View::Name).string().not_null()) .col(ColumnDef::new(View::Properties).json_binary().not_null()) .col(ColumnDef::new(View::Definition).text().not_null()) - .col(ColumnDef::new(View::Columns).json_binary().not_null()) + .col(ColumnDef::new(View::Columns).binary().not_null()) .foreign_key( &mut ForeignKey::create() .name("FK_view_object_id") @@ -731,7 +724,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Index::Name).string().not_null()) .col(ColumnDef::new(Index::IndexTableId).integer().not_null()) .col(ColumnDef::new(Index::PrimaryTableId).integer().not_null()) - .col(ColumnDef::new(Index::IndexItems).json_binary().not_null()) + .col(ColumnDef::new(Index::IndexItems).binary().not_null()) .col(ColumnDef::new(Index::IndexColumnsLen).integer().not_null()) .foreign_key( &mut ForeignKey::create() @@ -767,12 +760,8 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Function::FunctionId).integer().primary_key()) .col(ColumnDef::new(Function::Name).string().not_null()) .col(ColumnDef::new(Function::ArgNames).string().not_null()) - .col(ColumnDef::new(Function::ArgTypes).json_binary().not_null()) - .col( - ColumnDef::new(Function::ReturnType) - .json_binary() - .not_null(), - ) + .col(ColumnDef::new(Function::ArgTypes).binary().not_null()) + .col(ColumnDef::new(Function::ReturnType).binary().not_null()) .col(ColumnDef::new(Function::Language).string().not_null()) .col(ColumnDef::new(Function::Link).string()) .col(ColumnDef::new(Function::Identifier).string()) diff --git a/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs b/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs index b90e088da1f1..99f4d701c644 100644 --- a/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs +++ b/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs @@ -30,11 +30,7 @@ impl MigrationTrait for Migration { .not_null() .primary_key(), ) - .col( - ColumnDef::new(CompactionTask::Task) - .json_binary() - .not_null(), - ) + .col(ColumnDef::new(CompactionTask::Task).binary().not_null()) .col( ColumnDef::new(CompactionTask::ContextId) .integer() @@ -54,7 +50,7 @@ impl MigrationTrait for Migration { .not_null() .primary_key(), ) - .col(ColumnDef::new(CompactionConfig::Config).json_binary()) + .col(ColumnDef::new(CompactionConfig::Config).binary()) .to_owned(), ) .await?; @@ -69,7 +65,7 @@ impl MigrationTrait for Migration { .not_null() .primary_key(), ) - .col(ColumnDef::new(CompactionStatus::Status).json_binary()) + .col(ColumnDef::new(CompactionStatus::Status).binary()) .to_owned(), ) .await?; @@ -142,7 +138,7 @@ impl MigrationTrait for Migration { .boolean() .not_null(), ) - .col(ColumnDef::new(HummockVersionDelta::FullVersionDelta).json_binary()) + .col(ColumnDef::new(HummockVersionDelta::FullVersionDelta).binary()) .to_owned(), ) .await?; diff --git a/src/meta/model_v2/migration/src/m20240304_074901_subscription.rs b/src/meta/model_v2/migration/src/m20240304_074901_subscription.rs index 52bdb117727f..50564c9b211a 100644 --- a/src/meta/model_v2/migration/src/m20240304_074901_subscription.rs +++ b/src/meta/model_v2/migration/src/m20240304_074901_subscription.rs @@ -19,16 +19,8 @@ impl MigrationTrait for Migration { .primary_key(), ) .col(ColumnDef::new(Subscription::Name).string().not_null()) - .col( - ColumnDef::new(Subscription::Columns) - .json_binary() - .not_null(), - ) - .col( - ColumnDef::new(Subscription::PlanPk) - .json_binary() - .not_null(), - ) + .col(ColumnDef::new(Subscription::Columns).binary().not_null()) + .col(ColumnDef::new(Subscription::PlanPk).binary().not_null()) .col( ColumnDef::new(Subscription::DistributionKey) .json_binary() diff --git a/src/meta/model_v2/src/actor_dispatcher.rs b/src/meta/model_v2/src/actor_dispatcher.rs index fa5ea0daa8af..1d6a50665b12 100644 --- a/src/meta/model_v2/src/actor_dispatcher.rs +++ b/src/meta/model_v2/src/actor_dispatcher.rs @@ -61,7 +61,7 @@ impl From<(u32, PbDispatcher)> for Model { dispatcher_type: dispatcher.r#type().into(), dist_key_indices: dispatcher.dist_key_indices.into(), output_indices: dispatcher.output_indices.into(), - hash_mapping: dispatcher.hash_mapping.map(ActorMapping), + hash_mapping: dispatcher.hash_mapping.as_ref().map(ActorMapping::from), dispatcher_id: dispatcher.dispatcher_id as _, downstream_actor_ids: dispatcher.downstream_actor_id.into(), } @@ -74,7 +74,7 @@ impl From for PbDispatcher { r#type: PbDispatcherType::from(model.dispatcher_type) as _, dist_key_indices: model.dist_key_indices.into_u32_array(), output_indices: model.output_indices.into_u32_array(), - hash_mapping: model.hash_mapping.map(|mapping| mapping.into_inner()), + hash_mapping: model.hash_mapping.map(|mapping| mapping.to_protobuf()), dispatcher_id: model.dispatcher_id as _, downstream_actor_id: model.downstream_actor_ids.into_u32_array(), } diff --git a/src/meta/model_v2/src/compaction_config.rs b/src/meta/model_v2/src/compaction_config.rs index 39ce6bcef2d4..7d2612679717 100644 --- a/src/meta/model_v2/src/compaction_config.rs +++ b/src/meta/model_v2/src/compaction_config.rs @@ -14,7 +14,6 @@ use risingwave_pb::hummock::CompactionConfig as PbCompactionConfig; use sea_orm::entity::prelude::*; -use sea_orm::FromJsonQueryResult; use serde::{Deserialize, Serialize}; use crate::CompactionGroupId; @@ -32,4 +31,4 @@ pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} -crate::derive_from_json_struct!(CompactionConfig, PbCompactionConfig); +crate::derive_from_blob!(CompactionConfig, PbCompactionConfig); diff --git a/src/meta/model_v2/src/compaction_status.rs b/src/meta/model_v2/src/compaction_status.rs index 0e2afb84f9e9..79fbf8006258 100644 --- a/src/meta/model_v2/src/compaction_status.rs +++ b/src/meta/model_v2/src/compaction_status.rs @@ -14,8 +14,6 @@ use risingwave_pb::hummock::LevelHandler as PbLevelHandler; use sea_orm::entity::prelude::*; -use sea_orm::FromJsonQueryResult; -use serde::{Deserialize, Serialize}; use crate::CompactionGroupId; @@ -32,4 +30,4 @@ pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} -crate::derive_from_json_struct!(LevelHandlers, Vec); +crate::derive_array_from_blob!(LevelHandlers, PbLevelHandler, PbLevelHandlerArray); diff --git a/src/meta/model_v2/src/compaction_task.rs b/src/meta/model_v2/src/compaction_task.rs index 80d509d2299f..074fe9af450e 100644 --- a/src/meta/model_v2/src/compaction_task.rs +++ b/src/meta/model_v2/src/compaction_task.rs @@ -14,7 +14,6 @@ use risingwave_pb::hummock::{CompactTask as PbCompactTask, CompactTaskAssignment}; use sea_orm::entity::prelude::*; -use sea_orm::FromJsonQueryResult; use serde::{Deserialize, Serialize}; use crate::{CompactionTaskId, WorkerId}; @@ -33,12 +32,12 @@ pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} -crate::derive_from_json_struct!(CompactionTask, PbCompactTask); +crate::derive_from_blob!(CompactionTask, PbCompactTask); impl From for CompactTaskAssignment { fn from(value: Model) -> Self { Self { - compact_task: Some(value.task.0), + compact_task: Some(value.task.to_protobuf()), context_id: value.context_id as _, } } diff --git a/src/meta/model_v2/src/connection.rs b/src/meta/model_v2/src/connection.rs index fbd602434f15..0e513e7061fd 100644 --- a/src/meta/model_v2/src/connection.rs +++ b/src/meta/model_v2/src/connection.rs @@ -73,7 +73,7 @@ impl From for ActiveModel { Self { connection_id: Set(conn.id as _), name: Set(conn.name), - info: Set(PrivateLinkService(private_link_srv)), + info: Set(PrivateLinkService::from(&private_link_srv)), } } } diff --git a/src/meta/model_v2/src/fragment.rs b/src/meta/model_v2/src/fragment.rs index 27a589806459..af1d529a0598 100644 --- a/src/meta/model_v2/src/fragment.rs +++ b/src/meta/model_v2/src/fragment.rs @@ -12,13 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::Formatter; - use risingwave_pb::meta::table_fragments::fragment::PbFragmentDistributionType; -use risingwave_pb::stream_plan::PbStreamNode; use sea_orm::entity::prelude::*; -use crate::{FragmentId, FragmentVnodeMapping, I32Array, ObjectId}; +use crate::{FragmentId, FragmentVnodeMapping, I32Array, ObjectId, StreamNode}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "fragment")] @@ -34,34 +31,6 @@ pub struct Model { pub upstream_fragment_id: I32Array, } -/// This is a workaround to avoid stack overflow when deserializing the `StreamNode` field from sql -/// backend if we store it as Json. We'd better fix it before using it in production, because it's less -/// readable and maintainable. -#[derive(Clone, PartialEq, Eq, DeriveValueType)] -pub struct StreamNode(#[sea_orm] Vec); - -impl StreamNode { - pub fn to_protobuf(&self) -> PbStreamNode { - prost::Message::decode(self.0.as_slice()).unwrap() - } - - pub fn from_protobuf(val: &PbStreamNode) -> Self { - Self(prost::Message::encode_to_vec(val)) - } -} - -impl std::fmt::Debug for StreamNode { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - self.to_protobuf().fmt(f) - } -} - -impl Default for StreamNode { - fn default() -> Self { - Self::from_protobuf(&PbStreamNode::default()) - } -} - #[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum DistributionType { diff --git a/src/meta/model_v2/src/function.rs b/src/meta/model_v2/src/function.rs index eaf368aa15d1..e0d79f3bc557 100644 --- a/src/meta/model_v2/src/function.rs +++ b/src/meta/model_v2/src/function.rs @@ -95,8 +95,8 @@ impl From for ActiveModel { function_id: Set(function.id as _), name: Set(function.name), arg_names: Set(function.arg_names.join(",")), - arg_types: Set(DataTypeArray(function.arg_types)), - return_type: Set(DataType(function.return_type.unwrap())), + arg_types: Set(DataTypeArray::from(function.arg_types)), + return_type: Set(DataType::from(&function.return_type.unwrap())), language: Set(function.language), link: Set(function.link), identifier: Set(function.identifier), diff --git a/src/meta/model_v2/src/hummock_version_delta.rs b/src/meta/model_v2/src/hummock_version_delta.rs index f071034df434..ae69f5eca9a1 100644 --- a/src/meta/model_v2/src/hummock_version_delta.rs +++ b/src/meta/model_v2/src/hummock_version_delta.rs @@ -14,7 +14,6 @@ use risingwave_pb::hummock::PbHummockVersionDelta; use sea_orm::entity::prelude::*; -use sea_orm::FromJsonQueryResult; use serde::{Deserialize, Serialize}; use crate::{Epoch, HummockVersionId}; @@ -36,11 +35,11 @@ pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} -crate::derive_from_json_struct!(FullVersionDelta, PbHummockVersionDelta); +crate::derive_from_blob!(FullVersionDelta, PbHummockVersionDelta); impl From for PbHummockVersionDelta { fn from(value: Model) -> Self { - let ret = value.full_version_delta.into_inner(); + let ret = value.full_version_delta.to_protobuf(); assert_eq!(value.id, ret.id as i64); assert_eq!(value.prev_id, ret.prev_id as i64); assert_eq!(value.max_committed_epoch, ret.max_committed_epoch as i64); diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index 974cdf4fd267..a2eb99c16aed 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -16,6 +16,8 @@ use std::collections::{BTreeMap, HashMap}; use risingwave_pb::catalog::{PbCreateType, PbStreamJobStatus}; use risingwave_pb::meta::table_fragments::PbState as PbStreamJobState; +use risingwave_pb::stream_plan::PbStreamNode; +use sea_orm::entity::prelude::*; use sea_orm::{DeriveActiveEnum, EnumIter, FromJsonQueryResult}; use serde::{Deserialize, Serialize}; @@ -164,7 +166,95 @@ macro_rules! derive_from_json_struct { }; } -pub(crate) use derive_from_json_struct; +/// Defines struct with a byte array that derives `DeriveValueType`, it will helps to map blob stored in database to Pb struct. +macro_rules! derive_from_blob { + ($struct_name:ident, $field_type:ty) => { + #[derive(Clone, PartialEq, Eq, Serialize, Deserialize, DeriveValueType)] + pub struct $struct_name(#[sea_orm] Vec); + + impl $struct_name { + pub fn to_protobuf(&self) -> $field_type { + prost::Message::decode(self.0.as_slice()).unwrap() + } + + fn from_protobuf(val: &$field_type) -> Self { + Self(prost::Message::encode_to_vec(val)) + } + } + + impl sea_orm::sea_query::Nullable for $struct_name { + fn null() -> Value { + Value::Bytes(None) + } + } + + impl From<&$field_type> for $struct_name { + fn from(value: &$field_type) -> Self { + Self::from_protobuf(value) + } + } + + impl std::fmt::Debug for $struct_name { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.to_protobuf().fmt(f) + } + } + + impl Default for $struct_name { + fn default() -> Self { + Self::from_protobuf(&<$field_type>::default()) + } + } + }; +} + +/// Defines struct with a byte array that derives `DeriveValueType`, it will helps to map blob stored in database to Pb struct array. +macro_rules! derive_array_from_blob { + ($struct_name:ident, $field_type:ty, $field_array_name:ident) => { + #[derive(Clone, PartialEq, Eq, DeriveValueType)] + pub struct $struct_name(#[sea_orm] Vec); + + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct $field_array_name { + #[prost(message, repeated, tag = "1")] + inner: Vec<$field_type>, + } + impl Eq for $field_array_name {} + + impl $struct_name { + pub fn to_protobuf(&self) -> Vec<$field_type> { + let data: $field_array_name = prost::Message::decode(self.0.as_slice()).unwrap(); + data.inner + } + + fn from_protobuf(val: Vec<$field_type>) -> Self { + Self(prost::Message::encode_to_vec(&$field_array_name { + inner: val, + })) + } + } + + impl From> for $struct_name { + fn from(value: Vec<$field_type>) -> Self { + Self::from_protobuf(value) + } + } + + impl std::fmt::Debug for $struct_name { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.to_protobuf().fmt(f) + } + } + + impl Default for $struct_name { + fn default() -> Self { + Self(vec![]) + } + } + }; +} + +pub(crate) use {derive_array_from_blob, derive_from_blob, derive_from_json_struct}; derive_from_json_struct!(I32Array, Vec); @@ -192,38 +282,57 @@ impl From>> for ActorUpstreamActors { } } -derive_from_json_struct!(DataType, risingwave_pb::data::DataType); -derive_from_json_struct!(DataTypeArray, Vec); -derive_from_json_struct!(FieldArray, Vec); +derive_from_blob!(StreamNode, PbStreamNode); +derive_from_blob!(DataType, risingwave_pb::data::PbDataType); +derive_array_from_blob!( + DataTypeArray, + risingwave_pb::data::PbDataType, + PbDataTypeArray +); +derive_array_from_blob!( + FieldArray, + risingwave_pb::plan_common::PbField, + PbFieldArray +); derive_from_json_struct!(Property, HashMap); -derive_from_json_struct!(ColumnCatalog, risingwave_pb::plan_common::PbColumnCatalog); -derive_from_json_struct!( +derive_from_blob!(ColumnCatalog, risingwave_pb::plan_common::PbColumnCatalog); +derive_array_from_blob!( ColumnCatalogArray, - Vec + risingwave_pb::plan_common::PbColumnCatalog, + PbColumnCatalogArray ); -derive_from_json_struct!(StreamSourceInfo, risingwave_pb::catalog::PbStreamSourceInfo); -derive_from_json_struct!(WatermarkDesc, risingwave_pb::catalog::PbWatermarkDesc); -derive_from_json_struct!( +derive_from_blob!(StreamSourceInfo, risingwave_pb::catalog::PbStreamSourceInfo); +derive_from_blob!(WatermarkDesc, risingwave_pb::catalog::PbWatermarkDesc); +derive_array_from_blob!( WatermarkDescArray, - Vec + risingwave_pb::catalog::PbWatermarkDesc, + PbWatermarkDescArray +); +derive_array_from_blob!( + ExprNodeArray, + risingwave_pb::expr::PbExprNode, + PbExprNodeArray +); +derive_array_from_blob!( + ColumnOrderArray, + risingwave_pb::common::PbColumnOrder, + PbColumnOrderArray ); -derive_from_json_struct!(ExprNodeArray, Vec); -derive_from_json_struct!(ColumnOrderArray, Vec); -derive_from_json_struct!(SinkFormatDesc, risingwave_pb::catalog::PbSinkFormatDesc); -derive_from_json_struct!(Cardinality, risingwave_pb::plan_common::PbCardinality); -derive_from_json_struct!(TableVersion, risingwave_pb::catalog::table::PbTableVersion); -derive_from_json_struct!( +derive_from_blob!(SinkFormatDesc, risingwave_pb::catalog::PbSinkFormatDesc); +derive_from_blob!(Cardinality, risingwave_pb::plan_common::PbCardinality); +derive_from_blob!(TableVersion, risingwave_pb::catalog::table::PbTableVersion); +derive_from_blob!( PrivateLinkService, risingwave_pb::catalog::connection::PbPrivateLinkService ); -derive_from_json_struct!(AuthInfo, risingwave_pb::user::PbAuthInfo); +derive_from_blob!(AuthInfo, risingwave_pb::user::PbAuthInfo); -derive_from_json_struct!(ConnectorSplits, risingwave_pb::source::ConnectorSplits); -derive_from_json_struct!(VnodeBitmap, risingwave_pb::common::Buffer); -derive_from_json_struct!(ActorMapping, risingwave_pb::stream_plan::PbActorMapping); -derive_from_json_struct!(ExprContext, risingwave_pb::plan_common::PbExprContext); +derive_from_blob!(ConnectorSplits, risingwave_pb::source::ConnectorSplits); +derive_from_blob!(VnodeBitmap, risingwave_pb::common::Buffer); +derive_from_blob!(ActorMapping, risingwave_pb::stream_plan::PbActorMapping); +derive_from_blob!(ExprContext, risingwave_pb::plan_common::PbExprContext); -derive_from_json_struct!( +derive_from_blob!( FragmentVnodeMapping, risingwave_pb::common::ParallelUnitMapping ); diff --git a/src/meta/model_v2/src/sink.rs b/src/meta/model_v2/src/sink.rs index f5fa59c85ff5..ab7e869daee6 100644 --- a/src/meta/model_v2/src/sink.rs +++ b/src/meta/model_v2/src/sink.rs @@ -124,7 +124,7 @@ impl From for ActiveModel { connection_id: Set(pb_sink.connection_id.map(|x| x as _)), db_name: Set(pb_sink.db_name), sink_from_name: Set(pb_sink.sink_from_name), - sink_format_desc: Set(pb_sink.format_desc.map(|x| x.into())), + sink_format_desc: Set(pb_sink.format_desc.as_ref().map(|x| x.into())), target_table: Set(pb_sink.target_table.map(|x| x as _)), } } diff --git a/src/meta/model_v2/src/source.rs b/src/meta/model_v2/src/source.rs index b28dcd30ab13..2b0e511e4afe 100644 --- a/src/meta/model_v2/src/source.rs +++ b/src/meta/model_v2/src/source.rs @@ -91,12 +91,12 @@ impl From for ActiveModel { source_id: Set(source.id as _), name: Set(source.name), row_id_index: Set(source.row_id_index.map(|x| x as _)), - columns: Set(ColumnCatalogArray(source.columns)), + columns: Set(ColumnCatalogArray::from(source.columns)), pk_column_ids: Set(I32Array(source.pk_column_ids)), with_properties: Set(Property(source.with_properties)), definition: Set(source.definition), - source_info: Set(source.info.map(StreamSourceInfo)), - watermark_descs: Set(WatermarkDescArray(source.watermark_descs)), + source_info: Set(source.info.as_ref().map(StreamSourceInfo::from)), + watermark_descs: Set(WatermarkDescArray::from(source.watermark_descs)), optional_associated_table_id: Set(optional_associated_table_id), connection_id: Set(source.connection_id.map(|id| id as _)), version: Set(source.version as _), diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index 1ea78221d852..974f1c8defac 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -234,10 +234,10 @@ impl From for ActiveModel { watermark_indices: Set(pb_table.watermark_indices.into()), dist_key_in_pk: Set(pb_table.dist_key_in_pk.into()), dml_fragment_id, - cardinality: Set(pb_table.cardinality.map(|x| x.into())), + cardinality: Set(pb_table.cardinality.as_ref().map(|x| x.into())), cleaned_by_watermark: Set(pb_table.cleaned_by_watermark), description: Set(pb_table.description), - version: Set(pb_table.version.map(|v| v.into())), + version: Set(pb_table.version.as_ref().map(|v| v.into())), retention_seconds: Set(pb_table.retention_seconds.map(|i| i as _)), incoming_sinks: Set(pb_table.incoming_sinks.into()), } diff --git a/src/meta/model_v2/src/user.rs b/src/meta/model_v2/src/user.rs index c92e219a9b61..f238683a83b0 100644 --- a/src/meta/model_v2/src/user.rs +++ b/src/meta/model_v2/src/user.rs @@ -61,7 +61,7 @@ impl From for ActiveModel { can_create_db: Set(user.can_create_db), can_create_user: Set(user.can_create_user), can_login: Set(user.can_login), - auth_info: Set(user.auth_info.map(AuthInfo)), + auth_info: Set(user.auth_info.as_ref().map(AuthInfo::from)), } } } @@ -75,7 +75,7 @@ impl From for PbUserInfo { can_create_db: val.can_create_db, can_create_user: val.can_create_user, can_login: val.can_login, - auth_info: val.auth_info.map(|x| x.into_inner()), + auth_info: val.auth_info.map(|x| x.to_protobuf()), grant_privileges: vec![], // fill in later } } diff --git a/src/meta/model_v2/src/view.rs b/src/meta/model_v2/src/view.rs index 8b0f52f022ab..0e32cd227515 100644 --- a/src/meta/model_v2/src/view.rs +++ b/src/meta/model_v2/src/view.rs @@ -56,7 +56,7 @@ impl From for ActiveModel { name: Set(view.name), properties: Set(Property(view.properties)), definition: Set(view.sql), - columns: Set(FieldArray(view.columns)), + columns: Set(view.columns.into()), } } } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 559bfac5bc9d..bddc82d372d6 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -22,7 +22,6 @@ use risingwave_common::catalog::{TableOption, DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMA use risingwave_common::util::stream_graph_visitor::visit_stream_node_cont; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::source::UPSTREAM_SOURCE_KEY; -use risingwave_meta_model_v2::fragment::StreamNode; use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::table::TableType; @@ -31,7 +30,7 @@ use risingwave_meta_model_v2::{ sink, source, streaming_job, subscription, table, user_privilege, view, ActorId, ActorUpstreamActors, ColumnCatalogArray, ConnectionId, CreateType, DatabaseId, FragmentId, FunctionId, I32Array, IndexId, JobStatus, ObjectId, PrivateLinkService, Property, SchemaId, - SinkId, SourceId, StreamSourceInfo, StreamingParallelism, TableId, UserId, + SinkId, SourceId, StreamNode, StreamSourceInfo, StreamingParallelism, TableId, UserId, }; use risingwave_pb::catalog::table::PbTableType; use risingwave_pb::catalog::{ @@ -763,7 +762,7 @@ impl CatalogController { ) .col_expr( fragment::Column::StreamNode, - StreamNode::from_protobuf(&pb_stream_node).into(), + StreamNode::from(&pb_stream_node).into(), ) .filter(fragment::Column::FragmentId.eq(fragment_id)) .exec(txn) @@ -1824,15 +1823,16 @@ impl CatalogController { .ok_or_else(|| MetaError::catalog_id_not_found("table", comment.table_id))?; let table = if let Some(col_idx) = comment.column_index { - let mut columns: ColumnCatalogArray = Table::find_by_id(comment.table_id as TableId) + let columns: ColumnCatalogArray = Table::find_by_id(comment.table_id as TableId) .select_only() .column(table::Column::Columns) .into_tuple() .one(&txn) .await? .ok_or_else(|| MetaError::catalog_id_not_found("table", comment.table_id))?; - let column = columns - .0 + let mut pb_columns = columns.to_protobuf(); + + let column = pb_columns .get_mut(col_idx as usize) .ok_or_else(|| MetaError::catalog_id_not_found("column", col_idx))?; let column_desc = column.column_desc.as_mut().ok_or_else(|| { @@ -1845,7 +1845,7 @@ impl CatalogController { column_desc.description = comment.description; table::ActiveModel { table_id: Set(comment.table_id as _), - columns: Set(columns), + columns: Set(pb_columns.into()), ..Default::default() } .update(&txn) @@ -1986,7 +1986,7 @@ impl CatalogController { .await? .ok_or_else(|| MetaError::catalog_id_not_found("source", object_id))?; if let Some(source_info) = source_info - && source_info.into_inner().is_shared() + && source_info.to_protobuf().is_shared() { to_drop_streaming_jobs.push(object_id); } @@ -3177,8 +3177,8 @@ mod tests { .one(&mgr.inner.read().await.db) .await? .unwrap(); - assert_eq!(function.return_type.0, test_data_type); - assert_eq!(function.arg_types.into_inner().len(), 1); + assert_eq!(function.return_type.to_protobuf(), test_data_type); + assert_eq!(function.arg_types.to_protobuf().len(), 1); assert_eq!(function.language, "python"); mgr.drop_function(function.function_id).await?; diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 71142b829089..552008914d76 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -20,12 +20,11 @@ use itertools::Itertools; use risingwave_common::bail; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_meta_model_v2::actor::ActorStatus; -use risingwave_meta_model_v2::fragment::StreamNode; use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, Sink, StreamingJob}; use risingwave_meta_model_v2::{ actor, actor_dispatcher, fragment, object, sink, streaming_job, ActorId, ActorUpstreamActors, ConnectorSplits, ExprContext, FragmentId, FragmentVnodeMapping, I32Array, JobStatus, ObjectId, - SinkId, SourceId, StreamingParallelism, TableId, VnodeBitmap, WorkerId, + SinkId, SourceId, StreamNode, StreamingParallelism, TableId, VnodeBitmap, WorkerId, }; use risingwave_pb::common::PbParallelUnit; use risingwave_pb::meta::subscribe_response::{ @@ -75,7 +74,7 @@ impl CatalogControllerInner { Ok(fragment_mappings.into_iter().map(|(fragment_id, mapping)| { FragmentParallelUnitMapping { fragment_id: fragment_id as _, - mapping: Some(mapping.into_inner()), + mapping: Some(mapping.to_protobuf()), } })) } @@ -229,7 +228,7 @@ impl CatalogController { expr_context: pb_expr_context, } = actor; - let splits = pb_actor_splits.get(&actor_id).cloned().map(ConnectorSplits); + let splits = pb_actor_splits.get(&actor_id).map(ConnectorSplits::from); let status = pb_actor_status.get(&actor_id).cloned().ok_or_else(|| { anyhow::anyhow!( "actor {} in fragment {} has no actor_status", @@ -265,8 +264,8 @@ impl CatalogController { parallel_unit_id, worker_id, upstream_actor_ids: upstream_actors.into(), - vnode_bitmap: pb_vnode_bitmap.map(VnodeBitmap), - expr_context: ExprContext(pb_expr_context), + vnode_bitmap: pb_vnode_bitmap.as_ref().map(VnodeBitmap::from), + expr_context: ExprContext::from(&pb_expr_context), }); actor_dispatchers.insert( actor_id as ActorId, @@ -279,9 +278,12 @@ impl CatalogController { let upstream_fragment_id = pb_upstream_fragment_ids.into(); - let vnode_mapping = pb_vnode_mapping.map(FragmentVnodeMapping).unwrap(); + let vnode_mapping = pb_vnode_mapping + .as_ref() + .map(FragmentVnodeMapping::from) + .unwrap(); - let stream_node = StreamNode::from_protobuf(&stream_node); + let stream_node = StreamNode::from(&stream_node); let distribution_type = PbFragmentDistributionType::try_from(pb_distribution_type) .unwrap() @@ -415,8 +417,8 @@ impl CatalogController { Some(nodes) }; - let pb_vnode_bitmap = vnode_bitmap.map(|vnode_bitmap| vnode_bitmap.into_inner()); - let pb_expr_context = Some(expr_context.into_inner()); + let pb_vnode_bitmap = vnode_bitmap.map(|vnode_bitmap| vnode_bitmap.to_protobuf()); + let pb_expr_context = Some(expr_context.to_protobuf()); let pb_upstream_actor_id = upstream_fragment_actors .values() @@ -443,7 +445,7 @@ impl CatalogController { ); if let Some(splits) = splits { - pb_actor_splits.insert(actor_id as _, splits.into_inner()); + pb_actor_splits.insert(actor_id as _, splits.to_protobuf()); } pb_actors.push(PbStreamActor { @@ -459,7 +461,7 @@ impl CatalogController { } let pb_upstream_fragment_ids = upstream_fragment_id.into_u32_array(); - let pb_vnode_mapping = vnode_mapping.into_inner(); + let pb_vnode_mapping = vnode_mapping.to_protobuf(); let pb_state_table_ids = state_table_ids.into_u32_array(); let pb_distribution_type = PbFragmentDistributionType::from(distribution_type) as _; let pb_fragment = PbFragment { @@ -908,7 +910,7 @@ impl CatalogController { .await?; } - let mut fragment_mapping: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() + let fragment_mapping: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() .select_only() .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) .join(JoinType::InnerJoin, fragment::Relation::Actor.def()) @@ -917,15 +919,17 @@ impl CatalogController { .all(&txn) .await?; // TODO: we'd better not store vnode mapping in fragment table and derive it from actors. - for (fragment_id, vnode_mapping) in &mut fragment_mapping { - vnode_mapping.0.data.iter_mut().for_each(|id| { + + for (fragment_id, vnode_mapping) in &fragment_mapping { + let mut pb_vnode_mapping = vnode_mapping.to_protobuf(); + pb_vnode_mapping.data.iter_mut().for_each(|id| { if let Some(new_id) = plan.get(&(*id as i32)) { *id = new_id.id; } }); fragment::ActiveModel { fragment_id: Set(*fragment_id), - vnode_mapping: Set(vnode_mapping.clone()), + vnode_mapping: Set(FragmentVnodeMapping::from(&pb_vnode_mapping)), ..Default::default() } .update(&txn) @@ -940,7 +944,7 @@ impl CatalogController { .into_iter() .map(|(fragment_id, mapping)| PbFragmentParallelUnitMapping { fragment_id: fragment_id as _, - mapping: Some(mapping.into_inner()), + mapping: Some(mapping.to_protobuf()), }) .collect(), ) @@ -1053,13 +1057,13 @@ impl CatalogController { .ok_or_else(|| MetaError::catalog_id_not_found("actor_id", actor_id))?; let mut actor_splits = actor_splits - .map(|splits| splits.0.splits) + .map(|splits| splits.to_protobuf().splits) .unwrap_or_default(); actor_splits.extend(splits.iter().map(Into::into)); Actor::update(actor::ActiveModel { actor_id: Set(*actor_id as _), - splits: Set(Some(ConnectorSplits(PbConnectorSplits { + splits: Set(Some(ConnectorSplits::from(&PbConnectorSplits { splits: actor_splits, }))), ..Default::default() @@ -1349,10 +1353,11 @@ mod tests { use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_meta_model_v2::actor::ActorStatus; - use risingwave_meta_model_v2::fragment::{DistributionType, StreamNode}; + use risingwave_meta_model_v2::fragment::DistributionType; use risingwave_meta_model_v2::{ actor, actor_dispatcher, fragment, ActorId, ActorUpstreamActors, ConnectorSplits, - ExprContext, FragmentId, FragmentVnodeMapping, I32Array, ObjectId, TableId, VnodeBitmap, + ExprContext, FragmentId, FragmentVnodeMapping, I32Array, ObjectId, StreamNode, TableId, + VnodeBitmap, }; use risingwave_pb::common::ParallelUnit; use risingwave_pb::meta::table_fragments::actor_status::PbActorState; @@ -1564,9 +1569,9 @@ mod tests { let vnode_bitmap = actor_vnode_bitmaps .remove(¶llel_unit_id) - .map(|m| VnodeBitmap(m.to_protobuf())); + .map(|m| VnodeBitmap::from(&m.to_protobuf())); - let actor_splits = Some(ConnectorSplits(PbConnectorSplits { + let actor_splits = Some(ConnectorSplits::from(&PbConnectorSplits { splits: vec![PbConnectorSplit { split_type: "dummy".to_string(), ..Default::default() @@ -1585,7 +1590,7 @@ mod tests { worker_id: 0, upstream_actor_ids: ActorUpstreamActors(actor_upstream_actor_ids), vnode_bitmap, - expr_context: ExprContext(PbExprContext { + expr_context: ExprContext::from(&PbExprContext { time_zone: String::from("America/New_York"), }), } @@ -1623,8 +1628,8 @@ mod tests { job_id: TEST_JOB_ID, fragment_type_mask: 0, distribution_type: DistributionType::Hash, - stream_node: StreamNode::from_protobuf(&stream_node), - vnode_mapping: FragmentVnodeMapping(parallel_unit_mapping.to_protobuf()), + stream_node: StreamNode::from(&stream_node), + vnode_mapping: FragmentVnodeMapping::from(¶llel_unit_mapping.to_protobuf()), state_table_ids: I32Array(vec![TEST_STATE_TABLE_ID]), upstream_fragment_id: I32Array::default(), }; @@ -1709,10 +1714,8 @@ mod tests { assert_eq!(actor_dispatcher, pb_dispatcher); assert_eq!( - vnode_bitmap, - pb_vnode_bitmap - .as_ref() - .map(|bitmap| VnodeBitmap(bitmap.clone())) + vnode_bitmap.map(|bitmap| bitmap.to_protobuf()), + pb_vnode_bitmap, ); assert_eq!(mview_definition, ""); @@ -1733,13 +1736,10 @@ mod tests { assert_eq!( splits, - pb_actor_splits - .get(&pb_actor_id) - .cloned() - .map(ConnectorSplits) + pb_actor_splits.get(&pb_actor_id).map(ConnectorSplits::from) ); - assert_eq!(Some(expr_context.into_inner()), pb_expr_context); + assert_eq!(Some(expr_context.to_protobuf()), pb_expr_context); } } @@ -1761,8 +1761,8 @@ mod tests { PbFragmentDistributionType::from(fragment.distribution_type) as i32 ); assert_eq!( - pb_vnode_mapping.map(FragmentVnodeMapping).unwrap(), - fragment.vnode_mapping + pb_vnode_mapping.unwrap(), + fragment.vnode_mapping.to_protobuf() ); assert_eq!( diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index b637d871fd84..3d43c77d655a 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -98,8 +98,8 @@ impl From> for PbTable { schema_id: value.1.schema_id.unwrap() as _, database_id: value.1.database_id.unwrap() as _, name: value.0.name, - columns: value.0.columns.0, - pk: value.0.pk.0, + columns: value.0.columns.to_protobuf(), + pk: value.0.pk.to_protobuf(), dependent_relations: vec![], // todo: deprecate it. table_type: PbTableType::from(value.0.table_type) as _, distribution_key: value.0.distribution_key.0, @@ -119,7 +119,10 @@ impl From> for PbTable { watermark_indices: value.0.watermark_indices.0, dist_key_in_pk: value.0.dist_key_in_pk.0, dml_fragment_id: value.0.dml_fragment_id.map(|id| id as u32), - cardinality: value.0.cardinality.map(|cardinality| cardinality.0), + cardinality: value + .0 + .cardinality + .map(|cardinality| cardinality.to_protobuf()), initialized_at_epoch: Some( Epoch::from_unix_millis(value.1.initialized_at.timestamp_millis() as _).0, ), @@ -129,7 +132,7 @@ impl From> for PbTable { cleaned_by_watermark: value.0.cleaned_by_watermark, stream_job_status: PbStreamJobStatus::Created as _, // todo: deprecate it. create_type: PbCreateType::Foreground as _, - version: value.0.version.map(|v| v.into_inner()), + version: value.0.version.map(|v| v.to_protobuf()), optional_associated_source_id: value .0 .optional_associated_source_id @@ -151,12 +154,12 @@ impl From> for PbSource { database_id: value.1.database_id.unwrap() as _, name: value.0.name, row_id_index: value.0.row_id_index.map(|id| id as _), - columns: value.0.columns.0, + columns: value.0.columns.to_protobuf(), pk_column_ids: value.0.pk_column_ids.0, with_properties: value.0.with_properties.0, owner: value.1.owner_id as _, - info: value.0.source_info.map(|info| info.0), - watermark_descs: value.0.watermark_descs.0, + info: value.0.source_info.map(|info| info.to_protobuf()), + watermark_descs: value.0.watermark_descs.to_protobuf(), definition: value.0.definition, connection_id: value.0.connection_id.map(|id| id as _), // todo: using the timestamp from the database directly. @@ -184,8 +187,8 @@ impl From> for PbSink { schema_id: value.1.schema_id.unwrap() as _, database_id: value.1.database_id.unwrap() as _, name: value.0.name, - columns: value.0.columns.0, - plan_pk: value.0.plan_pk.0, + columns: value.0.columns.to_protobuf(), + plan_pk: value.0.plan_pk.to_protobuf(), dependent_relations: vec![], // todo: deprecate it. distribution_key: value.0.distribution_key.0, downstream_pk: value.0.downstream_pk.0, @@ -203,7 +206,7 @@ impl From> for PbSink { db_name: value.0.db_name, sink_from_name: value.0.sink_from_name, stream_job_status: PbStreamJobStatus::Created as _, // todo: deprecate it. - format_desc: value.0.sink_format_desc.map(|desc| desc.0), + format_desc: value.0.sink_format_desc.map(|desc| desc.to_protobuf()), target_table: value.0.target_table.map(|id| id as _), initialized_at_cluster_version: value.1.initialized_at_cluster_version, created_at_cluster_version: value.1.created_at_cluster_version, @@ -219,7 +222,7 @@ impl From> for PbSubscription { schema_id: value.1.schema_id.unwrap() as _, database_id: value.1.database_id.unwrap() as _, name: value.0.name, - plan_pk: value.0.plan_pk.0, + plan_pk: value.0.plan_pk.to_protobuf(), dependent_relations: vec![], // todo: deprecate it. distribution_key: value.0.distribution_key.0, owner: value.1.owner_id as _, @@ -232,7 +235,7 @@ impl From> for PbSubscription { Epoch::from_unix_millis(value.1.created_at.timestamp_millis() as _).0, ), stream_job_status: PbStreamJobStatus::Created as _, // todo: deprecate it. - column_catalogs: value.0.columns.0, + column_catalogs: value.0.columns.to_protobuf(), subscription_from_name: value.0.subscription_from_name, initialized_at_cluster_version: value.1.initialized_at_cluster_version, created_at_cluster_version: value.1.created_at_cluster_version, @@ -250,7 +253,7 @@ impl From> for PbIndex { owner: value.1.owner_id as _, index_table_id: value.0.index_table_id as _, primary_table_id: value.0.primary_table_id as _, - index_item: value.0.index_items.0, + index_item: value.0.index_items.to_protobuf(), index_columns_len: value.0.index_columns_len as _, initialized_at_epoch: Some( Epoch::from_unix_millis(value.1.initialized_at.timestamp_millis() as _).0, @@ -276,7 +279,7 @@ impl From> for PbView { properties: value.0.properties.0, sql: value.0.definition, dependent_relations: vec![], // todo: deprecate it. - columns: value.0.columns.0, + columns: value.0.columns.to_protobuf(), } } } @@ -289,7 +292,9 @@ impl From> for PbConnection { database_id: value.1.database_id.unwrap() as _, name: value.0.name, owner: value.1.owner_id as _, - info: Some(PbConnectionInfo::PrivateLinkService(value.0.info.0)), + info: Some(PbConnectionInfo::PrivateLinkService( + value.0.info.to_protobuf(), + )), } } } @@ -308,8 +313,8 @@ impl From> for PbFunction { .split(',') .map(|s| s.to_string()) .collect(), - arg_types: value.0.arg_types.into_inner(), - return_type: Some(value.0.return_type.into_inner()), + arg_types: value.0.arg_types.to_protobuf(), + return_type: Some(value.0.return_type.to_protobuf()), language: value.0.language, link: value.0.link, identifier: value.0.identifier, diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index d9a98d188f04..68bbc5fd025d 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -23,7 +23,6 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_meta_model_v2::actor::ActorStatus; use risingwave_meta_model_v2::actor_dispatcher::DispatcherType; -use risingwave_meta_model_v2::fragment::StreamNode; use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::{ Actor, ActorDispatcher, Fragment, Index, Object, ObjectDependency, Sink, Source, @@ -33,7 +32,7 @@ use risingwave_meta_model_v2::{ actor, actor_dispatcher, fragment, index, object, object_dependency, sink, source, streaming_job, subscription, table, ActorId, ActorUpstreamActors, CreateType, DatabaseId, ExprNodeArray, FragmentId, I32Array, IndexId, JobStatus, ObjectId, SchemaId, SourceId, - StreamingParallelism, TableId, TableVersion, UserId, + StreamNode, StreamingParallelism, TableId, TableVersion, UserId, }; use risingwave_pb::catalog::source::PbOptionalAssociatedTableId; use risingwave_pb::catalog::table::{PbOptionalAssociatedSourceId, PbTableVersion}; @@ -492,7 +491,7 @@ impl CatalogController { for splits in split_assignment.values() { for (actor_id, splits) in splits { let splits = splits.iter().map(PbConnectorSplit::from).collect_vec(); - let connector_splits = PbConnectorSplits { splits }; + let connector_splits = &PbConnectorSplits { splits }; actor::ActiveModel { actor_id: Set(*actor_id as _), splits: Set(Some(connector_splits.into())), @@ -553,7 +552,7 @@ impl CatalogController { .await? .ok_or_else(|| MetaError::catalog_id_not_found(ObjectType::Table.as_str(), id))?; let original_version = original_version.expect("version for table should exist"); - if version.version != original_version.inner_ref().version + 1 { + if version.version != original_version.to_protobuf().version + 1 { return Err(MetaError::permission_denied("table version is stale")); } @@ -745,7 +744,7 @@ impl CatalogController { } fragment::ActiveModel { fragment_id: Set(fragment_id), - stream_node: Set(StreamNode::from_protobuf(&stream_node)), + stream_node: Set(StreamNode::from(&stream_node)), upstream_fragment_id: Set(upstream_fragment_id), ..Default::default() } @@ -778,14 +777,14 @@ impl CatalogController { .into_tuple() .all(&txn) .await?; - for (index_id, mut nodes) in index_items { - nodes - .0 + for (index_id, nodes) in index_items { + let mut pb_nodes = nodes.to_protobuf(); + pb_nodes .iter_mut() .for_each(|x| expr_rewriter.rewrite_expr(x)); let index = index::ActiveModel { index_id: Set(index_id), - index_items: Set(nodes), + index_items: Set(pb_nodes.into()), ..Default::default() } .update(&txn) @@ -850,7 +849,7 @@ impl CatalogController { if let Some(table_id) = source.optional_associated_table_id { vec![table_id] } else if let Some(source_info) = &source.source_info - && source_info.inner_ref().is_shared() + && source_info.to_protobuf().is_shared() { vec![source_id] } else { @@ -911,7 +910,7 @@ impl CatalogController { for (id, _, stream_node) in fragments { fragment::ActiveModel { fragment_id: Set(id), - stream_node: Set(StreamNode::from_protobuf(&stream_node)), + stream_node: Set(StreamNode::from(&stream_node)), ..Default::default() } .update(&txn) @@ -972,7 +971,7 @@ impl CatalogController { for (id, _, stream_node) in fragments { fragment::ActiveModel { fragment_id: Set(id), - stream_node: Set(StreamNode::from_protobuf(&stream_node)), + stream_node: Set(StreamNode::from(&stream_node)), ..Default::default() } .update(&txn) @@ -1119,12 +1118,12 @@ impl CatalogController { actor_id: Set(actor_id as _), fragment_id: Set(fragment_id as _), status: Set(ActorStatus::Running), - splits: Set(splits.map(|splits| PbConnectorSplits { splits }.into())), + splits: Set(splits.map(|splits| (&PbConnectorSplits { splits }).into())), parallel_unit_id: Set(parallel_unit.id as _), worker_id: Set(parallel_unit.worker_node_id as _), upstream_actor_ids: Set(actor_upstreams), - vnode_bitmap: Set(vnode_bitmap.map(|bitmap| bitmap.into())), - expr_context: Set(expr_context.unwrap().into()), + vnode_bitmap: Set(vnode_bitmap.as_ref().map(|bitmap| bitmap.into())), + expr_context: Set(expr_context.as_ref().unwrap().into()), }); for PbDispatcher { @@ -1144,7 +1143,7 @@ impl CatalogController { .into()), dist_key_indices: Set(dist_key_indices.into()), output_indices: Set(output_indices.into()), - hash_mapping: Set(hash_mapping.map(|mapping| mapping.into())), + hash_mapping: Set(hash_mapping.as_ref().map(|mapping| mapping.into())), dispatcher_id: Set(dispatcher_id as _), downstream_actor_ids: Set(downstream_actor_id.into()), }) @@ -1169,7 +1168,7 @@ impl CatalogController { .ok_or_else(|| MetaError::catalog_id_not_found("actor", actor_id))?; let mut actor = actor.into_active_model(); - actor.vnode_bitmap = Set(Some(bitmap.to_protobuf().into())); + actor.vnode_bitmap = Set(Some((&bitmap.to_protobuf()).into())); actor.update(&txn).await?; } @@ -1186,7 +1185,7 @@ impl CatalogController { for actor in &fragment_actors { actor_to_parallel_unit.insert(actor.actor_id as u32, actor.parallel_unit_id as _); if let Some(vnode_bitmap) = &actor.vnode_bitmap { - let bitmap = Bitmap::from(vnode_bitmap.inner_ref()); + let bitmap = Bitmap::from(&vnode_bitmap.to_protobuf()); actor_to_vnode_bitmap.insert(actor.actor_id as u32, bitmap); } } @@ -1203,7 +1202,7 @@ impl CatalogController { .to_protobuf(); let mut fragment = fragment.into_active_model(); - fragment.vnode_mapping = Set(vnode_mapping.clone().into()); + fragment.vnode_mapping = Set((&vnode_mapping).into()); fragment.update(&txn).await?; fragment_mapping_to_notify.push(FragmentParallelUnitMapping { @@ -1248,7 +1247,7 @@ impl CatalogController { if dispatcher.dispatcher_type.as_ref() == &DispatcherType::Hash { dispatcher.hash_mapping = Set(upstream_dispatcher_mapping.as_ref().map(|m| { - risingwave_meta_model_v2::ActorMapping::from(m.to_protobuf()) + risingwave_meta_model_v2::ActorMapping::from(&m.to_protobuf()) })); } else { debug_assert!(upstream_dispatcher_mapping.is_none()); diff --git a/src/meta/src/controller/user.rs b/src/meta/src/controller/user.rs index 214ae971ecc9..f6afedf875f5 100644 --- a/src/meta/src/controller/user.rs +++ b/src/meta/src/controller/user.rs @@ -119,7 +119,7 @@ impl CatalogController { PbUpdateField::CreateDb => user.can_create_db = Set(update_user.can_create_db), PbUpdateField::CreateUser => user.can_create_user = Set(update_user.can_create_user), PbUpdateField::AuthInfo => { - user.auth_info = Set(update_user.auth_info.clone().map(AuthInfo)) + user.auth_info = Set(update_user.auth_info.as_ref().map(AuthInfo::from)) } PbUpdateField::Rename => user.name = Set(update_user.name.clone()), }); diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index dad880bbad77..173634ab03cc 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -18,14 +18,14 @@ use anyhow::anyhow; use itertools::Itertools; use risingwave_meta_model_migration::WithQuery; use risingwave_meta_model_v2::actor::ActorStatus; -use risingwave_meta_model_v2::fragment::{DistributionType, StreamNode}; +use risingwave_meta_model_v2::fragment::DistributionType; use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ actor, actor_dispatcher, connection, database, fragment, function, index, object, object_dependency, schema, sink, source, table, user, user_privilege, view, ActorId, DataTypeArray, DatabaseId, FragmentId, FragmentVnodeMapping, I32Array, ObjectId, PrivilegeId, - SchemaId, SourceId, UserId, + SchemaId, SourceId, StreamNode, UserId, }; use risingwave_pb::catalog::{PbConnection, PbFunction}; use risingwave_pb::meta::PbFragmentParallelUnitMapping; @@ -368,7 +368,10 @@ where .eq(pb_function.database_id as DatabaseId) .and(object::Column::SchemaId.eq(pb_function.schema_id as SchemaId)) .and(function::Column::Name.eq(&pb_function.name)) - .and(function::Column::ArgTypes.eq(DataTypeArray(pb_function.arg_types.clone()))), + .and( + function::Column::ArgTypes + .eq(DataTypeArray::from(pb_function.arg_types.clone())), + ), ) .count(db) .await?; @@ -802,7 +805,7 @@ where .into_iter() .map(|(fragment_id, mapping)| PbFragmentParallelUnitMapping { fragment_id: fragment_id as _, - mapping: Some(mapping.into_inner()), + mapping: Some(mapping.to_protobuf()), }) .collect()) } @@ -831,7 +834,7 @@ where .into_iter() .map(|(fragment_id, mapping)| PbFragmentParallelUnitMapping { fragment_id: fragment_id as _, - mapping: Some(mapping.into_inner()), + mapping: Some(mapping.to_protobuf()), }) .collect()) } diff --git a/src/meta/src/hummock/model/ext/hummock.rs b/src/meta/src/hummock/model/ext/hummock.rs index b7d136776a95..5cd1eb08402e 100644 --- a/src/meta/src/hummock/model/ext/hummock.rs +++ b/src/meta/src/hummock/model/ext/hummock.rs @@ -51,7 +51,7 @@ impl Transactional for CompactionGroup { async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { let m = compaction_config::ActiveModel { compaction_group_id: Set(self.group_id as _), - config: Set(CompactionConfig((*self.compaction_config).to_owned())), + config: Set(CompactionConfig::from(&(*self.compaction_config))), }; compaction_config::Entity::insert(m) .on_conflict( @@ -77,8 +77,8 @@ impl Transactional for CompactStatus { async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { let m = compaction_status::ActiveModel { compaction_group_id: Set(self.compaction_group_id as _), - status: Set(LevelHandlers( - self.level_handlers.iter().map_into().collect(), + status: Set(LevelHandlers::from( + self.level_handlers.iter().map_into().collect_vec(), )), }; compaction_status::Entity::insert(m) @@ -107,7 +107,7 @@ impl Transactional for CompactTaskAssignment { let m = compaction_task::ActiveModel { id: Set(task.task_id as _), context_id: Set(self.context_id as _), - task: Set(CompactionTask(task)), + task: Set(CompactionTask::from(&task)), }; compaction_task::Entity::insert(m) .on_conflict( @@ -220,7 +220,7 @@ impl Transactional for HummockVersionDelta { max_committed_epoch: Set(self.max_committed_epoch as _), safe_epoch: Set(self.safe_epoch as _), trivial_move: Set(self.trivial_move), - full_version_delta: Set(FullVersionDelta(self.to_protobuf())), + full_version_delta: Set(FullVersionDelta::from(&self.to_protobuf())), }; hummock_version_delta::Entity::insert(m) .on_conflict( @@ -249,7 +249,7 @@ impl Transactional for HummockVersionDelta { impl From for CompactionGroup { fn from(value: compaction_config::Model) -> Self { - Self::new(value.compaction_group_id as _, value.config.0) + Self::new(value.compaction_group_id as _, value.config.to_protobuf()) } } @@ -257,7 +257,7 @@ impl From for CompactStatus { fn from(value: compaction_status::Model) -> Self { Self { compaction_group_id: value.compaction_group_id as _, - level_handlers: value.status.0.iter().map_into().collect(), + level_handlers: value.status.to_protobuf().iter().map_into().collect(), } } } diff --git a/src/meta/src/rpc/ddl_controller_v2.rs b/src/meta/src/rpc/ddl_controller_v2.rs index 47d82d142046..c0ab2f13f2ff 100644 --- a/src/meta/src/rpc/ddl_controller_v2.rs +++ b/src/meta/src/rpc/ddl_controller_v2.rs @@ -400,7 +400,7 @@ impl DdlController { // delete vpc endpoints. for conn in connections { let _ = self - .delete_vpc_endpoint_v2(conn.into_inner()) + .delete_vpc_endpoint_v2(conn.to_protobuf()) .await .inspect_err(|err| { tracing::warn!(err = ?err.as_report(), "failed to delete vpc endpoint"); diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index cda40ff8fdce..0fe9d4a96142 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -680,7 +680,7 @@ impl SourceManager { ( actor_id as ActorId, splits - .into_inner() + .to_protobuf() .splits .iter() .map(|split| SplitImpl::try_from(split).unwrap()) From 8b09aeb30504bcf409b3fc5355d6783d98af89c1 Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 12 Apr 2024 14:44:37 +0800 Subject: [PATCH 07/64] fix(test): fix e2e iceberg test (#16271) --- ci/scripts/e2e-iceberg-sink-test.sh | 14 ++++++++++++-- e2e_test/sink/iceberg_sink.slt | 4 ++-- src/connector/src/sink/iceberg/mod.rs | 6 +++++- 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/ci/scripts/e2e-iceberg-sink-test.sh b/ci/scripts/e2e-iceberg-sink-test.sh index ba38fcc49260..d22e57669430 100755 --- a/ci/scripts/e2e-iceberg-sink-test.sh +++ b/ci/scripts/e2e-iceberg-sink-test.sh @@ -49,7 +49,7 @@ spark-3.3.1-bin-hadoop3/bin/spark-sql --packages $DEPENDENCIES \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.endpoint=http://127.0.0.1:9301 \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.access.key=hummockadmin \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.secret.key=hummockadmin \ - --S --e "CREATE TABLE demo.demo_db.demo_table(v1 int, v2 bigint, v3 string) TBLPROPERTIES ('format-version'='2');" + --S --e "CREATE TABLE demo.demo_db.e2e_demo_table(v1 int, v2 bigint, v3 string) TBLPROPERTIES ('format-version'='2');" echo "--- testing sinks" sqllogictest -p 4566 -d dev './e2e_test/sink/iceberg_sink.slt' @@ -63,7 +63,7 @@ spark-3.3.1-bin-hadoop3/bin/spark-sql --packages $DEPENDENCIES \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.endpoint=http://127.0.0.1:9301 \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.access.key=hummockadmin \ --conf spark.sql.catalog.demo.hadoop.fs.s3a.secret.key=hummockadmin \ - --S --e "INSERT OVERWRITE DIRECTORY './spark-output' USING CSV SELECT * FROM demo.demo_db.demo_table;" + --S --e "INSERT OVERWRITE DIRECTORY './spark-output' USING CSV SELECT * FROM demo.demo_db.e2e_demo_table;" # check sink destination using shell if cat ./spark-output/*.csv | sort | awk -F "," '{ @@ -81,5 +81,15 @@ else exit 1 fi +spark-3.3.1-bin-hadoop3/bin/spark-sql --packages $DEPENDENCIES \ + --conf spark.sql.catalog.demo=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.demo.type=hadoop \ + --conf spark.sql.catalog.demo.warehouse=s3a://iceberg/ \ + --conf spark.sql.catalog.demo.hadoop.fs.s3a.endpoint=http://127.0.0.1:9301 \ + --conf spark.sql.catalog.demo.hadoop.fs.s3a.access.key=hummockadmin \ + --conf spark.sql.catalog.demo.hadoop.fs.s3a.secret.key=hummockadmin \ + --S --e "drop table demo.demo_db.e2e_demo_table;" + + echo "--- Kill cluster" risedev ci-kill diff --git a/e2e_test/sink/iceberg_sink.slt b/e2e_test/sink/iceberg_sink.slt index adc9e86112f3..05cf2d1108b3 100644 --- a/e2e_test/sink/iceberg_sink.slt +++ b/e2e_test/sink/iceberg_sink.slt @@ -17,7 +17,7 @@ CREATE SINK s6 AS select mv6.v1 as v1, mv6.v2 as v2, mv6.v3 as v3 from mv6 WITH catalog.name = 'demo', catalog.type = 'storage', database.name='demo_db', - table.name='demo_table' + table.name='e2e_demo_table' ); statement ok @@ -31,7 +31,7 @@ CREATE SOURCE iceberg_demo_source WITH ( catalog.name = 'demo', catalog.type = 'storage', database.name='demo_db', - table.name='demo_table' + table.name='e2e_demo_table' ); statement ok diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index e41c5433fe10..22972d562907 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -978,7 +978,11 @@ impl SinkCommitCoordinator for IcebergSinkCommitter { .iter() .map(|meta| WriteResult::try_from(meta, &self.partition_type)) .collect::>>()?; - if write_results.is_empty() { + if write_results.is_empty() + || write_results + .iter() + .all(|r| r.data_files.is_empty() && r.delete_files.is_empty()) + { tracing::debug!(?epoch, "no data to commit"); return Ok(()); } From 089a99706624742e752cbc5a30a614294cc489e7 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 12 Apr 2024 16:35:39 +0800 Subject: [PATCH 08/64] feat(error): provide service name on the client side of gRPC (#16254) Signed-off-by: Bugen Zhao --- Cargo.lock | 1 + src/batch/src/error.rs | 7 --- src/batch/src/execution/grpc_exchange.rs | 3 +- .../common_service/src/observer_manager.rs | 4 +- src/error/src/tonic.rs | 59 ++++++++++++++++--- .../src/scheduler/distributed/stage.rs | 3 +- src/frontend/src/scheduler/error.rs | 11 ---- src/rpc_client/Cargo.toml | 1 + src/rpc_client/src/compactor_client.rs | 5 +- src/rpc_client/src/compute_client.rs | 43 ++++++++++---- src/rpc_client/src/connector_client.rs | 25 +++++--- src/rpc_client/src/error.rs | 27 +++++++-- src/rpc_client/src/lib.rs | 8 ++- src/rpc_client/src/meta_client.rs | 8 ++- src/rpc_client/src/sink_coordinate_client.rs | 8 ++- src/rpc_client/src/stream_client.rs | 8 +-- src/stream/src/executor/exchange/input.rs | 2 +- 17 files changed, 154 insertions(+), 69 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c835d484fb1e..41332d02bdf0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10360,6 +10360,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "moka", + "paste", "rand", "risingwave_common", "risingwave_error", diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index 13803437bb07..5751ab86ea9a 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -150,13 +150,6 @@ impl From for BatchError { } } -impl From for BatchError { - fn from(status: tonic::Status) -> Self { - // Always wrap the status into a `RpcError`. - Self::from(RpcError::from(status)) - } -} - impl<'a> From<&'a BatchError> for Status { fn from(err: &'a BatchError) -> Self { err.to_status(tonic::Code::Internal, "batch") diff --git a/src/batch/src/execution/grpc_exchange.rs b/src/batch/src/execution/grpc_exchange.rs index 727a005eb2bf..27f64b1d5ed4 100644 --- a/src/batch/src/execution/grpc_exchange.rs +++ b/src/batch/src/execution/grpc_exchange.rs @@ -20,6 +20,7 @@ use risingwave_expr::expr_context::capture_expr_context; use risingwave_pb::batch_plan::exchange_source::LocalExecutePlan::{self, Plan}; use risingwave_pb::batch_plan::TaskOutputId; use risingwave_pb::task_service::{ExecuteRequest, GetDataResponse}; +use risingwave_rpc_client::error::RpcError; use risingwave_rpc_client::ComputeClient; use tonic::Streaming; @@ -81,7 +82,7 @@ impl ExchangeSource for GrpcExchangeSource { } Some(r) => r, }; - let task_data = res?; + let task_data = res.map_err(RpcError::from_batch_status)?; let data = DataChunk::from_protobuf(task_data.get_record_batch()?)?.compact(); trace!( "Receiver taskOutput = {:?}, data = {:?}", diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index 53225ab3515e..aa1eeed8f0fa 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -94,8 +94,8 @@ pub enum ObserverError { } impl From for ObserverError { - fn from(value: tonic::Status) -> Self { - Self::Rpc(value.into()) + fn from(status: tonic::Status) -> Self { + Self::Rpc(RpcError::from_meta_status(status)) } } diff --git a/src/error/src/tonic.rs b/src/error/src/tonic.rs index 8aec1837da97..11cbb7106365 100644 --- a/src/error/src/tonic.rs +++ b/src/error/src/tonic.rs @@ -98,7 +98,24 @@ where /// A wrapper of [`tonic::Status`] that provides better error message and extracts /// the source chain from the `details` field. #[derive(Debug)] -pub struct TonicStatusWrapper(tonic::Status); +pub struct TonicStatusWrapper { + inner: tonic::Status, + + /// Optional service name from the client side. + /// + /// # Explanation + /// + /// [`tonic::Status`] is used for both client and server side. When the error is created on + /// the server side, we encourage developers to provide the service name with + /// [`ToTonicStatus::to_status`], so that the info can be included in the HTTP response and + /// then extracted by the client side (in [`TonicStatusWrapper::new`]). + /// + /// However, if there's something wrong with the server side and the error is directly + /// created on the client side, the approach above is not applicable. In this case, the + /// caller should set a "client side" service name to provide better error message. This is + /// achieved by [`TonicStatusWrapperExt::with_client_side_service_name`]. + client_side_service_name: Option, +} impl TonicStatusWrapper { /// Create a new [`TonicStatusWrapper`] from the given [`tonic::Status`] and extract @@ -115,17 +132,21 @@ impl TonicStatusWrapper { } } } - Self(status) + + Self { + inner: status, + client_side_service_name: None, + } } /// Returns the reference to the inner [`tonic::Status`]. pub fn inner(&self) -> &tonic::Status { - &self.0 + &self.inner } /// Consumes `self` and returns the inner [`tonic::Status`]. pub fn into_inner(self) -> tonic::Status { - self.0 + self.inner } } @@ -138,28 +159,50 @@ impl From for TonicStatusWrapper { impl std::fmt::Display for TonicStatusWrapper { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "gRPC request")?; + if let Some(service_name) = self .source() .and_then(|s| s.downcast_ref::()) .and_then(|s| s.service_name.as_ref()) + // if no service name from the server side, use the client side one + .or(self.client_side_service_name.as_ref()) { write!(f, " to {} service", service_name)?; } - write!(f, " failed: {}: ", self.0.code())?; + write!(f, " failed: {}: ", self.inner.code())?; + #[expect(rw::format_error)] // intentionally format the source itself if let Some(source) = self.source() { // Prefer the source chain from the `details` field. write!(f, "{}", source) } else { - write!(f, "{}", self.0.message()) + write!(f, "{}", self.inner.message()) } } } +#[easy_ext::ext(TonicStatusWrapperExt)] +impl T +where + T: Into, +{ + /// Set the client side service name to provide better error message. + /// + /// See the documentation on the field `client_side_service_name` for more details. + pub fn with_client_side_service_name( + self, + service_name: impl Into, + ) -> TonicStatusWrapper { + let mut this = self.into(); + this.client_side_service_name = Some(service_name.into()); + this + } +} + impl std::error::Error for TonicStatusWrapper { fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { - // Delegate to `self.0` as if we're transparent. - self.0.source() + // Delegate to `self.inner` as if we're transparent. + self.inner.source() } } diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index e957c81483fa..9a5e00fbfd70 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -45,6 +45,7 @@ use risingwave_pb::batch_plan::{ use risingwave_pb::common::{BatchQueryEpoch, HostAddress, WorkerNode}; use risingwave_pb::plan_common::ExprContext; use risingwave_pb::task_service::{CancelTaskRequest, TaskInfoResponse}; +use risingwave_rpc_client::error::RpcError; use risingwave_rpc_client::ComputeClientPoolRef; use rw_futures_util::select_all; use thiserror_ext::AsReport; @@ -525,7 +526,7 @@ impl StageRunner { |_| StageState::Failed, QueryMessage::Stage(Failed { id: self.stage.id, - reason: SchedulerError::from(e), + reason: RpcError::from_batch_status(e).into(), }), ) .await; diff --git a/src/frontend/src/scheduler/error.rs b/src/frontend/src/scheduler/error.rs index f841a27b6b42..3fefe1233cbd 100644 --- a/src/frontend/src/scheduler/error.rs +++ b/src/frontend/src/scheduler/error.rs @@ -17,7 +17,6 @@ use risingwave_common::session_config::QueryMode; use risingwave_connector::error::ConnectorError; use risingwave_rpc_client::error::RpcError; use thiserror::Error; -use tonic::{Code, Status}; use crate::error::{ErrorCode, RwError}; use crate::scheduler::plan_fragmenter::QueryId; @@ -69,16 +68,6 @@ pub enum SchedulerError { ), } -/// Only if the code is Internal, change it to Execution Error. Otherwise convert to Rpc Error. -impl From for SchedulerError { - fn from(s: Status) -> Self { - match s.code() { - Code::Internal => Self::TaskExecutionError(s.message().to_string()), - _ => Self::RpcError(s.into()), - } - } -} - impl From for RwError { fn from(s: SchedulerError) -> Self { ErrorCode::SchedulerError(Box::new(s)).into() diff --git a/src/rpc_client/Cargo.toml b/src/rpc_client/Cargo.toml index 5a85c2ac4bdc..6852fb11f717 100644 --- a/src/rpc_client/Cargo.toml +++ b/src/rpc_client/Cargo.toml @@ -24,6 +24,7 @@ hyper = "0.14" # required by tonic itertools = { workspace = true } lru = { workspace = true } moka = { version = "0.12", features = ["future"] } +paste = "1" rand = { workspace = true } risingwave_common = { workspace = true } risingwave_error = { workspace = true } diff --git a/src/rpc_client/src/compactor_client.rs b/src/rpc_client/src/compactor_client.rs index ed0061beb48f..e571dbd504d4 100644 --- a/src/rpc_client/src/compactor_client.rs +++ b/src/rpc_client/src/compactor_client.rs @@ -30,7 +30,7 @@ use tokio::sync::RwLock; use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tonic::transport::{Channel, Endpoint}; -use crate::error::Result; +use crate::error::{Result, RpcError}; use crate::retry_rpc; const ENDPOINT_KEEP_ALIVE_INTERVAL_SEC: u64 = 60; const ENDPOINT_KEEP_ALIVE_TIMEOUT_SEC: u64 = 60; @@ -59,7 +59,8 @@ impl CompactorClient { .monitor_client .to_owned() .stack_trace(StackTraceRequest::default()) - .await? + .await + .map_err(RpcError::from_compactor_status)? .into_inner()) } } diff --git a/src/rpc_client/src/compute_client.rs b/src/rpc_client/src/compute_client.rs index 8e96f7a81702..f908bb21aa3a 100644 --- a/src/rpc_client/src/compute_client.rs +++ b/src/rpc_client/src/compute_client.rs @@ -45,7 +45,7 @@ use tokio_stream::wrappers::UnboundedReceiverStream; use tonic::transport::{Channel, Endpoint}; use tonic::Streaming; -use crate::error::Result; +use crate::error::{Result, RpcError}; use crate::{RpcClient, RpcClientPool}; #[derive(Clone)] @@ -98,7 +98,8 @@ impl ComputeClient { .get_data(GetDataRequest { task_output_id: Some(output_id), }) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } @@ -149,7 +150,8 @@ impl ComputeClient { up_actor_id, down_actor_id ) - })? + }) + .map_err(RpcError::from_compute_status)? .into_inner(); Ok((response_stream, permits_tx)) @@ -172,12 +174,19 @@ impl ComputeClient { tracing_context: TracingContext::from_current_span().to_protobuf(), expr_context: Some(expr_context), }) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } pub async fn execute(&self, req: ExecuteRequest) -> Result> { - Ok(self.task_client.to_owned().execute(req).await?.into_inner()) + Ok(self + .task_client + .to_owned() + .execute(req) + .await + .map_err(RpcError::from_compute_status)? + .into_inner()) } pub async fn cancel(&self, req: CancelTaskRequest) -> Result { @@ -185,7 +194,8 @@ impl ComputeClient { .task_client .to_owned() .cancel_task(req) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } @@ -194,7 +204,8 @@ impl ComputeClient { .monitor_client .to_owned() .stack_trace(StackTraceRequest::default()) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } @@ -203,7 +214,8 @@ impl ComputeClient { .monitor_client .to_owned() .get_back_pressure(GetBackPressureRequest::default()) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } @@ -212,7 +224,8 @@ impl ComputeClient { .monitor_client .to_owned() .profiling(ProfilingRequest { sleep_s }) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } @@ -221,7 +234,8 @@ impl ComputeClient { .monitor_client .to_owned() .heap_profiling(HeapProfilingRequest { dir }) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } @@ -230,7 +244,8 @@ impl ComputeClient { .monitor_client .to_owned() .list_heap_profiling(ListHeapProfilingRequest {}) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } @@ -239,7 +254,8 @@ impl ComputeClient { .monitor_client .to_owned() .analyze_heap(AnalyzeHeapRequest { path }) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } @@ -248,7 +264,8 @@ impl ComputeClient { .config_client .to_owned() .show_config(ShowConfigRequest {}) - .await? + .await + .map_err(RpcError::from_compute_status)? .into_inner()) } } diff --git a/src/rpc_client/src/connector_client.rs b/src/rpc_client/src/connector_client.rs index 1acc538eca23..896f198b8a5f 100644 --- a/src/rpc_client/src/connector_client.rs +++ b/src/rpc_client/src/connector_client.rs @@ -223,7 +223,8 @@ impl ConnectorClient { source_id, err.message() ) - })? + }) + .map_err(RpcError::from_connector_status)? .into_inner()) } @@ -251,7 +252,8 @@ impl ConnectorClient { .await .inspect_err(|err| { tracing::error!("failed to validate source#{}: {}", source_id, err.message()) - })? + }) + .map_err(RpcError::from_connector_status)? .into_inner(); response.error.map_or(Ok(()), |err| { @@ -281,8 +283,12 @@ impl ConnectorClient { rpc_client .sink_writer_stream(ReceiverStream::new(rx)) .await - .map(|response| response.into_inner().map_err(RpcError::from)) - .map_err(RpcError::from) + .map(|response| { + response + .into_inner() + .map_err(RpcError::from_connector_status) + }) + .map_err(RpcError::from_connector_status) }, ) .await?; @@ -313,8 +319,12 @@ impl ConnectorClient { rpc_client .sink_coordinator_stream(ReceiverStream::new(rx)) .await - .map(|response| response.into_inner().map_err(RpcError::from)) - .map_err(RpcError::from) + .map(|response| { + response + .into_inner() + .map_err(RpcError::from_connector_status) + }) + .map_err(RpcError::from_connector_status) }, ) .await?; @@ -340,7 +350,8 @@ impl ConnectorClient { .await .inspect_err(|err| { tracing::error!("failed to validate sink properties: {}", err.message()) - })? + }) + .map_err(RpcError::from_connector_status)? .into_inner(); response.error.map_or_else( || Ok(()), // If there is no error message, return Ok here. diff --git a/src/rpc_client/src/error.rs b/src/rpc_client/src/error.rs index e44d3ac8c0b9..5626912c2f88 100644 --- a/src/rpc_client/src/error.rs +++ b/src/rpc_client/src/error.rs @@ -13,14 +13,16 @@ // limitations under the License. use risingwave_common::util::meta_addr::MetaAddressStrategyParseError; +use risingwave_error::tonic::TonicStatusWrapperExt as _; use thiserror::Error; +use thiserror_ext::Construct; pub type Result = std::result::Result; // Re-export these types as they're commonly used together with `RpcError`. pub use risingwave_error::tonic::{ToTonicStatus, TonicStatusWrapper}; -#[derive(Error, Debug)] +#[derive(Error, Debug, Construct)] pub enum RpcError { #[error(transparent)] TransportError(Box), @@ -48,8 +50,23 @@ impl From for RpcError { } } -impl From for RpcError { - fn from(s: tonic::Status) -> Self { - RpcError::GrpcStatus(Box::new(TonicStatusWrapper::new(s))) - } +/// Intentionally not implemented to enforce using `RpcError::from_xxx_status`, so that +/// the service name can always be included in the error message. +impl !From for RpcError {} + +macro_rules! impl_from_status { + ($($service:ident),* $(,)?) => { + paste::paste! { + impl RpcError { + $( + #[doc = "Convert a gRPC status from " $service " service into an [`RpcError`]."] + pub fn [](s: tonic::Status) -> Self { + Self::grpc_status(s.with_client_side_service_name(stringify!($service))) + } + )* + } + } + }; } + +impl_from_status!(stream, batch, meta, compute, compactor, connector); diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index 6058371556b6..fa276bdd0a5c 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -26,6 +26,7 @@ #![feature(impl_trait_in_assoc_type)] #![feature(error_generic_member_access)] #![feature(panic_update_hook)] +#![feature(negative_impls)] use std::any::type_name; use std::fmt::{Debug, Formatter}; @@ -145,7 +146,7 @@ pub trait ExtraInfoSource: Send + Sync { pub type ExtraInfoSourceRef = Arc; #[macro_export] -macro_rules! rpc_client_method_impl { +macro_rules! stream_rpc_client_method_impl { ($( { $client:tt, $fn_name:ident, $req:ty, $resp:ty }),*) => { $( pub async fn $fn_name(&self, request: $req) -> $crate::Result<$resp> { @@ -153,7 +154,8 @@ macro_rules! rpc_client_method_impl { .$client .to_owned() .$fn_name(request) - .await? + .await + .map_err($crate::error::RpcError::from_stream_status)? .into_inner()) } )* @@ -170,7 +172,7 @@ macro_rules! meta_rpc_client_method_impl { Ok(resp) => Ok(resp.into_inner()), Err(e) => { self.refresh_client_if_needed(e.code()).await; - Err(RpcError::from(e)) + Err($crate::error::RpcError::from_meta_status(e)) } } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index f0dcba7d729e..0f7bfeb6bccb 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -1554,7 +1554,7 @@ impl GrpcMetaClientCore { /// Client to meta server. Cloning the instance is lightweight. /// -/// It is a wrapper of tonic client. See [`crate::rpc_client_method_impl`]. +/// It is a wrapper of tonic client. See [`crate::meta_rpc_client_method_impl`]. #[derive(Debug, Clone)] struct GrpcMetaClient { member_monitor_event_sender: mpsc::Sender>>, @@ -1591,7 +1591,11 @@ impl MetaMemberManagement { async fn refresh_members(&mut self) -> Result<()> { let leader_addr = match self.members.as_mut() { Either::Left(client) => { - let resp = client.to_owned().members(MembersRequest {}).await?; + let resp = client + .to_owned() + .members(MembersRequest {}) + .await + .map_err(RpcError::from_meta_status)?; let resp = resp.into_inner(); resp.members.into_iter().find(|member| member.is_leader) } diff --git a/src/rpc_client/src/sink_coordinate_client.rs b/src/rpc_client/src/sink_coordinate_client.rs index 5d021caa87d0..74c05fa85de8 100644 --- a/src/rpc_client/src/sink_coordinate_client.rs +++ b/src/rpc_client/src/sink_coordinate_client.rs @@ -67,8 +67,12 @@ impl CoordinatorStreamHandle { move |rx| async move { init_stream(rx) .await - .map(|response| response.into_inner().map_err(RpcError::from)) - .map_err(RpcError::from) + .map(|response| { + response + .into_inner() + .map_err(RpcError::from_connector_status) + }) + .map_err(RpcError::from_connector_status) }, ) .await?; diff --git a/src/rpc_client/src/stream_client.rs b/src/rpc_client/src/stream_client.rs index ae5af65f2822..4710be7085ef 100644 --- a/src/rpc_client/src/stream_client.rs +++ b/src/rpc_client/src/stream_client.rs @@ -30,7 +30,7 @@ use tonic::transport::Endpoint; use crate::error::{Result, RpcError}; use crate::tracing::{Channel, TracingInjectedChannelExt}; -use crate::{rpc_client_method_impl, RpcClient, RpcClientPool, UnboundedBidiStreamHandle}; +use crate::{stream_rpc_client_method_impl, RpcClient, RpcClientPool, UnboundedBidiStreamHandle}; #[derive(Clone)] pub struct StreamClient(StreamServiceClient); @@ -79,7 +79,7 @@ macro_rules! for_all_stream_rpc { } impl StreamClient { - for_all_stream_rpc! { rpc_client_method_impl } + for_all_stream_rpc! { stream_rpc_client_method_impl } } pub type StreamingControlHandle = @@ -98,8 +98,8 @@ impl StreamClient { client .streaming_control_stream(UnboundedReceiverStream::new(rx)) .await - .map(|response| response.into_inner().map_err(RpcError::from)) - .map_err(RpcError::from) + .map(|response| response.into_inner().map_err(RpcError::from_stream_status)) + .map_err(RpcError::from_stream_status) }) .await?; match first_rsp { diff --git a/src/stream/src/executor/exchange/input.rs b/src/stream/src/executor/exchange/input.rs index 59bd3fe6e2d3..7b7cc151f46c 100644 --- a/src/stream/src/executor/exchange/input.rs +++ b/src/stream/src/executor/exchange/input.rs @@ -217,7 +217,7 @@ impl RemoteInput { // TODO(error-handling): maintain the source chain return Err(StreamExecutorError::channel_closed(format!( "RemoteInput tonic error: {}", - TonicStatusWrapper::from(e).as_report() + TonicStatusWrapper::new(e).as_report() ))); } } From af175db581bfd06ae9840fcbd5598dad6cdd897a Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Fri, 12 Apr 2024 16:47:42 +0800 Subject: [PATCH 09/64] feat(source): `encode json` option `timestamptz.handling.mode` (#16265) --- e2e_test/source/basic/handling_mode.slt | 136 ++++++++++++++++++ .../json_timestamptz_handling_mode.1 | 4 + src/connector/benches/parser.rs | 1 + .../src/parser/debezium/debezium_parser.rs | 11 +- .../src/parser/debezium/simd_json_parser.rs | 23 ++- src/connector/src/parser/json_parser.rs | 15 +- .../src/parser/maxwell/simd_json_parser.rs | 1 + src/connector/src/parser/mod.rs | 9 +- src/connector/src/parser/plain_parser.rs | 3 +- src/connector/src/parser/unified/json.rs | 109 +++++++++++--- src/connector/src/schema/mod.rs | 2 +- .../src/source/datagen/source/generator.rs | 1 + src/frontend/src/handler/create_source.rs | 17 ++- .../src/executor/backfill/cdc/cdc_backfill.rs | 1 + 14 files changed, 294 insertions(+), 39 deletions(-) create mode 100644 e2e_test/source/basic/handling_mode.slt create mode 100644 scripts/source/test_data/json_timestamptz_handling_mode.1 diff --git a/e2e_test/source/basic/handling_mode.slt b/e2e_test/source/basic/handling_mode.slt new file mode 100644 index 000000000000..caf9be9a01fc --- /dev/null +++ b/e2e_test/source/basic/handling_mode.slt @@ -0,0 +1,136 @@ +statement error unrecognized +create table t ( + payload struct>) +with ( + connector = 'kafka', + properties.bootstrap.server='message_queue:29092', + topic = 'json_timestamptz_handling_mode') +format plain encode json (timestamptz.handling.mode = 'mili'); + +# spelling error on config name would raise a non-fatal notice and use the default +statement ok +create table plain_guess ( + "case" varchar, + payload struct>) +with ( + connector = 'kafka', + properties.bootstrap.server='message_queue:29092', + topic = 'json_timestamptz_handling_mode') +format plain encode json (timestamptz.handling.mod = 'mili'); + +statement ok +create table plain_milli ( + "case" varchar, + payload struct>) +with ( + connector = 'kafka', + properties.bootstrap.server='message_queue:29092', + topic = 'json_timestamptz_handling_mode') +format plain encode json (timestamptz.handling.mode = 'milli'); + +statement ok +create table plain_micro ( + "case" varchar, + payload struct>) +with ( + connector = 'kafka', + properties.bootstrap.server='message_queue:29092', + topic = 'json_timestamptz_handling_mode') +format plain encode json (timestamptz.handling.mode = 'micro'); + +statement ok +create table plain_utc ( + "case" varchar, + payload struct>) +with ( + connector = 'kafka', + properties.bootstrap.server='message_queue:29092', + topic = 'json_timestamptz_handling_mode') +format plain encode json (timestamptz.handling.mode = 'utc_string'); + +statement ok +create table plain_naive ( + "case" varchar, + payload struct>) +with ( + connector = 'kafka', + properties.bootstrap.server='message_queue:29092', + topic = 'json_timestamptz_handling_mode') +format plain encode json (timestamptz.handling.mode = 'utc_without_suffix'); + +statement ok +create table debezium_milli ( + "case" varchar, at timestamptz, primary key("case")) +with ( + connector = 'kafka', + properties.bootstrap.server='message_queue:29092', + topic = 'json_timestamptz_handling_mode') +format debezium encode json (timestamptz.handling.mode = 'milli'); + +sleep 2s + +query TT +select "case", (payload).after.at from plain_guess order by 1; +---- +0 number small 1970-01-01 00:01:40+00:00 +1 number recent 2024-04-11 02:00:00.123456+00:00 +2 string utc 2024-04-11 02:00:00.654321+00:00 +3 string naive NULL + +query TT +select "case", (payload).after.at from plain_milli order by 1; +---- +0 number small 1970-01-01 00:00:00.100+00:00 +1 number recent 56246-07-01 08:02:03.456+00:00 +2 string utc 2024-04-11 02:00:00.654321+00:00 +3 string naive NULL + +query TT +select "case", (payload).after.at from plain_micro order by 1; +---- +0 number small 1970-01-01 00:00:00.000100+00:00 +1 number recent 2024-04-11 02:00:00.123456+00:00 +2 string utc 2024-04-11 02:00:00.654321+00:00 +3 string naive NULL + +query TT +select "case", (payload).after.at from plain_utc order by 1; +---- +0 number small NULL +1 number recent NULL +2 string utc 2024-04-11 02:00:00.654321+00:00 +3 string naive NULL + +query TT +select "case", (payload).after.at from plain_naive order by 1; +---- +0 number small NULL +1 number recent NULL +2 string utc NULL +3 string naive 2024-04-11 02:00:00.234321+00:00 + +query TT +select "case", at from debezium_milli order by 1; +---- +0 number small 1970-01-01 00:00:00.100+00:00 +1 number recent 56246-07-01 08:02:03.456+00:00 +2 string utc 2024-04-11 02:00:00.654321+00:00 +3 string naive NULL + +statement ok +drop table plain_guess; + +statement ok +drop table plain_milli; + +statement ok +drop table plain_micro; + +statement ok +drop table plain_utc; + +statement ok +drop table plain_naive; + +statement ok +drop table debezium_milli; diff --git a/scripts/source/test_data/json_timestamptz_handling_mode.1 b/scripts/source/test_data/json_timestamptz_handling_mode.1 new file mode 100644 index 000000000000..4ff1440b47f2 --- /dev/null +++ b/scripts/source/test_data/json_timestamptz_handling_mode.1 @@ -0,0 +1,4 @@ +{"case":"0 number small","payload":{"after":{"case":"0 number small","at":100},"op":"r"}} +{"case":"1 number recent","payload":{"after":{"case":"1 number recent","at":1712800800123456},"op":"r"}} +{"case":"2 string utc","payload":{"after":{"case":"2 string utc","at":"2024-04-11T02:00:00.654321Z"},"op":"r"}} +{"case":"3 string naive","payload":{"after":{"case":"3 string naive","at":"2024-04-11 02:00:00.234321"},"op":"r"}} diff --git a/src/connector/benches/parser.rs b/src/connector/benches/parser.rs index eff6f310bc54..f0527e119086 100644 --- a/src/connector/benches/parser.rs +++ b/src/connector/benches/parser.rs @@ -77,6 +77,7 @@ fn create_parser( key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), protocol_config: ProtocolProperties::Plain, }; diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index 1903c00a3b96..7fb497c399c6 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -21,6 +21,7 @@ use super::{DebeziumAvroAccessBuilder, DebeziumAvroParserConfig}; use crate::error::ConnectorResult; use crate::extract_key_config; use crate::parser::unified::debezium::DebeziumChangeEvent; +use crate::parser::unified::json::TimestamptzHandling; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; use crate::parser::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, JsonProperties, @@ -69,8 +70,12 @@ async fn build_accessor_builder( DebeziumAvroAccessBuilder::new(config, encoding_type)?, )) } - EncodingProperties::Json(_) => Ok(AccessBuilderImpl::DebeziumJson( - DebeziumJsonAccessBuilder::new()?, + EncodingProperties::Json(json_config) => Ok(AccessBuilderImpl::DebeziumJson( + DebeziumJsonAccessBuilder::new( + json_config + .timestamptz_handling + .unwrap_or(TimestamptzHandling::GuessNumberUnit), + )?, )), EncodingProperties::Protobuf(_) => { Ok(AccessBuilderImpl::new_default(config, encoding_type).await?) @@ -111,6 +116,7 @@ impl DebeziumParser { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()), }; @@ -216,6 +222,7 @@ mod tests { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()), }; diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index f61545a5443f..72c1688d95e7 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -20,18 +20,22 @@ use simd_json::BorrowedValue; use crate::error::ConnectorResult; use crate::parser::unified::debezium::MongoJsonAccess; -use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; +use crate::parser::unified::json::{JsonAccess, JsonParseOptions, TimestamptzHandling}; use crate::parser::unified::AccessImpl; use crate::parser::AccessBuilder; #[derive(Debug)] pub struct DebeziumJsonAccessBuilder { value: Option>, + json_parse_options: JsonParseOptions, } impl DebeziumJsonAccessBuilder { - pub fn new() -> ConnectorResult { - Ok(Self { value: None }) + pub fn new(timestamptz_handling: TimestamptzHandling) -> ConnectorResult { + Ok(Self { + value: None, + json_parse_options: JsonParseOptions::new_for_debezium(timestamptz_handling), + }) } } @@ -51,7 +55,7 @@ impl AccessBuilder for DebeziumJsonAccessBuilder { Ok(AccessImpl::Json(JsonAccess::new_with_options( payload, - &JsonParseOptions::DEBEZIUM, + &self.json_parse_options, ))) } } @@ -59,11 +63,17 @@ impl AccessBuilder for DebeziumJsonAccessBuilder { #[derive(Debug)] pub struct DebeziumMongoJsonAccessBuilder { value: Option>, + json_parse_options: JsonParseOptions, } impl DebeziumMongoJsonAccessBuilder { pub fn new() -> anyhow::Result { - Ok(Self { value: None }) + Ok(Self { + value: None, + json_parse_options: JsonParseOptions::new_for_debezium( + TimestamptzHandling::GuessNumberUnit, + ), + }) } } @@ -82,7 +92,7 @@ impl AccessBuilder for DebeziumMongoJsonAccessBuilder { }; Ok(AccessImpl::MongoJson(MongoJsonAccess::new( - JsonAccess::new_with_options(payload, &JsonParseOptions::DEBEZIUM), + JsonAccess::new_with_options(payload, &self.json_parse_options), ))) } } @@ -128,6 +138,7 @@ mod tests { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()), }; diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 47db36a0b2b8..454e8f1d0dd5 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -23,7 +23,7 @@ use risingwave_pb::plan_common::ColumnDesc; use super::avro::schema_resolver::ConfluentSchemaResolver; use super::util::{bytes_from_url, get_kafka_topic}; -use super::{EncodingProperties, SchemaRegistryAuth, SpecificParserConfig}; +use super::{EncodingProperties, JsonProperties, SchemaRegistryAuth, SpecificParserConfig}; use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::avro::util::avro_schema_to_column_descs; @@ -40,6 +40,7 @@ use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; pub struct JsonAccessBuilder { value: Option>, payload_start_idx: usize, + json_parse_options: JsonParseOptions, } impl AccessBuilder for JsonAccessBuilder { @@ -58,16 +59,21 @@ impl AccessBuilder for JsonAccessBuilder { value, // Debezium and Canal have their special json access builder and will not // use this - &JsonParseOptions::DEFAULT, + &self.json_parse_options, ))) } } impl JsonAccessBuilder { - pub fn new(use_schema_registry: bool) -> ConnectorResult { + pub fn new(config: JsonProperties) -> ConnectorResult { + let mut json_parse_options = JsonParseOptions::DEFAULT; + if let Some(mode) = config.timestamptz_handling { + json_parse_options.timestamptz_handling = mode; + } Ok(Self { value: None, - payload_start_idx: if use_schema_registry { 5 } else { 0 }, + payload_start_idx: if config.use_schema_registry { 5 } else { 0 }, + json_parse_options, }) } } @@ -595,6 +601,7 @@ mod tests { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), protocol_config: ProtocolProperties::Upsert, }; diff --git a/src/connector/src/parser/maxwell/simd_json_parser.rs b/src/connector/src/parser/maxwell/simd_json_parser.rs index 45bb4dfbc0dc..d018ef73b050 100644 --- a/src/connector/src/parser/maxwell/simd_json_parser.rs +++ b/src/connector/src/parser/maxwell/simd_json_parser.rs @@ -36,6 +36,7 @@ mod tests { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), protocol_config: ProtocolProperties::Maxwell, }; diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index c218639c3e0e..ddcbc3558b90 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -45,6 +45,7 @@ pub use self::mysql::mysql_row_to_owned_row; use self::plain_parser::PlainParser; pub use self::postgres::postgres_row_to_owned_row; use self::simd_json_parser::DebeziumJsonAccessBuilder; +pub use self::unified::json::TimestamptzHandling; use self::unified::AccessImpl; use self::upsert_parser::UpsertParser; use self::util::get_kafka_topic; @@ -876,7 +877,7 @@ impl AccessBuilderImpl { AccessBuilderImpl::Bytes(BytesAccessBuilder::new(config)?) } EncodingProperties::Json(config) => { - AccessBuilderImpl::Json(JsonAccessBuilder::new(config.use_schema_registry)?) + AccessBuilderImpl::Json(JsonAccessBuilder::new(config)?) } _ => unreachable!(), }; @@ -1006,6 +1007,7 @@ impl SpecificParserConfig { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), protocol_config: ProtocolProperties::Plain, }; @@ -1046,6 +1048,7 @@ pub struct CsvProperties { #[derive(Debug, Default, Clone)] pub struct JsonProperties { pub use_schema_registry: bool, + pub timestamptz_handling: Option, } #[derive(Debug, Default, Clone)] @@ -1200,10 +1203,14 @@ impl SpecificParserConfig { SourceEncode::Json, ) => EncodingProperties::Json(JsonProperties { use_schema_registry: info.use_schema_registry, + timestamptz_handling: TimestamptzHandling::from_options( + &info.format_encode_options, + )?, }), (SourceFormat::DebeziumMongo, SourceEncode::Json) => { EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }) } (SourceFormat::Plain, SourceEncode::Bytes) => { diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 4dfc59c1a1aa..b50f0d57645f 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -14,6 +14,7 @@ use risingwave_common::bail; +use super::unified::json::TimestamptzHandling; use super::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, @@ -66,7 +67,7 @@ impl PlainParser { }; let transaction_meta_builder = Some(AccessBuilderImpl::DebeziumJson( - DebeziumJsonAccessBuilder::new()?, + DebeziumJsonAccessBuilder::new(TimestamptzHandling::GuessNumberUnit)?, )); Ok(Self { key_builder, diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index a765f333ef31..ae3da8de4dc3 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -34,6 +34,7 @@ use thiserror_ext::AsReport; use super::{Access, AccessError, AccessResult}; use crate::parser::common::json_object_get_case_insensitive; use crate::parser::unified::avro::extract_decimal; +use crate::schema::{bail_invalid_option_error, InvalidOptionError}; #[derive(Clone, Debug)] pub enum ByteaHandling { @@ -46,6 +47,44 @@ pub enum TimeHandling { Milli, Micro, } + +#[derive(Clone, Debug)] +pub enum TimestamptzHandling { + /// `"2024-04-11T02:00:00.123456Z"` + UtcString, + /// `"2024-04-11 02:00:00.123456"` + UtcWithoutSuffix, + /// `1712800800123` + Milli, + /// `1712800800123456` + Micro, + /// Both `1712800800123` (ms) and `1712800800123456` (us) maps to `2024-04-11`. + /// + /// Only works for `[1973-03-03 09:46:40, 5138-11-16 09:46:40)`. + /// + /// This option is backward compatible. + GuessNumberUnit, +} + +impl TimestamptzHandling { + pub const OPTION_KEY: &'static str = "timestamptz.handling.mode"; + + pub fn from_options( + options: &std::collections::BTreeMap, + ) -> Result, InvalidOptionError> { + let mode = match options.get(Self::OPTION_KEY).map(std::ops::Deref::deref) { + Some("utc_string") => Self::UtcString, + Some("utc_without_suffix") => Self::UtcWithoutSuffix, + Some("micro") => Self::Micro, + Some("milli") => Self::Milli, + Some("guess_number_unit") => Self::GuessNumberUnit, + Some(v) => bail_invalid_option_error!("unrecognized {} value {}", Self::OPTION_KEY, v), + None => return Ok(None), + }; + Ok(Some(mode)) + } +} + #[derive(Clone, Debug)] pub enum JsonValueHandling { AsValue, @@ -95,6 +134,7 @@ pub enum StructHandling { pub struct JsonParseOptions { pub bytea_handling: ByteaHandling, pub time_handling: TimeHandling, + pub timestamptz_handling: TimestamptzHandling, pub json_value_handling: JsonValueHandling, pub numeric_handling: NumericHandling, pub boolean_handling: BooleanHandling, @@ -113,6 +153,7 @@ impl JsonParseOptions { pub const CANAL: JsonParseOptions = JsonParseOptions { bytea_handling: ByteaHandling::Standard, time_handling: TimeHandling::Micro, + timestamptz_handling: TimestamptzHandling::GuessNumberUnit, // backward-compatible json_value_handling: JsonValueHandling::AsValue, numeric_handling: NumericHandling::Relax { string_parsing: true, @@ -125,24 +166,10 @@ impl JsonParseOptions { struct_handling: StructHandling::Strict, ignoring_keycase: true, }; - pub const DEBEZIUM: JsonParseOptions = JsonParseOptions { - bytea_handling: ByteaHandling::Base64, - time_handling: TimeHandling::Micro, - json_value_handling: JsonValueHandling::AsString, - numeric_handling: NumericHandling::Relax { - string_parsing: false, - }, - boolean_handling: BooleanHandling::Relax { - string_parsing: false, - string_integer_parsing: false, - }, - varchar_handling: VarcharHandling::Strict, - struct_handling: StructHandling::Strict, - ignoring_keycase: true, - }; pub const DEFAULT: JsonParseOptions = JsonParseOptions { bytea_handling: ByteaHandling::Standard, time_handling: TimeHandling::Micro, + timestamptz_handling: TimestamptzHandling::GuessNumberUnit, // backward-compatible json_value_handling: JsonValueHandling::AsValue, numeric_handling: NumericHandling::Relax { string_parsing: true, @@ -153,6 +180,25 @@ impl JsonParseOptions { ignoring_keycase: true, }; + pub fn new_for_debezium(timestamptz_handling: TimestamptzHandling) -> Self { + Self { + bytea_handling: ByteaHandling::Base64, + time_handling: TimeHandling::Micro, + timestamptz_handling, + json_value_handling: JsonValueHandling::AsString, + numeric_handling: NumericHandling::Relax { + string_parsing: false, + }, + boolean_handling: BooleanHandling::Relax { + string_parsing: false, + string_integer_parsing: false, + }, + varchar_handling: VarcharHandling::Strict, + struct_handling: StructHandling::Strict, + ignoring_keycase: true, + } + } + pub fn parse( &self, value: &BorrowedValue<'_>, @@ -436,17 +482,34 @@ impl JsonParseOptions { .map_err(|_| create_error())? .into(), // ---- Timestamptz ----- - (Some(DataType::Timestamptz), ValueType::String) => value - .as_str() - .unwrap() - .parse::() - .map_err(|_| create_error())? - .into(), + (Some(DataType::Timestamptz), ValueType::String) => match self.timestamptz_handling { + TimestamptzHandling::UtcWithoutSuffix => value + .as_str() + .unwrap() + .parse::() + .map(|naive_utc| Timestamptz::from_micros(naive_utc.0.timestamp_micros())) + .map_err(|_| create_error())? + .into(), + // Unless explicitly requested `utc_without_utc`, we parse string with `YYYY-MM-DDTHH:MM:SSZ`. + _ => value + .as_str() + .unwrap() + .parse::() + .map_err(|_| create_error())? + .into(), + } ( Some(DataType::Timestamptz), ValueType::I64 | ValueType::I128 | ValueType::U64 | ValueType::U128, - ) => i64_to_timestamptz(value.as_i64().unwrap()) - .map_err(|_| create_error())? + ) => value.as_i64() + .and_then(|num| match self.timestamptz_handling { + TimestamptzHandling::GuessNumberUnit => i64_to_timestamptz(num).ok(), + TimestamptzHandling::Micro => Some(Timestamptz::from_micros(num)), + TimestamptzHandling::Milli => Timestamptz::from_millis(num), + // When explicitly requested string format, number without units are rejected. + TimestamptzHandling::UtcString | TimestamptzHandling::UtcWithoutSuffix => None, + }) + .ok_or_else(create_error)? .into(), // ---- Interval ----- (Some(DataType::Interval), ValueType::String) => { diff --git a/src/connector/src/schema/mod.rs b/src/connector/src/schema/mod.rs index 28151e60895e..585dd43fa8bf 100644 --- a/src/connector/src/schema/mod.rs +++ b/src/connector/src/schema/mod.rs @@ -30,7 +30,7 @@ const NAME_STRATEGY_KEY: &str = "schema.registry.name.strategy"; #[derive(Debug, thiserror::Error, thiserror_ext::Macro)] #[error("Invalid option: {message}")] pub struct InvalidOptionError { - message: String, + pub message: String, // #[backtrace] // source: Option, } diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index 1c05c6b4ffc8..5716631dff62 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -276,6 +276,7 @@ mod tests { protocol_config: ProtocolProperties::Plain, encoding_config: EncodingProperties::Json(crate::parser::JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), key_encoding_config: None, }, diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 6691b457da61..d3f7ea12434a 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -32,7 +32,7 @@ use risingwave_connector::parser::additional_columns::{ }; use risingwave_connector::parser::{ schema_to_columns, AvroParserConfig, DebeziumAvroParserConfig, ProtobufParserConfig, - SpecificParserConfig, DEBEZIUM_IGNORE_KEY, + SpecificParserConfig, TimestamptzHandling, DEBEZIUM_IGNORE_KEY, }; use risingwave_connector::schema::schema_registry::{ name_strategy_from_str, SchemaRegistryAuth, SCHEMA_REGISTRY_PASSWORD, SCHEMA_REGISTRY_USERNAME, @@ -438,6 +438,21 @@ pub(crate) async fn bind_columns_from_source( Format::Plain | Format::Upsert | Format::Maxwell | Format::Canal | Format::Debezium, Encode::Json, ) => { + if matches!( + source_schema.format, + Format::Plain | Format::Upsert | Format::Debezium + ) { + // Parse the value but throw it away. + // It would be too late to report error in `SpecificParserConfig::new`, + // which leads to recovery loop. + TimestamptzHandling::from_options(&format_encode_options_to_consume) + .map_err(|err| InvalidInputSyntax(err.message))?; + try_consume_string_from_options( + &mut format_encode_options_to_consume, + TimestamptzHandling::OPTION_KEY, + ); + } + let schema_config = get_json_schema_location(&mut format_encode_options_to_consume)?; stream_source_info.use_schema_registry = json_schema_infer_use_schema_registry(&schema_config); diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index be5326686811..9ccaa74ce6f3 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -607,6 +607,7 @@ pub async fn transform_upstream(upstream: BoxedMessageStream, schema: &Schema) { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { use_schema_registry: false, + timestamptz_handling: None, }), // the cdc message is generated internally so the key must exist. protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()), From 5fb6f698d7bcee376e420bd0f35d46a437be80d6 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Fri, 12 Apr 2024 16:53:22 +0800 Subject: [PATCH 10/64] fix(direct cdc): use `time.precision.mode=connect` (#16119) --- .../src/main/resources/debezium.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/debezium.properties b/java/connector-node/risingwave-connector-service/src/main/resources/debezium.properties index 754cade321f9..4d85e1b98c78 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/debezium.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/debezium.properties @@ -11,7 +11,7 @@ decimal.handling.mode=${debezium.decimal.handling.mode:-string} interval.handling.mode=string max.batch.size=${debezium.max.batch.size:-1024} max.queue.size=${debezium.max.queue.size:-8192} -time.precision.mode=adaptive_time_microseconds +time.precision.mode=connect # Quoted from the debezium document: # > Your application should always properly stop the engine to ensure graceful and complete # > shutdown and that each source record is sent to the application exactly one time. From 4f53f8927a2979dc46f7f5e6679a8761216ef8dd Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 12 Apr 2024 17:05:44 +0800 Subject: [PATCH 11/64] style: apply shellcheck autofix to sh scripts (#16274) Signed-off-by: xxchan --- ci/scripts/backwards-compat-test.sh | 8 ++-- ci/scripts/build.sh | 6 +-- ci/scripts/common.sh | 4 +- ci/scripts/connector-node-integration-test.sh | 14 +++---- ci/scripts/deterministic-e2e-test.sh | 14 +++---- ci/scripts/deterministic-it-test.sh | 2 +- ci/scripts/deterministic-recovery-test.sh | 12 +++--- ci/scripts/e2e-pulsar-sink-test.sh | 2 +- ci/scripts/e2e-source-test.sh | 10 ++--- ci/scripts/e2e-test-parallel-for-opendal.sh | 10 ++--- ci/scripts/e2e-test-parallel-in-memory.sh | 8 ++-- ci/scripts/e2e-test-parallel.sh | 10 ++--- ci/scripts/gen-flamegraph.sh | 12 +++--- ci/scripts/integration-tests.sh | 16 +++---- ci/scripts/java-binding-test.sh | 4 +- ci/scripts/multi-arch-docker.sh | 12 +++--- ci/scripts/regress-test.sh | 14 +++---- ci/scripts/release.sh | 42 +++++++++---------- ci/scripts/run-e2e-test.sh | 2 +- ci/scripts/run-micro-benchmarks.sh | 10 ++--- ci/scripts/run-unit-test.sh | 4 +- .../s3-source-test-for-opendal-fs-engine.sh | 2 +- ci/scripts/s3-source-test.sh | 2 +- 23 files changed, 110 insertions(+), 110 deletions(-) diff --git a/ci/scripts/backwards-compat-test.sh b/ci/scripts/backwards-compat-test.sh index 90ddcf09b740..d539748a23fd 100755 --- a/ci/scripts/backwards-compat-test.sh +++ b/ci/scripts/backwards-compat-test.sh @@ -111,26 +111,26 @@ setup_old_cluster() { echo "--- Get RisingWave binary for $OLD_VERSION" OLD_URL=https://github.com/risingwavelabs/risingwave/releases/download/v${OLD_VERSION}/risingwave-v${OLD_VERSION}-x86_64-unknown-linux.tar.gz set +e - wget $OLD_URL + wget "$OLD_URL" if [[ "$?" -ne 0 ]]; then set -e echo "Failed to download ${OLD_VERSION} from github releases, build from source later during \`risedev d\`" configure_rw_build else set -e - tar -xvf risingwave-v${OLD_VERSION}-x86_64-unknown-linux.tar.gz + tar -xvf risingwave-v"${OLD_VERSION}"-x86_64-unknown-linux.tar.gz mv risingwave target/debug/risingwave echo "--- Start cluster on tag $OLD_VERSION" git config --global --add safe.directory /risingwave - configure_rw $OLD_VERSION + configure_rw "$OLD_VERSION" fi } setup_new_cluster() { echo "--- Setup Risingwave @ $RW_COMMIT" git checkout - - download_and_prepare_rw $profile common + download_and_prepare_rw "$profile" common # Make sure we always start w/o old config rm -r .risingwave/config } diff --git a/ci/scripts/build.sh b/ci/scripts/build.sh index ac02b99d1688..0f4d40d8ff65 100755 --- a/ci/scripts/build.sh +++ b/ci/scripts/build.sh @@ -41,9 +41,9 @@ cargo fmt --all -- --check echo "--- Build Rust components" if [[ "$profile" == "ci-dev" ]]; then - RISINGWAVE_FEATURE_FLAGS="--features rw-dynamic-link --no-default-features" + RISINGWAVE_FEATURE_FLAGS=(--features rw-dynamic-link --no-default-features) else - RISINGWAVE_FEATURE_FLAGS="--features rw-static-link" + RISINGWAVE_FEATURE_FLAGS=(--features rw-static-link) fi cargo build \ @@ -53,7 +53,7 @@ cargo build \ -p risingwave_sqlsmith \ -p risingwave_compaction_test \ -p risingwave_e2e_extended_mode_test \ - $RISINGWAVE_FEATURE_FLAGS \ + "${RISINGWAVE_FEATURE_FLAGS[@]}" \ --features embedded-python-udf \ --profile "$profile" diff --git a/ci/scripts/common.sh b/ci/scripts/common.sh index 5c4ed6e5bc91..d99066cb3d5e 100755 --- a/ci/scripts/common.sh +++ b/ci/scripts/common.sh @@ -105,11 +105,11 @@ get_latest_kafka_version() { local versions=$(curl -s https://downloads.apache.org/kafka/ | grep -Eo 'href="[0-9]+\.[0-9]+\.[0-9]+/"' | grep -Eo "[0-9]+\.[0-9]+\.[0-9]+") # Sort the version numbers and get the latest one local latest_version=$(echo "$versions" | sort -V | tail -n1) - echo $latest_version + echo "$latest_version" } get_latest_kafka_download_url() { local latest_version=$(get_latest_kafka_version) local download_url="https://downloads.apache.org/kafka/${latest_version}/kafka_2.13-${latest_version}.tgz" - echo $download_url + echo "$download_url" } diff --git a/ci/scripts/connector-node-integration-test.sh b/ci/scripts/connector-node-integration-test.sh index a6b748c5728c..0ebdd35ea682 100755 --- a/ci/scripts/connector-node-integration-test.sh +++ b/ci/scripts/connector-node-integration-test.sh @@ -67,8 +67,8 @@ MC_PATH=${PWD}/mc ${MC_PATH} config host add minio http://127.0.0.1:9000 minioadmin minioadmin echo "--- starting connector-node service" -mkdir -p ${RISINGWAVE_ROOT}/java/connector-node/assembly/target/ -cd ${RISINGWAVE_ROOT}/java/connector-node/assembly/target/ +mkdir -p "${RISINGWAVE_ROOT}"/java/connector-node/assembly/target/ +cd "${RISINGWAVE_ROOT}"/java/connector-node/assembly/target/ # tar xvf risingwave-connector-1.0.0.tar.gz > /dev/null buildkite-agent artifact download risingwave-connector.tar.gz ./ tar xvf risingwave-connector.tar.gz > /dev/null @@ -77,7 +77,7 @@ sleep 3 # generate data echo "--- starting generate streamchunk data" -cd ${RISINGWAVE_ROOT}/java/connector-node/python-client +cd "${RISINGWAVE_ROOT}"/java/connector-node/python-client buildkite-agent artifact download java-binding-integration-test.tar.zst ./ tar xf java-binding-integration-test.tar.zst bin ./bin/data-chunk-payload-convert-generator data/sink_input.json > ./data/sink_input @@ -85,13 +85,13 @@ tar xf java-binding-integration-test.tar.zst bin ./bin/data-chunk-payload-generator 30 > ./data/stream_chunk_data echo "--- prepare integration tests" -cd ${RISINGWAVE_ROOT}/java/connector-node +cd "${RISINGWAVE_ROOT}"/java/connector-node pip3 install grpcio grpcio-tools psycopg2 psycopg2-binary pyspark==3.3 black cd python-client && bash gen-stub.sh && bash format-python.sh --check export PYTHONPATH=proto echo "--- running streamchunk data format integration tests" -cd ${RISINGWAVE_ROOT}/java/connector-node/python-client +cd "${RISINGWAVE_ROOT}"/java/connector-node/python-client if python3 integration_tests.py --stream_chunk_format_test --input_binary_file="./data/stream_chunk_data" --data_format_use_json=False; then echo "StreamChunk data format test passed" else @@ -106,8 +106,8 @@ type=("StreamChunk format") ${MC_PATH} mb minio/bucket for ((i=0; i<${#type[@]}; i++)); do echo "--- running file ${type[i]} integration tests" - cd ${RISINGWAVE_ROOT}/java/connector-node/python-client - if python3 integration_tests.py --file_sink ${sink_input_feature[i]}; then + cd "${RISINGWAVE_ROOT}"/java/connector-node/python-client + if python3 integration_tests.py --file_sink "${sink_input_feature[i]}"; then echo "File sink ${type[i]} test passed" else echo "File sink ${type[i]} test failed" diff --git a/ci/scripts/deterministic-e2e-test.sh b/ci/scripts/deterministic-e2e-test.sh index 68a41fafe743..cb23f1cd7f24 100755 --- a/ci/scripts/deterministic-e2e-test.sh +++ b/ci/scripts/deterministic-e2e-test.sh @@ -31,25 +31,25 @@ export LOGDIR=.risingwave/log mkdir -p $LOGDIR echo "--- deterministic simulation e2e, ci-3cn-2fe, ddl" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/ddl/\*\*/\*.slt 2> $LOGDIR/ddl-{}.log && rm $LOGDIR/ddl-{}.log' +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/ddl/\*\*/\*.slt 2> $LOGDIR/ddl-{}.log && rm $LOGDIR/ddl-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, streaming" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/streaming-{}.log && rm $LOGDIR/streaming-{}.log' +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/streaming-{}.log && rm $LOGDIR/streaming-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, batch" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/batch-{}.log && rm $LOGDIR/batch-{}.log' +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/batch-{}.log && rm $LOGDIR/batch-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, kafka source" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation --kafka-datadir=./scripts/source/test_data ./e2e_test/source/basic/kafka\*.slt 2> $LOGDIR/source-{}.log && rm $LOGDIR/source-{}.log' +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation --kafka-datadir=./scripts/source/test_data ./e2e_test/source/basic/kafka\*.slt 2> $LOGDIR/source-{}.log && rm $LOGDIR/source-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, streaming" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/parallel-streaming-{}.log && rm $LOGDIR/parallel-streaming-{}.log' +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/parallel-streaming-{}.log && rm $LOGDIR/parallel-streaming-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, parallel, batch" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/parallel-batch-{}.log && rm $LOGDIR/parallel-batch-{}.log' +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -j 16 ./e2e_test/batch/\*\*/\*.slt 2> $LOGDIR/parallel-batch-{}.log && rm $LOGDIR/parallel-batch-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, fuzzing (pre-generated-queries)" timeout 10m seq 64 | parallel MADSIM_TEST_SEED={} './risingwave_simulation --run-sqlsmith-queries ./src/tests/sqlsmith/tests/sqlsmith-query-snapshots/{} 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe, e2e extended mode test" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation -e 2> $LOGDIR/extended-{}.log && rm $LOGDIR/extended-{}.log' +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation -e 2> $LOGDIR/extended-{}.log && rm $LOGDIR/extended-{}.log' diff --git a/ci/scripts/deterministic-it-test.sh b/ci/scripts/deterministic-it-test.sh index 1c43bd3faae9..40288f5848b1 100755 --- a/ci/scripts/deterministic-it-test.sh +++ b/ci/scripts/deterministic-it-test.sh @@ -19,7 +19,7 @@ mv target/ci-sim target/sim TEST_PATTERN="$@" echo "--- Run integration tests in deterministic simulation mode" -seq $TEST_NUM | parallel "MADSIM_TEST_SEED={} NEXTEST_PROFILE=ci-sim \ +seq "$TEST_NUM" | parallel "MADSIM_TEST_SEED={} NEXTEST_PROFILE=ci-sim \ cargo nextest run \ --no-fail-fast \ --cargo-metadata target/nextest/cargo-metadata.json \ diff --git a/ci/scripts/deterministic-recovery-test.sh b/ci/scripts/deterministic-recovery-test.sh index b14cbce36cbb..4dd2c1ec8893 100755 --- a/ci/scripts/deterministic-recovery-test.sh +++ b/ci/scripts/deterministic-recovery-test.sh @@ -36,14 +36,14 @@ mkdir -p $LOGDIR filter_stack_trace_for_all_logs() { # Defined in `common.sh` for log in "${LOGDIR}"/*.log; do - filter_stack_trace $log + filter_stack_trace "$log" done } trap filter_stack_trace_for_all_logs ERR echo "--- deterministic simulation e2e, ci-3cn-2fe-3meta, recovery, background_ddl" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ ${USE_ARRANGEMENT_BACKFILL:-} \ @@ -51,7 +51,7 @@ ${USE_ARRANGEMENT_BACKFILL:-} \ 2> $LOGDIR/recovery-background-ddl-{}.log && rm $LOGDIR/recovery-background-ddl-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe-3meta, recovery, ddl" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ --background-ddl-rate=${BACKGROUND_DDL_RATE} \ @@ -59,7 +59,7 @@ ${USE_ARRANGEMENT_BACKFILL:-} \ ./e2e_test/ddl/\*\*/\*.slt 2> $LOGDIR/recovery-ddl-{}.log && rm $LOGDIR/recovery-ddl-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe-3meta, recovery, streaming" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ --background-ddl-rate=${BACKGROUND_DDL_RATE} \ @@ -67,7 +67,7 @@ ${USE_ARRANGEMENT_BACKFILL:-} \ ./e2e_test/streaming/\*\*/\*.slt 2> $LOGDIR/recovery-streaming-{}.log && rm $LOGDIR/recovery-streaming-{}.log' echo "--- deterministic simulation e2e, ci-3cn-2fe-3meta, recovery, batch" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ --background-ddl-rate=${BACKGROUND_DDL_RATE} \ @@ -75,7 +75,7 @@ ${USE_ARRANGEMENT_BACKFILL:-} \ ./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" -seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ +seq "$TEST_NUM" | parallel MADSIM_TEST_SEED={} './risingwave_simulation \ --kill \ --kill-rate=${KILL_RATE} \ --kafka-datadir=./scripts/source/test_data \ diff --git a/ci/scripts/e2e-pulsar-sink-test.sh b/ci/scripts/e2e-pulsar-sink-test.sh index bf9a390c7e9e..1ffd5cf56b1c 100755 --- a/ci/scripts/e2e-pulsar-sink-test.sh +++ b/ci/scripts/e2e-pulsar-sink-test.sh @@ -30,7 +30,7 @@ MAX_RETRY=20 while [[ $HTTP_CODE -ne 200 && MAX_RETRY -gt 0 ]] do HTTP_CODE=$(curl --connect-timeout 2 -s -o /dev/null -w ''%{http_code}'' http://pulsar-server:8080/admin/v2/clusters) - echo Got HTTP Code: $HTTP_CODE + echo Got HTTP Code: "$HTTP_CODE" ((MAX_RETRY--)) sleep 5 done diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 8785bc50e2b1..f3c99c404f6e 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -9,9 +9,9 @@ source ci/scripts/common.sh # $1: subject name # $2: schema file path function register_schema_registry() { - curl -X POST http://message_queue:8081/subjects/$1/versions \ + curl -X POST http://message_queue:8081/subjects/"$1"/versions \ -H ‘Content-Type: application/vnd.schemaregistry.v1+json’ \ - --data-binary @<(jq -n --arg schema “$(cat $2)” ‘{schemaType: “PROTOBUF”, schema: $schema}’) + --data-binary @<(jq -n --arg schema “$(cat "$2")” ‘{schemaType: “PROTOBUF”, schema: "$schema"}’) } # prepare environment @@ -93,9 +93,9 @@ sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.share_stream.slt' # create a share source and check whether heartbeat message is received sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.create_source_job.slt' -table_id=`psql -U root -h localhost -p 4566 -d dev -t -c "select id from rw_internal_tables where name like '%mysql_source%';" | xargs`; -table_count=`psql -U root -h localhost -p 4566 -d dev -t -c "select count(*) from rw_table(${table_id}, public);" | xargs`; -if [ $table_count -eq 0 ]; then +table_id=$(psql -U root -h localhost -p 4566 -d dev -t -c "select id from rw_internal_tables where name like '%mysql_source%';" | xargs); +table_count=$(psql -U root -h localhost -p 4566 -d dev -t -c "select count(*) from rw_table(${table_id}, public);" | xargs); +if [ "$table_count" -eq 0 ]; then echo "ERROR: internal table of cdc share source is empty!" exit 1 fi diff --git a/ci/scripts/e2e-test-parallel-for-opendal.sh b/ci/scripts/e2e-test-parallel-for-opendal.sh index fcd63c9f0d58..606adcf929cd 100755 --- a/ci/scripts/e2e-test-parallel-for-opendal.sh +++ b/ci/scripts/e2e-test-parallel-for-opendal.sh @@ -26,12 +26,12 @@ download_and_prepare_rw "$profile" common echo "--- Download artifacts" download-and-decompress-artifact e2e_test_generated ./ -host_args="-h localhost -p 4565 -h localhost -p 4566 -h localhost -p 4567" +host_args=(-h localhost -p 4565 -h localhost -p 4566 -h localhost -p 4567) echo "--- e2e, ci-3cn-3fe-opendal-fs-backend, streaming" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-3cn-3fe-opendal-fs-backend -sqllogictest ${host_args} -d dev './e2e_test/streaming/**/*.slt' -j 16 --junit "parallel-opendal-fs-backend-${profile}" +sqllogictest "${host_args[@]}" -d dev './e2e_test/streaming/**/*.slt' -j 16 --junit "parallel-opendal-fs-backend-${profile}" echo "--- Kill cluster Streaming" risedev ci-kill @@ -41,10 +41,10 @@ rm -rf /tmp/rw_ci echo "--- e2e, ci-3cn-3fe-opendal-fs-backend, batch" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-3cn-3fe-opendal-fs-backend -sqllogictest ${host_args} -d dev './e2e_test/ddl/**/*.slt' --junit "parallel-opendal-fs-backend-ddl-${profile}" -sqllogictest ${host_args} -d dev './e2e_test/visibility_mode/*.slt' -j 16 --junit "parallel-opendal-fs-backend-batch-${profile}" +sqllogictest "${host_args[@]}" -d dev './e2e_test/ddl/**/*.slt' --junit "parallel-opendal-fs-backend-ddl-${profile}" +sqllogictest "${host_args[@]}" -d dev './e2e_test/visibility_mode/*.slt' -j 16 --junit "parallel-opendal-fs-backend-batch-${profile}" echo "--- Kill cluster Batch" risedev ci-kill sleep 1 -rm -rf /tmp/rw_ci \ No newline at end of file +rm -rf /tmp/rw_ci diff --git a/ci/scripts/e2e-test-parallel-in-memory.sh b/ci/scripts/e2e-test-parallel-in-memory.sh index 4b55320f115d..fcde15644c2b 100755 --- a/ci/scripts/e2e-test-parallel-in-memory.sh +++ b/ci/scripts/e2e-test-parallel-in-memory.sh @@ -23,20 +23,20 @@ shift $((OPTIND -1)) download_and_prepare_rw "$profile" common -host_args="-h localhost -p 4565 -h localhost -p 4566 -h localhost -p 4567" +host_args=(-h localhost -p 4565 -h localhost -p 4566 -h localhost -p 4567) echo "--- e2e, ci-3cn-3fe-in-memory, streaming" risedev ci-start ci-3cn-3fe-in-memory sqllogictest --version -sqllogictest ${host_args} -d dev './e2e_test/streaming/**/*.slt' -j 16 --junit "parallel-in-memory-streaming-${profile}" --label in-memory +sqllogictest "${host_args[@]}" -d dev './e2e_test/streaming/**/*.slt' -j 16 --junit "parallel-in-memory-streaming-${profile}" --label in-memory echo "--- Kill cluster" risedev ci-kill echo "--- e2e, ci-3cn-3fe-in-memory, batch" risedev ci-start ci-3cn-3fe-in-memory -sqllogictest ${host_args} -d dev './e2e_test/ddl/**/*.slt' --junit "parallel-in-memory-batch-ddl-${profile}" --label in-memory -sqllogictest ${host_args} -d dev './e2e_test/batch/**/*.slt' -j 16 --junit "parallel-in-memory-batch-${profile}" --label in-memory +sqllogictest "${host_args[@]}" -d dev './e2e_test/ddl/**/*.slt' --junit "parallel-in-memory-batch-ddl-${profile}" --label in-memory +sqllogictest "${host_args[@]}" -d dev './e2e_test/batch/**/*.slt' -j 16 --junit "parallel-in-memory-batch-${profile}" --label in-memory echo "--- Kill cluster" risedev ci-kill diff --git a/ci/scripts/e2e-test-parallel.sh b/ci/scripts/e2e-test-parallel.sh index ad1f11a75f7b..5f16a4c81787 100755 --- a/ci/scripts/e2e-test-parallel.sh +++ b/ci/scripts/e2e-test-parallel.sh @@ -31,28 +31,28 @@ kill_cluster() { risedev ci-kill } -host_args="-h localhost -p 4565 -h localhost -p 4566 -h localhost -p 4567" +host_args=(-h localhost -p 4565 -h localhost -p 4566 -h localhost -p 4567) RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info,risingwave_storage::hummock::compactor::compactor_runner=warn" echo "--- e2e, ci-3streaming-2serving-3fe, streaming" RUST_LOG=$RUST_LOG \ risedev ci-start ci-3streaming-2serving-3fe -sqllogictest ${host_args} -d dev './e2e_test/streaming/**/*.slt' -j 16 --junit "parallel-streaming-${profile}" +sqllogictest "${host_args[@]}" -d dev './e2e_test/streaming/**/*.slt' -j 16 --junit "parallel-streaming-${profile}" kill_cluster echo "--- e2e, ci-3streaming-2serving-3fe, batch" RUST_LOG=$RUST_LOG \ risedev ci-start ci-3streaming-2serving-3fe -sqllogictest ${host_args} -d dev './e2e_test/ddl/**/*.slt' --junit "parallel-batch-ddl-${profile}" -sqllogictest ${host_args} -d dev './e2e_test/visibility_mode/*.slt' -j 16 --junit "parallel-batch-${profile}" +sqllogictest "${host_args[@]}" -d dev './e2e_test/ddl/**/*.slt' --junit "parallel-batch-ddl-${profile}" +sqllogictest "${host_args[@]}" -d dev './e2e_test/visibility_mode/*.slt' -j 16 --junit "parallel-batch-${profile}" kill_cluster echo "--- e2e, ci-3streaming-2serving-3fe, generated" RUST_LOG=$RUST_LOG \ risedev ci-start ci-3streaming-2serving-3fe -sqllogictest ${host_args} -d dev './e2e_test/generated/**/*.slt' -j 16 --junit "parallel-generated-${profile}" +sqllogictest "${host_args[@]}" -d dev './e2e_test/generated/**/*.slt' -j 16 --junit "parallel-generated-${profile}" kill_cluster diff --git a/ci/scripts/gen-flamegraph.sh b/ci/scripts/gen-flamegraph.sh index 8fb71a1aee8a..1ea2777fce8a 100755 --- a/ci/scripts/gen-flamegraph.sh +++ b/ci/scripts/gen-flamegraph.sh @@ -124,7 +124,7 @@ AUCTION_TOPIC="nexmark-auction" BID_TOPIC="nexmark-bid" PERSON_TOPIC="nexmark-person" NUM_PARTITIONS=8 -# NOTE: Due to https://github.com/risingwavelabs/risingwave/issues/6747, use `SEPARATE_TOPICS=false` +# NOTE: Due to https://github.com/risingwavelabs/risingwave/issues/6747, use SEPARATE_TOPICS=false SEPARATE_TOPICS=false RUST_LOG="nexmark_server=info" @@ -180,7 +180,7 @@ configure_all() { # This has minor effect on the flamegraph, so can ignore for now. # could it be related to profiling on Docker? Needs further investigation. start_nperf() { - ./nperf record -p `pidof compute-node` -o perf.data & + ./nperf record -p $(pidof compute-node) -o perf.data & } start_kafka() { @@ -231,7 +231,7 @@ gen_heap_flamegraph() { JEPROF=$(find . -name 'jeprof' | head -1) chmod +x "$JEPROF" COMPUTE_NODE=".risingwave/bin/risingwave/compute-node" - $JEPROF --collapsed $COMPUTE_NODE $LATEST_HEAP_PROFILE > heap.collapsed + $JEPROF --collapsed $COMPUTE_NODE "$LATEST_HEAP_PROFILE" > heap.collapsed ../flamegraph.pl --color=mem --countname=bytes heap.collapsed > perf.svg mv perf.svg .. popd @@ -268,7 +268,7 @@ run_heap_flamegraph() { echo "--- Running benchmark for $QUERY" echo "--- Setting variables" QUERY_LABEL="$1" - QUERY_FILE_NAME="$(echo $QUERY_LABEL | sed 's/nexmark\-\(.*\)/\1.sql/')" + QUERY_FILE_NAME="$(echo "$QUERY_LABEL" | sed 's/nexmark\-\(.*\)/\1.sql/')" QUERY_PATH="$QUERY_DIR/$QUERY_FILE_NAME" FLAMEGRAPH_PATH="perf-$QUERY_LABEL.svg" echo "QUERY_LABEL: $QUERY_LABEL" @@ -328,7 +328,7 @@ run_cpu_flamegraph() { echo "--- Running benchmark for $QUERY" echo "--- Setting variables" QUERY_LABEL="$1" - QUERY_FILE_NAME="$(echo $QUERY_LABEL | sed 's/nexmark\-\(.*\)/\1.sql/')" + QUERY_FILE_NAME="$(echo "$QUERY_LABEL" | sed 's/nexmark\-\(.*\)/\1.sql/')" QUERY_PATH="$QUERY_DIR/$QUERY_FILE_NAME" FLAMEGRAPH_PATH="perf-$QUERY_LABEL.svg" echo "QUERY_LABEL: $QUERY_LABEL" @@ -367,7 +367,7 @@ run_cpu_flamegraph() { echo "--- Generate flamegraph" gen_cpu_flamegraph - mv perf.svg $FLAMEGRAPH_PATH + mv perf.svg "$FLAMEGRAPH_PATH" echo "--- Uploading flamegraph" buildkite-agent artifact upload "./$FLAMEGRAPH_PATH" diff --git a/ci/scripts/integration-tests.sh b/ci/scripts/integration-tests.sh index ef6c024ac9db..90d23b2d8035 100755 --- a/ci/scripts/integration-tests.sh +++ b/ci/scripts/integration-tests.sh @@ -38,13 +38,13 @@ echo "--- case: ${case}, format: ${format}" if [[ -n "${RW_IMAGE_TAG+x}" ]]; then export RW_IMAGE="ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_TAG}" - echo Docker image: $RW_IMAGE + echo Docker image: "$RW_IMAGE" fi if [ "${BUILDKITE_SOURCE}" == "schedule" ]; then # Use ghcr nightly image for scheduled build. If not specified, we use dockerhub's 'risingwavelabs/risingwave'. export RW_IMAGE="ghcr.io/risingwavelabs/risingwave:nightly-$(date '+%Y%m%d')" - echo Docker image: $RW_IMAGE + echo Docker image: "$RW_IMAGE" fi if [ "${case}" == "client-library" ]; then @@ -68,7 +68,7 @@ cd integration_tests/scripts echo "--- rewrite docker compose for protobuf" if [ "${format}" == "protobuf" ]; then - python3 gen_pb_compose.py ${case} ${format} + python3 gen_pb_compose.py "${case}" "${format}" fi echo "--- set vm.max_map_count=2000000 for doris" @@ -76,18 +76,18 @@ max_map_count_original_value=$(sysctl -n vm.max_map_count) sudo sysctl -w vm.max_map_count=2000000 echo "--- run Demos" -python3 run_demos.py --case ${case} --format ${format} +python3 run_demos.py --case "${case}" --format "${format}" echo "--- run docker ps" docker ps echo "--- check if the ingestion is successful" # extract the type of upstream source,e.g. mysql,postgres,etc -upstream=$(echo ${case} | cut -d'-' -f 1) -python3 check_data.py ${case} ${upstream} +upstream=$(echo "${case}" | cut -d'-' -f 1) +python3 check_data.py "${case}" "${upstream}" echo "--- clean Demos" -python3 clean_demos.py --case ${case} +python3 clean_demos.py --case "${case}" echo "--- reset vm.max_map_count={$max_map_count_original_value}" -sudo sysctl -w vm.max_map_count=$max_map_count_original_value +sudo sysctl -w vm.max_map_count="$max_map_count_original_value" diff --git a/ci/scripts/java-binding-test.sh b/ci/scripts/java-binding-test.sh index 0c8807411a2e..0391dcdb5128 100755 --- a/ci/scripts/java-binding-test.sh +++ b/ci/scripts/java-binding-test.sh @@ -39,8 +39,8 @@ risedev ci-kill echo "--- run stream chunk java binding" RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) -cd ${RISINGWAVE_ROOT}/java +cd "${RISINGWAVE_ROOT}"/java -(${RISINGWAVE_ROOT}/bin/data-chunk-payload-generator) | \ +("${RISINGWAVE_ROOT}"/bin/data-chunk-payload-generator) | \ java -cp "./java-binding-integration-test/target/dependency/*:./java-binding-integration-test/target/classes" \ com.risingwave.java.binding.StreamChunkDemo diff --git a/ci/scripts/multi-arch-docker.sh b/ci/scripts/multi-arch-docker.sh index dd61fab27eba..b97b606201d6 100755 --- a/ci/scripts/multi-arch-docker.sh +++ b/ci/scripts/multi-arch-docker.sh @@ -62,14 +62,14 @@ echo "--- multi arch image create " if [[ "${#BUILDKITE_COMMIT}" = 40 ]]; then # If the commit is 40 characters long, it's probably a SHA. TAG="git-${BUILDKITE_COMMIT}" - pushGchr ${TAG} + pushGchr "${TAG}" fi if [ "${BUILDKITE_SOURCE}" == "schedule" ]; then # If this is a schedule build, tag the image with the date. TAG="nightly-${date}" - pushGchr ${TAG} - pushDockerhub ${TAG} + pushGchr "${TAG}" + pushDockerhub "${TAG}" TAG="latest" pushGchr ${TAG} fi @@ -77,14 +77,14 @@ fi if [[ -n "${IMAGE_TAG+x}" ]]; then # Tag the image with the $IMAGE_TAG. TAG="${IMAGE_TAG}" - pushGchr ${TAG} + pushGchr "${TAG}" fi if [[ -n "${BUILDKITE_TAG}" ]]; then # If there's a tag, we tag the image. TAG="${BUILDKITE_TAG}" - pushGchr ${TAG} - pushDockerhub ${TAG} + pushGchr "${TAG}" + pushDockerhub "${TAG}" TAG="latest" pushDockerhub ${TAG} diff --git a/ci/scripts/regress-test.sh b/ci/scripts/regress-test.sh index 0dacff818bad..4d32f8a30ef4 100755 --- a/ci/scripts/regress-test.sh +++ b/ci/scripts/regress-test.sh @@ -51,20 +51,20 @@ RUST_BACKTRACE=1 target/debug/risingwave_regress_test --host db \ -p 5432 \ -u postgres \ --database postgres \ - --input `pwd`/src/tests/regress/data \ - --output `pwd`/src/tests/regress/output \ - --schedule `pwd`/src/tests/regress/data/schedule \ + --input $(pwd)/src/tests/regress/data \ + --output $(pwd)/src/tests/regress/output \ + --schedule $(pwd)/src/tests/regress/data/schedule \ --mode postgres echo "--- ci-3cn-1fe, RisingWave regress test" -rm -rf `pwd`/src/tests/regress/output +rm -rf $(pwd)/src/tests/regress/output risedev ci-start ci-3cn-1fe RUST_BACKTRACE=1 target/debug/risingwave_regress_test --host 127.0.0.1 \ -p 4566 \ -u root \ - --input `pwd`/src/tests/regress/data \ - --output `pwd`/src/tests/regress/output \ - --schedule `pwd`/src/tests/regress/data/schedule \ + --input $(pwd)/src/tests/regress/data \ + --output $(pwd)/src/tests/regress/output \ + --schedule $(pwd)/src/tests/regress/data/schedule \ --mode risingwave echo "--- Kill cluster" diff --git a/ci/scripts/release.sh b/ci/scripts/release.sh index 7fde7eb9b00e..1c68b8acf9fa 100755 --- a/ci/scripts/release.sh +++ b/ci/scripts/release.sh @@ -38,12 +38,12 @@ unset RUSTC_WRAPPER # disable sccache echo "--- Install protoc3" PROTOC_ARCH=${ARCH} -if [ ${ARCH} == "aarch64" ]; then +if [ "${ARCH}" == "aarch64" ]; then # shellcheck disable=SC1068 PROTOC_ARCH="aarch_64" fi -curl -LO https://github.com/protocolbuffers/protobuf/releases/download/v3.15.8/protoc-3.15.8-linux-${PROTOC_ARCH}.zip -unzip -o protoc-3.15.8-linux-${PROTOC_ARCH}.zip -d protoc +curl -LO https://github.com/protocolbuffers/protobuf/releases/download/v3.15.8/protoc-3.15.8-linux-"${PROTOC_ARCH}".zip +unzip -o protoc-3.15.8-linux-"${PROTOC_ARCH}".zip -d protoc mv ./protoc/bin/protoc /usr/local/bin/ mv ./protoc/include/* /usr/local/include/ @@ -66,7 +66,7 @@ fi echo "--- Build risingwave release binary" export ENABLE_BUILD_DASHBOARD=1 -if [ ${ARCH} == "aarch64" ]; then +if [ "${ARCH}" == "aarch64" ]; then # enable large page size support for jemalloc # see https://github.com/tikv/jemallocator/blob/802969384ae0c581255f3375ee2ba774c8d2a754/jemalloc-sys/build.rs#L218 export JEMALLOC_SYS_WITH_LG_PAGE=16 @@ -77,21 +77,21 @@ cd target/release && chmod +x risingwave risectl echo "--- Upload nightly binary to s3" if [ "${BUILDKITE_SOURCE}" == "schedule" ]; then - tar -czvf risingwave-"$(date '+%Y%m%d')"-${ARCH}-unknown-linux.tar.gz risingwave - aws s3 cp risingwave-"$(date '+%Y%m%d')"-${ARCH}-unknown-linux.tar.gz s3://rw-nightly-pre-built-binary + tar -czvf risingwave-"$(date '+%Y%m%d')"-"${ARCH}"-unknown-linux.tar.gz risingwave + aws s3 cp risingwave-"$(date '+%Y%m%d')"-"${ARCH}"-unknown-linux.tar.gz s3://rw-nightly-pre-built-binary elif [[ -n "${BINARY_NAME+x}" ]]; then - tar -czvf risingwave-${BINARY_NAME}-${ARCH}-unknown-linux.tar.gz risingwave - aws s3 cp risingwave-${BINARY_NAME}-${ARCH}-unknown-linux.tar.gz s3://rw-nightly-pre-built-binary + tar -czvf risingwave-"${BINARY_NAME}"-"${ARCH}"-unknown-linux.tar.gz risingwave + aws s3 cp risingwave-"${BINARY_NAME}"-"${ARCH}"-unknown-linux.tar.gz s3://rw-nightly-pre-built-binary fi echo "--- Build connector node" -cd ${REPO_ROOT}/java && mvn -B package -Dmaven.test.skip=true -Dno-build-rust +cd "${REPO_ROOT}"/java && mvn -B package -Dmaven.test.skip=true -Dno-build-rust if [[ -n "${BUILDKITE_TAG}" ]]; then echo "--- Collect all release assets" - cd ${REPO_ROOT} && mkdir release-assets && cd release-assets - cp -r ${REPO_ROOT}/target/release/* . - mv ${REPO_ROOT}/java/connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz risingwave-connector-"${BUILDKITE_TAG}".tar.gz + cd "${REPO_ROOT}" && mkdir release-assets && cd release-assets + cp -r "${REPO_ROOT}"/target/release/* . + mv "${REPO_ROOT}"/java/connector-node/assembly/target/risingwave-connector-1.0.0.tar.gz risingwave-connector-"${BUILDKITE_TAG}".tar.gz tar -zxvf risingwave-connector-"${BUILDKITE_TAG}".tar.gz libs ls -l @@ -102,7 +102,7 @@ if [[ -n "${BUILDKITE_TAG}" ]]; then echo "--- Release create" set +e - response=$(gh api repos/risingwavelabs/risingwave/releases/tags/${BUILDKITE_TAG} 2>&1) + response=$(gh api repos/risingwavelabs/risingwave/releases/tags/"${BUILDKITE_TAG}" 2>&1) set -euo pipefail if [[ $response == *"Not Found"* ]]; then echo "Tag ${BUILDKITE_TAG} does not exist. Creating release..." @@ -112,18 +112,18 @@ if [[ -n "${BUILDKITE_TAG}" ]]; then fi echo "--- Release upload risingwave asset" - tar -czvf risingwave-"${BUILDKITE_TAG}"-${ARCH}-unknown-linux.tar.gz risingwave - gh release upload "${BUILDKITE_TAG}" risingwave-"${BUILDKITE_TAG}"-${ARCH}-unknown-linux.tar.gz + tar -czvf risingwave-"${BUILDKITE_TAG}"-"${ARCH}"-unknown-linux.tar.gz risingwave + gh release upload "${BUILDKITE_TAG}" risingwave-"${BUILDKITE_TAG}"-"${ARCH}"-unknown-linux.tar.gz echo "--- Release upload risingwave debug info" - tar -czvf risingwave-"${BUILDKITE_TAG}"-${ARCH}-unknown-linux.dwp.tar.gz risingwave.dwp - gh release upload "${BUILDKITE_TAG}" risingwave-"${BUILDKITE_TAG}"-${ARCH}-unknown-linux.dwp.tar.gz + tar -czvf risingwave-"${BUILDKITE_TAG}"-"${ARCH}"-unknown-linux.dwp.tar.gz risingwave.dwp + gh release upload "${BUILDKITE_TAG}" risingwave-"${BUILDKITE_TAG}"-"${ARCH}"-unknown-linux.dwp.tar.gz echo "--- Release upload risectl asset" - tar -czvf risectl-"${BUILDKITE_TAG}"-${ARCH}-unknown-linux.tar.gz risectl - gh release upload "${BUILDKITE_TAG}" risectl-"${BUILDKITE_TAG}"-${ARCH}-unknown-linux.tar.gz + tar -czvf risectl-"${BUILDKITE_TAG}"-"${ARCH}"-unknown-linux.tar.gz risectl + gh release upload "${BUILDKITE_TAG}" risectl-"${BUILDKITE_TAG}"-"${ARCH}"-unknown-linux.tar.gz echo "--- Release upload risingwave-all-in-one asset" - tar -czvf risingwave-"${BUILDKITE_TAG}"-${ARCH}-unknown-linux-all-in-one.tar.gz risingwave libs - gh release upload "${BUILDKITE_TAG}" risingwave-"${BUILDKITE_TAG}"-${ARCH}-unknown-linux-all-in-one.tar.gz + tar -czvf risingwave-"${BUILDKITE_TAG}"-"${ARCH}"-unknown-linux-all-in-one.tar.gz risingwave libs + gh release upload "${BUILDKITE_TAG}" risingwave-"${BUILDKITE_TAG}"-"${ARCH}"-unknown-linux-all-in-one.tar.gz fi diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index 8d9e393b25ef..84b336362613 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -169,7 +169,7 @@ if [[ "$RUN_COMPACTION" -eq "1" ]]; then # Poll the current version id until we have around 100 version deltas delta_log_cnt=0 - while [ $delta_log_cnt -le 90 ] + while [ "$delta_log_cnt" -le 90 ] do delta_log_cnt="$(./target/debug/risingwave risectl hummock list-version --verbose | grep -w '^ *id:' | grep -o '[0-9]\+' | head -n 1)" echo "Current version $delta_log_cnt" diff --git a/ci/scripts/run-micro-benchmarks.sh b/ci/scripts/run-micro-benchmarks.sh index 371cc416e7ac..6094b3b7f284 100755 --- a/ci/scripts/run-micro-benchmarks.sh +++ b/ci/scripts/run-micro-benchmarks.sh @@ -12,7 +12,7 @@ BENCHMARKS="stream_hash_agg json_parser bench_block_iter bench_compactor bench_l # Reference: https://docs.aws.amazon.com/AWSEC2/latest/UserGuide/instancedata-data-retrieval.html get_instance_type() { - TOKEN=`curl -X PUT "http://169.254.169.254/latest/api/token" -H "X-aws-ec2-metadata-token-ttl-seconds: 21600"` \ + TOKEN=$(curl -X PUT "http://169.254.169.254/latest/api/token" -H "X-aws-ec2-metadata-token-ttl-seconds: 21600") \ && curl -H "X-aws-ec2-metadata-token: $TOKEN" -v http://169.254.169.254/latest/meta-data/instance-type } @@ -26,8 +26,8 @@ bench() { if [[ $REASON == \"benchmark-complete\" ]]; then ID="$(jq ".id" <<< "$LINE")" MEAN="$(jq ".mean" <<< "$LINE")" - EST="$(jq ".estimate" <<< $MEAN)" - UNIT="$(jq ".unit" <<< $MEAN)" + EST="$(jq ".estimate" <<< "$MEAN")" + UNIT="$(jq ".unit" <<< "$MEAN")" echo "Benchmark ID: $ID" echo "Average Time Taken: $EST" @@ -71,7 +71,7 @@ main() { OLD_IFS=$IFS IFS=$'\n' - bench $BENCHMARK + bench "$BENCHMARK" IFS=$OLD_IFS @@ -90,7 +90,7 @@ local_test() { for BENCHMARK in $BENCHMARKS do echo "--- Running $BENCHMARK" - bench $BENCHMARK + bench "$BENCHMARK" done NO_TRAILING_COMMA=$(sed -E '$ s/(.*),$/\1/' ./results.json) echo "$NO_TRAILING_COMMA" > ./results.json diff --git a/ci/scripts/run-unit-test.sh b/ci/scripts/run-unit-test.sh index 8fe4b150bf86..d9a723a34fa1 100755 --- a/ci/scripts/run-unit-test.sh +++ b/ci/scripts/run-unit-test.sh @@ -6,9 +6,9 @@ set -euo pipefail REPO_ROOT=${PWD} echo "+++ Run python UDF SDK unit tests" -cd ${REPO_ROOT}/src/expr/udf/python +cd "${REPO_ROOT}"/src/expr/udf/python python3 -m pytest -cd ${REPO_ROOT} +cd "${REPO_ROOT}" echo "+++ Run unit tests" # use tee to disable progress bar diff --git a/ci/scripts/s3-source-test-for-opendal-fs-engine.sh b/ci/scripts/s3-source-test-for-opendal-fs-engine.sh index 7521eaa97b33..11bdb8ff2ce6 100755 --- a/ci/scripts/s3-source-test-for-opendal-fs-engine.sh +++ b/ci/scripts/s3-source-test-for-opendal-fs-engine.sh @@ -30,7 +30,7 @@ risedev ci-start ci-3cn-3fe-opendal-fs-backend echo "--- Run test" python3 -m pip install minio psycopg2-binary -python3 e2e_test/s3/$script +python3 e2e_test/s3/"$script" echo "--- Kill cluster" rm -rf /tmp/rw_ci diff --git a/ci/scripts/s3-source-test.sh b/ci/scripts/s3-source-test.sh index 80c4f5fa3f16..62a58ab3242f 100755 --- a/ci/scripts/s3-source-test.sh +++ b/ci/scripts/s3-source-test.sh @@ -30,7 +30,7 @@ risedev ci-start ci-1cn-1fe echo "--- Run test" python3 -m pip install minio psycopg2-binary opendal -python3 e2e_test/s3/$script +python3 e2e_test/s3/"$script" echo "--- Kill cluster" risedev ci-kill From 2268ef463fdd61a08813690dc3bb8c2ac2b39141 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Fri, 12 Apr 2024 18:04:32 +0800 Subject: [PATCH 12/64] fix(binder): RCTE context shall not leak to main query (#16282) --- .../tests/testdata/input/recursive_cte.yaml | 12 ++++++++++++ .../tests/testdata/output/recursive_cte.yaml | 18 ++++++++++++++++++ src/frontend/src/binder/bind_context.rs | 10 +--------- src/frontend/src/binder/mod.rs | 10 ---------- src/frontend/src/binder/query.rs | 12 ++++++++++++ 5 files changed, 43 insertions(+), 19 deletions(-) create mode 100644 src/frontend/planner_test/tests/testdata/input/recursive_cte.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/recursive_cte.yaml diff --git a/src/frontend/planner_test/tests/testdata/input/recursive_cte.yaml b/src/frontend/planner_test/tests/testdata/input/recursive_cte.yaml new file mode 100644 index 000000000000..411d129d415f --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/recursive_cte.yaml @@ -0,0 +1,12 @@ +- name: basic + sql: WITH RECURSIVE t1 AS (SELECT 1 AS a UNION ALL SELECT a + 1 FROM t1 WHERE a < 10) SELECT * FROM t1; + expected_outputs: + - planner_error +- name: output column follows lhs + sql: WITH RECURSIVE t1 AS (SELECT 1 AS a UNION ALL SELECT a + 1 FROM t1 WHERE a < 10) SELECT a FROM t1; + expected_outputs: + - planner_error +- name: name a is leaked outside + sql: WITH RECURSIVE t1 AS (SELECT 1 AS a UNION ALL SELECT a + 1 FROM t1 WHERE a < 10) SELECT a; + expected_outputs: + - binder_error diff --git a/src/frontend/planner_test/tests/testdata/output/recursive_cte.yaml b/src/frontend/planner_test/tests/testdata/output/recursive_cte.yaml new file mode 100644 index 000000000000..18e0e18726d3 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/recursive_cte.yaml @@ -0,0 +1,18 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- name: basic + sql: WITH RECURSIVE t1 AS (SELECT 1 AS a UNION ALL SELECT a + 1 FROM t1 WHERE a < 10) SELECT * FROM t1; + planner_error: |- + Feature is not yet implemented: recursive CTE is not supported + Tracking issue: https://github.com/risingwavelabs/risingwave/issues/15135 +- name: output column follows lhs + sql: WITH RECURSIVE t1 AS (SELECT 1 AS a UNION ALL SELECT a + 1 FROM t1 WHERE a < 10) SELECT a FROM t1; + planner_error: |- + Feature is not yet implemented: recursive CTE is not supported + Tracking issue: https://github.com/risingwavelabs/risingwave/issues/15135 +- name: name a is leaked outside + sql: WITH RECURSIVE t1 AS (SELECT 1 AS a UNION ALL SELECT a + 1 FROM t1 WHERE a < 10) SELECT a; + binder_error: | + Failed to bind expression: a + + Caused by: + Item not found: Invalid column: a diff --git a/src/frontend/src/binder/bind_context.rs b/src/frontend/src/binder/bind_context.rs index 21a987cbfac0..7a52de7decec 100644 --- a/src/frontend/src/binder/bind_context.rs +++ b/src/frontend/src/binder/bind_context.rs @@ -371,16 +371,8 @@ impl BindContext { entry.extend(v.into_iter().map(|x| x + begin)); } for (k, (x, y)) in other.range_of { - match self.range_of.entry(k.clone()) { + match self.range_of.entry(k) { Entry::Occupied(e) => { - // check if this is a merge with recursive cte - if let Some(r) = self.cte_to_relation.get(&k) { - if let BindingCteState::Bound { .. } = r.borrow().state { - // no-op - continue; - } - } - // otherwise this merge in invalid return Err(ErrorCode::InternalError(format!( "Duplicated table name while merging adjacent contexts: {}", e.key() diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index 2b2765bdba97..a9dc6250054c 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -563,20 +563,11 @@ mod tests { data_type: Int32, }, ), - InputRef( - InputRef { - index: 1, - data_type: Int32, - }, - ), ], aliases: [ Some( "a", ), - Some( - "a", - ), ], from: Some( Share( @@ -718,7 +709,6 @@ mod tests { schema: Schema { fields: [ a:Int32, - a:Int32, ], }, }, diff --git a/src/frontend/src/binder/query.rs b/src/frontend/src/binder/query.rs index fbacc1178d8d..39de52666297 100644 --- a/src/frontend/src/binder/query.rs +++ b/src/frontend/src/binder/query.rs @@ -346,6 +346,7 @@ impl Binder { .get() .clone(); + self.push_context(); if let Some(with) = with { self.bind_with(with)?; } @@ -359,8 +360,16 @@ impl Binder { schema: base.schema().clone(), }; + // Reset context for right side, but keep `cte_to_relation`. + let new_context = std::mem::take(&mut self.context); + self.context + .cte_to_relation + .clone_from(&new_context.cte_to_relation); // bind the rest of the recursive cte let mut recursive = self.bind_set_expr(*right)?; + // Reset context for the set operation. + self.context = Default::default(); + self.context.cte_to_relation = new_context.cte_to_relation; Self::align_schema(&mut base, &mut recursive, SetOperator::Union)?; let schema = base.schema().clone(); @@ -375,6 +384,9 @@ impl Binder { entry.borrow_mut().state = BindingCteState::Bound { query: either::Either::Right(recursive_union), }; + // TODO: This does not execute during early return by `?` + // We shall extract it similar to `bind_query` and `bind_query_inner`. + self.pop_context()?; } else { let bound_query = self.bind_query(query)?; self.context.cte_to_relation.insert( From cc795daf8dd039b37f7dcb3fd8013fc1578e8f36 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 12 Apr 2024 19:37:17 +0800 Subject: [PATCH 13/64] fix(cdc): commit offset to upstream after checkpoint has commit (#16058) --- .../connector/source/core/DbzCdcEngine.java | 17 +-- .../source/core/DbzCdcEngineRunner.java | 18 +-- ...sumer.java => DbzChangeEventConsumer.java} | 74 ++++++++++-- .../source/core/DbzSourceHandler.java | 7 +- .../source/core/JniDbzSourceHandler.java | 51 +++++++-- .../source/core/JniDbzSourceRegistry.java | 35 ++++++ .../EmbeddedEngineChangeEventProxy.java | 27 +++++ src/connector/src/lib.rs | 1 + src/connector/src/source/base.rs | 20 ++++ src/connector/src/source/cdc/jni_source.rs | 39 +++++++ src/connector/src/source/cdc/mod.rs | 2 + src/connector/src/source/cdc/source/reader.rs | 14 +-- src/connector/src/source/reader/reader.rs | 11 ++ src/stream/src/common/table/state_table.rs | 4 + .../src/executor/source/fetch_executor.rs | 12 +- .../src/executor/source/fs_source_executor.rs | 4 +- .../src/executor/source/source_executor.rs | 107 +++++++++++++++++- .../executor/source/state_table_handler.rs | 26 ++--- 18 files changed, 394 insertions(+), 75 deletions(-) rename java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/{DbzCdcEventConsumer.java => DbzChangeEventConsumer.java} (76%) create mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/JniDbzSourceRegistry.java create mode 100644 java/connector-node/risingwave-source-cdc/src/main/java/io/debezium/embedded/EmbeddedEngineChangeEventProxy.java create mode 100644 src/connector/src/source/cdc/jni_source.rs diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java index 61d1f6284a67..b515ce8bd79b 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java @@ -17,7 +17,6 @@ import static io.debezium.config.CommonConnectorConfig.TOPIC_PREFIX; import static io.debezium.schema.AbstractTopicNamingStrategy.*; -import com.risingwave.connector.api.source.CdcEngine; import com.risingwave.proto.ConnectorServiceProto; import io.debezium.embedded.Connect; import io.debezium.engine.DebeziumEngine; @@ -25,11 +24,11 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; -public class DbzCdcEngine implements CdcEngine { +public class DbzCdcEngine implements Runnable { static final int DEFAULT_QUEUE_CAPACITY = 16; private final DebeziumEngine engine; - private final DbzCdcEventConsumer consumer; + private final DbzChangeEventConsumer changeEventConsumer; private final long id; /** If config is not valid will throw exceptions */ @@ -41,7 +40,7 @@ public DbzCdcEngine( var topicPrefix = config.getProperty(TOPIC_PREFIX.name()); var transactionTopic = String.format("%s.%s", topicPrefix, DEFAULT_TRANSACTION_TOPIC); var consumer = - new DbzCdcEventConsumer( + new DbzChangeEventConsumer( sourceId, heartbeatTopicPrefix, transactionTopic, @@ -49,7 +48,7 @@ public DbzCdcEngine( // Builds a debezium engine but not start it this.id = sourceId; - this.consumer = consumer; + this.changeEventConsumer = consumer; this.engine = DebeziumEngine.create(Connect.class) .using(config) @@ -64,7 +63,6 @@ public void run() { engine.run(); } - @Override public long getId() { return id; } @@ -73,8 +71,11 @@ public void stop() throws Exception { engine.close(); } - @Override public BlockingQueue getOutputChannel() { - return consumer.getOutputChannel(); + return changeEventConsumer.getOutputChannel(); + } + + public DbzChangeEventConsumer getChangeEventConsumer() { + return changeEventConsumer; } } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java index 2f2a9408e7e0..b223d0dfba14 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java @@ -28,15 +28,15 @@ import org.slf4j.LoggerFactory; /** Single-thread engine runner */ -public class DbzCdcEngineRunner implements CdcEngineRunner { +public class DbzCdcEngineRunner { static final Logger LOG = LoggerFactory.getLogger(DbzCdcEngineRunner.class); private final ExecutorService executor; private final AtomicBoolean running = new AtomicBoolean(false); - private CdcEngine engine; + private DbzCdcEngine engine; private final DbzConnectorConfig config; - public static CdcEngineRunner newCdcEngineRunner( + public static DbzCdcEngineRunner newCdcEngineRunner( DbzConnectorConfig config, StreamObserver responseObserver) { DbzCdcEngineRunner runner = null; try { @@ -69,7 +69,7 @@ public static CdcEngineRunner newCdcEngineRunner( return runner; } - public static CdcEngineRunner create(DbzConnectorConfig config, long channelPtr) { + public static DbzCdcEngineRunner create(DbzConnectorConfig config, long channelPtr) { DbzCdcEngineRunner runner = new DbzCdcEngineRunner(config); try { var sourceId = config.getSourceId(); @@ -123,7 +123,7 @@ private DbzCdcEngineRunner(DbzConnectorConfig config) { this.config = config; } - private void withEngine(CdcEngine engine) { + private void withEngine(DbzCdcEngine engine) { this.engine = engine; } @@ -160,16 +160,18 @@ public void stop() throws Exception { } } - @Override - public CdcEngine getEngine() { + public DbzCdcEngine getEngine() { return engine; } - @Override public boolean isRunning() { return running.get(); } + public DbzChangeEventConsumer getChangeEventConsumer() { + return engine.getChangeEventConsumer(); + } + private void cleanUp() { running.set(false); // interrupt the runner thread if it is still running diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java similarity index 76% rename from java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEventConsumer.java rename to java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index bcc532038c9c..b6d030537c10 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -18,13 +18,17 @@ import com.risingwave.connector.cdc.debezium.internal.DebeziumOffsetSerializer; import com.risingwave.proto.ConnectorServiceProto.CdcMessage; import com.risingwave.proto.ConnectorServiceProto.GetEventStreamResponse; +import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.embedded.EmbeddedEngineChangeEventProxy; import io.debezium.engine.ChangeEvent; import io.debezium.engine.DebeziumEngine; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.BlockingQueue; +import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; @@ -40,9 +44,9 @@ enum EventType { DATA, } -public class DbzCdcEventConsumer +public class DbzChangeEventConsumer implements DebeziumEngine.ChangeConsumer> { - static final Logger LOG = LoggerFactory.getLogger(DbzCdcEventConsumer.class); + static final Logger LOG = LoggerFactory.getLogger(DbzChangeEventConsumer.class); private final BlockingQueue outputChannel; private final long sourceId; @@ -51,7 +55,10 @@ public class DbzCdcEventConsumer private final String heartbeatTopicPrefix; private final String transactionTopic; - DbzCdcEventConsumer( + private volatile DebeziumEngine.RecordCommitter> + currentRecordCommitter; + + DbzChangeEventConsumer( long sourceId, String heartbeatTopicPrefix, String transactionTopic, @@ -108,6 +115,7 @@ public void handleBatch( DebeziumEngine.RecordCommitter> committer) throws InterruptedException { var respBuilder = GetEventStreamResponse.newBuilder(); + currentRecordCommitter = committer; for (ChangeEvent event : events) { var record = event.value(); EventType eventType = getEventType(record); @@ -199,9 +207,6 @@ var record = event.value(); default: break; } - - // mark the event as processed - committer.markProcessed(event); } LOG.debug("recv {} events", respBuilder.getEventsCount()); @@ -211,16 +216,61 @@ var record = event.value(); var response = respBuilder.build(); outputChannel.put(response); } + } - committer.markBatchFinished(); + public BlockingQueue getOutputChannel() { + return this.outputChannel; } - @Override - public boolean supportsTombstoneEvents() { - return DebeziumEngine.ChangeConsumer.super.supportsTombstoneEvents(); + /** + * Commit the offset to the Debezium engine. NOTES: The input offset is passed from the source + * executor to here + * + * @param offset persisted offset in the Source state table + */ + @SuppressWarnings("unchecked") + public void commitOffset(DebeziumOffset offset) throws InterruptedException { + // Although the committer is read/write by multi-thread, the committer will be not changed + // frequently. + if (currentRecordCommitter == null) { + LOG.info( + "commitOffset() called on Debezium change consumer which doesn't receive records yet."); + return; + } + + // only the offset is used + SourceRecord recordWrapper = + new SourceRecord( + offset.sourcePartition, + adjustSourceOffset((Map) offset.sourceOffset), + "DUMMY", + Schema.BOOLEAN_SCHEMA, + true); + ChangeEvent changeEvent = + EmbeddedEngineChangeEventProxy.create(null, recordWrapper, recordWrapper); + currentRecordCommitter.markProcessed(changeEvent); + currentRecordCommitter.markBatchFinished(); } - public BlockingQueue getOutputChannel() { - return this.outputChannel; + /** + * We have to adjust type of LSN values to Long, because it might be Integer after + * deserialization, however {@link + * io.debezium.connector.postgresql.PostgresStreamingChangeEventSource#commitOffset(Map, Map)} + * requires Long. + */ + private Map adjustSourceOffset(Map sourceOffset) { + if (sourceOffset.containsKey(PostgresOffsetContext.LAST_COMPLETELY_PROCESSED_LSN_KEY)) { + String value = + sourceOffset + .get(PostgresOffsetContext.LAST_COMPLETELY_PROCESSED_LSN_KEY) + .toString(); + sourceOffset.put( + PostgresOffsetContext.LAST_COMPLETELY_PROCESSED_LSN_KEY, Long.parseLong(value)); + } + if (sourceOffset.containsKey(PostgresOffsetContext.LAST_COMMIT_LSN_KEY)) { + String value = sourceOffset.get(PostgresOffsetContext.LAST_COMMIT_LSN_KEY).toString(); + sourceOffset.put(PostgresOffsetContext.LAST_COMMIT_LSN_KEY, Long.parseLong(value)); + } + return sourceOffset; } } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzSourceHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzSourceHandler.java index e5da5a3680a2..76aff296bf35 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzSourceHandler.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzSourceHandler.java @@ -14,7 +14,6 @@ package com.risingwave.connector.source.core; -import com.risingwave.connector.api.source.CdcEngineRunner; import com.risingwave.connector.api.source.SourceHandler; import com.risingwave.connector.source.common.DbzConnectorConfig; import com.risingwave.metrics.ConnectorNodeMetrics; @@ -25,7 +24,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** * handler for starting a debezium source connectors */ +/** handler for starting a debezium source connectors */ public class DbzSourceHandler implements SourceHandler { static final Logger LOG = LoggerFactory.getLogger(DbzSourceHandler.class); @@ -36,11 +35,11 @@ public DbzSourceHandler(DbzConnectorConfig config) { } class OnReadyHandler implements Runnable { - private final CdcEngineRunner runner; + private final DbzCdcEngineRunner runner; private final ServerCallStreamObserver responseObserver; public OnReadyHandler( - CdcEngineRunner runner, + DbzCdcEngineRunner runner, ServerCallStreamObserver responseObserver) { this.runner = runner; this.responseObserver = responseObserver; diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/JniDbzSourceHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/JniDbzSourceHandler.java index 949ccd403edc..30092195f40f 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/JniDbzSourceHandler.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/JniDbzSourceHandler.java @@ -16,28 +16,40 @@ import static com.risingwave.proto.ConnectorServiceProto.SourceType.POSTGRES; -import com.risingwave.connector.api.source.CdcEngineRunner; import com.risingwave.connector.api.source.SourceTypeE; +import com.risingwave.connector.cdc.debezium.internal.DebeziumOffset; +import com.risingwave.connector.cdc.debezium.internal.DebeziumOffsetSerializer; +import com.risingwave.connector.source.common.CdcConnectorException; import com.risingwave.connector.source.common.DbzConnectorConfig; import com.risingwave.connector.source.common.DbzSourceUtils; import com.risingwave.java.binding.Binding; import com.risingwave.metrics.ConnectorNodeMetrics; import com.risingwave.proto.ConnectorServiceProto; import com.risingwave.proto.ConnectorServiceProto.GetEventStreamResponse; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** handler for starting a debezium source connectors for jni */ + /** handler for starting a debezium source connectors for jni */ public class JniDbzSourceHandler { static final Logger LOG = LoggerFactory.getLogger(JniDbzSourceHandler.class); private final DbzConnectorConfig config; + private final DbzCdcEngineRunner runner; - public JniDbzSourceHandler(DbzConnectorConfig config) { + public JniDbzSourceHandler(DbzConnectorConfig config, long channelPtr) { this.config = config; + this.runner = DbzCdcEngineRunner.create(config, channelPtr); + + if (runner == null) { + throw new CdcConnectorException("Failed to create engine runner"); + } } public static void runJniDbzSourceThread(byte[] getEventStreamRequestBytes, long channelPtr) @@ -66,15 +78,31 @@ public static void runJniDbzSourceThread(byte[] getEventStreamRequestBytes, long mutableUserProps, request.getSnapshotDone(), isCdcSourceJob); - JniDbzSourceHandler handler = new JniDbzSourceHandler(config); + JniDbzSourceHandler handler = new JniDbzSourceHandler(config, channelPtr); + // register handler to the registry + JniDbzSourceRegistry.register(config.getSourceId(), handler); handler.start(channelPtr); } - public void start(long channelPtr) { - var runner = DbzCdcEngineRunner.create(config, channelPtr); - if (runner == null) { - return; + public void commitOffset(String encodedOffset) throws InterruptedException { + try { + DebeziumOffset offset = + DebeziumOffsetSerializer.INSTANCE.deserialize( + encodedOffset.getBytes(StandardCharsets.UTF_8)); + var changeEventConsumer = runner.getChangeEventConsumer(); + if (changeEventConsumer != null) { + changeEventConsumer.commitOffset(offset); + LOG.info("Engine#{}: committed offset {}", config.getSourceId(), offset); + } else { + LOG.warn("Engine#{}: changeEventConsumer is null", config.getSourceId()); + } + } catch (IOException err) { + LOG.error("Engine#{}: fail to commit offset.", config.getSourceId(), err); + throw new CdcConnectorException(err.getMessage()); } + } + + public void start(long channelPtr) { try { // Start the engine @@ -83,6 +111,8 @@ public void start(long channelPtr) { LOG.error( "Failed to send handshake message to channel. sourceId={}", config.getSourceId()); + // remove the handler from registry + JniDbzSourceRegistry.unregister(config.getSourceId()); return; } @@ -122,10 +152,13 @@ public void start(long channelPtr) { LOG.warn("Failed to stop Engine#{}", config.getSourceId(), e); } } + + // remove the handler from registry + JniDbzSourceRegistry.unregister(config.getSourceId()); } - private boolean sendHandshakeMessage(CdcEngineRunner runner, long channelPtr, boolean startOk) - throws Exception { + private boolean sendHandshakeMessage( + DbzCdcEngineRunner runner, long channelPtr, boolean startOk) throws Exception { // send a handshake message to notify the Source executor // if the handshake is not ok, the split reader will return error to source actor var controlInfo = diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/JniDbzSourceRegistry.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/JniDbzSourceRegistry.java new file mode 100644 index 000000000000..97eca6003b72 --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/JniDbzSourceRegistry.java @@ -0,0 +1,35 @@ +// 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. + +package com.risingwave.connector.source.core; + +import java.util.concurrent.ConcurrentHashMap; + +/** Global registry for all JNI Debezium source handlers. */ +public class JniDbzSourceRegistry { + private static final ConcurrentHashMap sourceHandlers = + new ConcurrentHashMap<>(); + + public static void register(long sourceId, JniDbzSourceHandler handler) { + sourceHandlers.put(sourceId, handler); + } + + public static JniDbzSourceHandler getSourceHandler(long sourceId) { + return sourceHandlers.get(sourceId); + } + + public static void unregister(long sourceId) { + sourceHandlers.remove(sourceId); + } +} diff --git a/java/connector-node/risingwave-source-cdc/src/main/java/io/debezium/embedded/EmbeddedEngineChangeEventProxy.java b/java/connector-node/risingwave-source-cdc/src/main/java/io/debezium/embedded/EmbeddedEngineChangeEventProxy.java new file mode 100644 index 000000000000..d76075a91e7e --- /dev/null +++ b/java/connector-node/risingwave-source-cdc/src/main/java/io/debezium/embedded/EmbeddedEngineChangeEventProxy.java @@ -0,0 +1,27 @@ +// 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. + +package io.debezium.embedded; + +import io.debezium.engine.ChangeEvent; +import java.util.Collections; +import org.apache.kafka.connect.source.SourceRecord; + +/** Act as a proxy to the package-private class EmbeddedEngineChangeEvent */ +public class EmbeddedEngineChangeEventProxy { + public static ChangeEvent create( + SourceRecord key, SourceRecord value, SourceRecord sourceRecord) { + return new EmbeddedEngineChangeEvent<>(key, value, Collections.emptyList(), sourceRecord); + } +} diff --git a/src/connector/src/lib.rs b/src/connector/src/lib.rs index 4beff70c921a..1153a133eb1d 100644 --- a/src/connector/src/lib.rs +++ b/src/connector/src/lib.rs @@ -54,6 +54,7 @@ pub mod source; pub mod connector_common; pub use paste::paste; +pub use risingwave_jni_core::{call_method, call_static_method, jvm_runtime}; mod with_options; pub use with_options::WithPropertiesExt; diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 7b6c6a71897c..420194fe4a94 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -45,6 +45,7 @@ use crate::parser::ParserConfig; pub(crate) use crate::source::common::CommonSplitReader; use crate::source::filesystem::FsPageItem; use crate::source::monitor::EnumeratorMetrics; +use crate::source::SplitImpl::{CitusCdc, MongodbCdc, MysqlCdc, PostgresCdc}; use crate::with_options::WithOptions; use crate::{ dispatch_source_prop, dispatch_split_impl, for_all_sources, impl_connector_properties, @@ -424,6 +425,24 @@ impl SplitImpl { |other| bail!("connector '{}' is not supported", other) ) } + + pub fn is_cdc_split(&self) -> bool { + matches!( + self, + MysqlCdc(_) | PostgresCdc(_) | MongodbCdc(_) | CitusCdc(_) + ) + } + + /// Get the current split offset. + pub fn get_cdc_split_offset(&self) -> String { + match self { + MysqlCdc(split) => split.start_offset().clone().unwrap_or_default(), + PostgresCdc(split) => split.start_offset().clone().unwrap_or_default(), + MongodbCdc(split) => split.start_offset().clone().unwrap_or_default(), + CitusCdc(split) => split.start_offset().clone().unwrap_or_default(), + _ => unreachable!("get_cdc_split_offset() is only for cdc split"), + } + } } impl SplitMetaData for SplitImpl { @@ -538,6 +557,7 @@ pub trait SplitMetaData: Sized { Self::restore_from_json(JsonbVal::value_deserialize(bytes).unwrap()) } + /// Encode the whole split metadata to a JSON object fn encode_to_json(&self) -> JsonbVal; fn restore_from_json(value: JsonbVal) -> Result; fn update_offset(&mut self, last_seen_offset: String) -> crate::error::ConnectorResult<()>; diff --git a/src/connector/src/source/cdc/jni_source.rs b/src/connector/src/source/cdc/jni_source.rs new file mode 100644 index 000000000000..406756d0f84f --- /dev/null +++ b/src/connector/src/source/cdc/jni_source.rs @@ -0,0 +1,39 @@ +// 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 anyhow::Context; +use risingwave_jni_core::jvm_runtime::{execute_with_jni_env, JVM}; +use risingwave_jni_core::{call_method, call_static_method}; + +pub fn commit_cdc_offset(source_id: u64, encoded_offset: String) -> anyhow::Result<()> { + let jvm = JVM.get_or_init()?; + execute_with_jni_env(jvm, |env| { + // get source handler by source id + let handler = call_static_method!( + env, + {com.risingwave.connector.source.core.JniDbzSourceRegistry}, + {com.risingwave.connector.source.core.JniDbzSourceHandler getSourceHandler(long sourceId)}, + source_id + )?; + + let offset_str = env.new_string(&encoded_offset).with_context(|| { + format!("Failed to create jni string from source offset: {encoded_offset}.") + })?; + // commit offset to upstream + call_method!(env, handler, {void commitOffset(String)}, &offset_str).with_context( + || format!("Failed to commit offset to upstream for source: {source_id}."), + )?; + Ok(()) + }) +} diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index d0d66ea221fd..decbcffebc36 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -14,8 +14,10 @@ pub mod enumerator; pub mod external; +pub mod jni_source; pub mod source; pub mod split; + use std::collections::HashMap; use std::marker::PhantomData; diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 029be2a6e30e..cf2b5c3d17e0 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -97,7 +97,6 @@ impl SplitReader for CdcSplitReader { let (mut tx, mut rx) = mpsc::channel(DEFAULT_CHANNEL_SIZE); let jvm = JVM.get_or_init()?; - let get_event_stream_request = GetEventStreamRequest { source_id, source_type: source_type as _, @@ -158,8 +157,8 @@ impl SplitReader for CdcSplitReader { } tracing::info!(?source_id, "cdc connector started"); - match T::source_type() { - CdcSourceType::Mysql | CdcSourceType::Postgres | CdcSourceType::Mongodb => Ok(Self { + let instance = match T::source_type() { + CdcSourceType::Mysql | CdcSourceType::Postgres | CdcSourceType::Mongodb => Self { source_id: split.split_id() as u64, start_offset: split.start_offset().clone(), server_addr: None, @@ -169,8 +168,8 @@ impl SplitReader for CdcSplitReader { parser_config, source_ctx, rx, - }), - CdcSourceType::Citus => Ok(Self { + }, + CdcSourceType::Citus => Self { source_id: split.split_id() as u64, start_offset: split.start_offset().clone(), server_addr: citus_server_addr, @@ -180,11 +179,12 @@ impl SplitReader for CdcSplitReader { parser_config, source_ctx, rx, - }), + }, CdcSourceType::Unspecified => { unreachable!(); } - } + }; + Ok(instance) } fn into_stream(self) -> BoxChunkSourceStream { diff --git a/src/connector/src/source/reader/reader.rs b/src/connector/src/source/reader/reader.rs index 5cfd10835998..e8c20f66dedc 100644 --- a/src/connector/src/source/reader/reader.rs +++ b/src/connector/src/source/reader/reader.rs @@ -105,6 +105,17 @@ impl SourceReader { } } + /// Postgres and Oracle connectors need to commit the offset to upstream. + pub fn need_commit_offset_to_upstream(&self) -> bool { + matches!( + &self.config, + ConnectorProperties::PostgresCdc(_) + | ConnectorProperties::MysqlCdc(_) + | ConnectorProperties::MongodbCdc(_) + | ConnectorProperties::CitusCdc(_) + ) + } + pub async fn to_stream( &self, state: ConnectorState, diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index d05cd891565e..09e0488f6ff9 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -212,6 +212,10 @@ where .expect("non-replicated state store should start immediately.") .expect("non-replicated state store should not wait_for_epoch, and fail because of it.") } + + pub fn state_store(&self) -> &S { + &self.store + } } fn consistent_old_value_op(row_serde: impl ValueRowSerde) -> OpConsistencyLevel { diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index bb337bb43155..3c6d89fe59a5 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -90,9 +90,9 @@ impl FsFetchExecutor { stream: &mut StreamReaderWithPause, ) -> StreamExecutorResult<()> { let mut batch = Vec::with_capacity(SPLIT_BATCH_SIZE); - 'vnodes: for vnode in state_store_handler.state_store.vnodes().iter_vnodes() { + 'vnodes: for vnode in state_store_handler.state_table.vnodes().iter_vnodes() { let table_iter = state_store_handler - .state_store + .state_table .iter_with_vnode( vnode, &(Bound::::Unbounded, Bound::::Unbounded), @@ -212,7 +212,7 @@ impl FsFetchExecutor { // Hence we try building a reader first. Self::replace_with_new_batch_reader( &mut splits_on_fetch, - &state_store_handler, + &state_store_handler, // move into the function core.column_ids.clone(), self.build_source_ctx(&source_desc, core.source_id, &core.source_name), &source_desc, @@ -243,7 +243,7 @@ impl FsFetchExecutor { } state_store_handler - .state_store + .state_table .commit(barrier.epoch) .await?; @@ -253,7 +253,7 @@ impl FsFetchExecutor { // if _cache_may_stale, we must rebuild the stream to adjust vnode mappings let (_prev_vnode_bitmap, cache_may_stale) = state_store_handler - .state_store + .state_table .update_vnode_bitmap(vnode_bitmap); if cache_may_stale { @@ -297,7 +297,7 @@ impl FsFetchExecutor { }) .collect(); state_store_handler.set_states(file_assignment).await?; - state_store_handler.state_store.try_flush().await?; + state_store_handler.state_table.try_flush().await?; } _ => unreachable!(), } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 3e53702e720f..73b76c247864 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -243,7 +243,7 @@ impl FsSourceExecutor { core.split_state_store.set_all_complete(completed).await? } // commit anyway, even if no message saved - core.split_state_store.state_store.commit(epoch).await?; + core.split_state_store.state_table.commit(epoch).await?; core.updated_splits_in_epoch.clear(); Ok(()) @@ -252,7 +252,7 @@ impl FsSourceExecutor { async fn try_flush_data(&mut self) -> StreamExecutorResult<()> { let core = &mut self.stream_source_core; - core.split_state_store.state_store.try_flush().await?; + core.split_state_store.state_table.try_flush().await?; Ok(()) } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index d1d21473dce2..65635d0e4074 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::fmt::Formatter; +use std::str::FromStr; use std::time::Duration; use anyhow::anyhow; @@ -22,13 +23,16 @@ use futures_async_stream::try_stream; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; +use risingwave_connector::source::cdc::jni_source; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ BoxChunkSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitMetaData, }; +use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::StateStore; use thiserror_ext::AsReport; -use tokio::sync::mpsc::UnboundedReceiver; +use tokio::sync::mpsc; +use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::time::Instant; use super::executor_core::StreamSourceCore; @@ -78,6 +82,18 @@ impl SourceExecutor { } } + pub fn spawn_wait_epoch_worker( + core: &StreamSourceCore, + ) -> UnboundedSender<(Epoch, HashMap)> { + let (wait_epoch_tx, wait_epoch_rx) = mpsc::unbounded_channel(); + let wait_epoch_worker = WaitEpochWorker { + wait_epoch_rx, + state_store: core.split_state_store.state_table.state_store().clone(), + }; + tokio::spawn(wait_epoch_worker.run()); + wait_epoch_tx + } + pub async fn build_stream_source_reader( &self, source_desc: &SourceDesc, @@ -297,7 +313,7 @@ impl SourceExecutor { async fn persist_state_and_clear_cache( &mut self, epoch: EpochPair, - ) -> StreamExecutorResult<()> { + ) -> StreamExecutorResult> { let core = self.stream_source_core.as_mut().unwrap(); let cache = core @@ -312,16 +328,19 @@ impl SourceExecutor { } // commit anyway, even if no message saved - core.split_state_store.state_store.commit(epoch).await?; + core.split_state_store.state_table.commit(epoch).await?; + + let updated_splits = core.updated_splits_in_epoch.clone(); + core.updated_splits_in_epoch.clear(); - Ok(()) + Ok(updated_splits) } /// try mem table spill async fn try_flush_data(&mut self) -> StreamExecutorResult<()> { let core = self.stream_source_core.as_mut().unwrap(); - core.split_state_store.state_store.try_flush().await?; + core.split_state_store.state_table.try_flush().await?; Ok(()) } @@ -353,6 +372,12 @@ impl SourceExecutor { .build() .map_err(StreamExecutorError::connector_error)?; + let wait_epoch_tx = if source_desc.source.need_commit_offset_to_upstream() { + Some(Self::spawn_wait_epoch_worker(&core)) + } else { + None + }; + let (Some(split_idx), Some(offset_idx)) = get_split_offset_col_idx(&source_desc.columns) else { unreachable!("Partition and offset columns must be set."); @@ -479,7 +504,24 @@ impl SourceExecutor { } } - self.persist_state_and_clear_cache(epoch).await?; + let updated_splits = self.persist_state_and_clear_cache(epoch).await?; + + // when handle a checkpoint barrier, spawn a task to wait for epoch commit notification + if barrier.kind.is_checkpoint() + && !updated_splits.is_empty() + && let Some(ref tx) = wait_epoch_tx + { + let mut updated_offsets = HashMap::new(); + for (split_id, split_impl) in updated_splits { + if split_impl.is_cdc_split() { + updated_offsets.insert(split_id, split_impl.get_cdc_split_offset()); + } + } + + tracing::debug!("epoch to wait {:?}", epoch); + tx.send((Epoch(epoch.prev), updated_offsets)) + .expect("wait_epoch_tx send success"); + } yield Message::Barrier(barrier); } @@ -610,6 +652,59 @@ impl Debug for SourceExecutor { } } +struct WaitEpochWorker { + wait_epoch_rx: UnboundedReceiver<(Epoch, HashMap)>, + state_store: S, +} + +impl WaitEpochWorker { + pub async fn run(mut self) { + tracing::debug!("wait epoch worker start success"); + loop { + // poll the rx and wait for the epoch commit + match self.wait_epoch_rx.recv().await { + Some((epoch, updated_offsets)) => { + tracing::debug!("start to wait epoch {}", epoch.0); + let ret = self + .state_store + .try_wait_epoch(HummockReadEpoch::Committed(epoch.0)) + .await; + + match ret { + Ok(()) => { + tracing::debug!(epoch = epoch.0, "wait epoch success"); + // cdc source only has one split + assert_eq!(1, updated_offsets.len()); + let (split_id, offset) = updated_offsets.into_iter().next().unwrap(); + let source_id: u64 = u64::from_str(split_id.as_ref()).unwrap(); + // notify cdc connector to commit offset + match jni_source::commit_cdc_offset(source_id, offset.clone()) { + Ok(_) => {} + Err(e) => { + tracing::error!( + error = %e.as_report(), + "source#{source_id}: failed to commit cdc offset: {offset}.", + ) + } + } + } + Err(e) => { + tracing::error!( + error = %e.as_report(), + "wait epoch {} failed", epoch.0 + ); + } + } + } + None => { + tracing::error!("wait epoch rx closed"); + break; + } + } + } + } +} + #[cfg(test)] mod tests { use std::time::Duration; diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index dbead349822e..f85f12c793a2 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -47,13 +47,13 @@ use crate::executor::StreamExecutorResult; const COMPLETE_SPLIT_PREFIX: &str = "SsGLdzRDqBuKzMf9bDap"; pub struct SourceStateTableHandler { - pub state_store: StateTable, + pub state_table: StateTable, } impl SourceStateTableHandler { pub async fn from_table_catalog(table_catalog: &PbTable, store: S) -> Self { Self { - state_store: StateTable::from_table_catalog(table_catalog, store, None).await, + state_table: StateTable::from_table_catalog(table_catalog, store, None).await, } } @@ -63,12 +63,12 @@ impl SourceStateTableHandler { vnodes: Option>, ) -> Self { Self { - state_store: StateTable::from_table_catalog(table_catalog, store, vnodes).await, + state_table: StateTable::from_table_catalog(table_catalog, store, vnodes).await, } } pub fn init_epoch(&mut self, epoch: EpochPair) { - self.state_store.init_epoch(epoch); + self.state_table.init_epoch(epoch); } fn string_to_scalar(rhs: impl Into) -> ScalarImpl { @@ -76,7 +76,7 @@ impl SourceStateTableHandler { } pub(crate) async fn get(&self, key: SplitId) -> StreamExecutorResult> { - self.state_store + self.state_table .get_row(row::once(Some(Self::string_to_scalar(key.deref())))) .await .map_err(StreamExecutorError::from) @@ -94,7 +94,7 @@ impl SourceStateTableHandler { // all source executor has vnode id zero let iter = self - .state_store + .state_table .iter_with_vnode(VirtualNode::ZERO, &(start, end), PrefetchOptions::default()) .await?; @@ -126,9 +126,9 @@ impl SourceStateTableHandler { Some(ScalarImpl::Jsonb(value)), ]; if let Some(prev_row) = self.get(key).await? { - self.state_store.delete(prev_row); + self.state_table.delete(prev_row); } - self.state_store.insert(row); + self.state_table.insert(row); Ok(()) } @@ -157,10 +157,10 @@ impl SourceStateTableHandler { ]; match self.get(key).await? { Some(prev_row) => { - self.state_store.update(prev_row, row); + self.state_table.update(prev_row, row); } None => { - self.state_store.insert(row); + self.state_table.insert(row); } } Ok(()) @@ -168,7 +168,7 @@ impl SourceStateTableHandler { pub async fn delete(&mut self, key: SplitId) -> StreamExecutorResult<()> { if let Some(prev_row) = self.get(key).await? { - self.state_store.delete(prev_row); + self.state_table.delete(prev_row); } Ok(()) @@ -310,9 +310,9 @@ pub(crate) mod tests { state_table_handler .set_states(vec![split_impl.clone()]) .await?; - state_table_handler.state_store.commit(epoch_2).await?; + state_table_handler.state_table.commit(epoch_2).await?; - state_table_handler.state_store.commit(epoch_3).await?; + state_table_handler.state_table.commit(epoch_3).await?; match state_table_handler .try_recover_from_state_store(&split_impl) From f31698df52575e0e43d11147cf515bac262d325f Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Fri, 12 Apr 2024 20:46:04 +0800 Subject: [PATCH 14/64] chore(deps): Bump h2 from 0.4.3 to 0.4.4 (#16276) --- Cargo.lock | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 41332d02bdf0..83d966077ae8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4984,9 +4984,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51ee2dd2e4f378392eeff5d51618cd9a63166a2513846bbc55f21cfacd9199d4" +checksum = "816ec7294445779408f36fe57bc5b7fc1cf59664059096c65f905c1c61f58069" dependencies = [ "bytes", "fnv", @@ -5271,7 +5271,7 @@ dependencies = [ "bytes", "futures-channel", "futures-util", - "h2 0.4.3", + "h2 0.4.4", "http 1.0.0", "http-body 1.0.0", "httparse", @@ -8907,7 +8907,7 @@ dependencies = [ "futures-channel", "futures-core", "futures-util", - "h2 0.4.3", + "h2 0.4.4", "http 1.0.0", "http-body 1.0.0", "http-body-util", From 1d254979d6381d8d35a9b2697e351b3a318ae3ce Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Fri, 12 Apr 2024 11:17:38 -0500 Subject: [PATCH 15/64] feat(session_config): system wide session config by alter system set (#16062) --- Cargo.lock | 5 +- e2e_test/ddl/alter_session_params.slt | 33 +++ e2e_test/error_ui/simple/main.slt | 12 - proto/meta.proto | 24 ++ .../common_service/src/observer_manager.rs | 2 +- src/common/proc_macro/src/lib.rs | 1 + src/common/proc_macro/src/session_config.rs | 90 ++++++-- src/common/src/lib.rs | 2 +- src/common/src/session_config/mod.rs | 38 +++- src/common/src/session_config/search_path.rs | 2 +- .../transaction_isolation_level.rs | 16 +- src/frontend/src/binder/mod.rs | 4 +- .../src/catalog/system_catalog/mod.rs | 6 +- src/frontend/src/handler/alter_system.rs | 44 ++-- src/frontend/src/meta_client.rs | 17 ++ src/frontend/src/observer/observer_manager.rs | 13 ++ src/frontend/src/session.rs | 32 ++- src/frontend/src/test_utils.rs | 10 + src/meta/model_v2/migration/src/lib.rs | 2 + .../src/m20240410_154406_session_params.rs | 43 ++++ src/meta/model_v2/src/lib.rs | 1 + src/meta/model_v2/src/prelude.rs | 1 + src/meta/model_v2/src/session_parameter.rs | 29 +++ src/meta/node/src/lib.rs | 1 + src/meta/node/src/server.rs | 20 +- src/meta/service/Cargo.toml | 1 + src/meta/service/src/lib.rs | 1 + src/meta/service/src/notification_service.rs | 17 +- src/meta/service/src/session_config.rs | 77 +++++++ src/meta/src/controller/mod.rs | 1 + src/meta/src/controller/session_params.rs | 211 ++++++++++++++++++ src/meta/src/error.rs | 8 + src/meta/src/manager/env.rs | 47 +++- src/meta/src/manager/mod.rs | 2 + src/meta/src/manager/session_params.rs | 204 +++++++++++++++++ src/rpc_client/src/meta_client.rs | 18 ++ src/utils/pgwire/src/pg_server.rs | 6 +- 37 files changed, 947 insertions(+), 94 deletions(-) create mode 100644 e2e_test/ddl/alter_session_params.slt create mode 100644 src/meta/model_v2/migration/src/m20240410_154406_session_params.rs create mode 100644 src/meta/model_v2/src/session_parameter.rs create mode 100644 src/meta/service/src/session_config.rs create mode 100644 src/meta/src/controller/session_params.rs create mode 100644 src/meta/src/manager/session_params.rs diff --git a/Cargo.lock b/Cargo.lock index 83d966077ae8..4e7ae1d81200 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4346,9 +4346,9 @@ dependencies = [ [[package]] name = "foyer-memory" -version = "0.1.4" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d1c86096f23d567783f7fd3ad34e9d7e94a56854b405305c6b0fe24850a2731" +checksum = "f59727e86709e4eab603f8a5086f3b2c7d3ddddf7c8b31d8dea2b00364b7fb95" dependencies = [ "ahash 0.8.6", "bitflags 2.5.0", @@ -10245,6 +10245,7 @@ dependencies = [ "risingwave_meta_model_v2", "risingwave_pb", "sea-orm", + "serde_json", "sync-point", "thiserror-ext", "tokio-stream", diff --git a/e2e_test/ddl/alter_session_params.slt b/e2e_test/ddl/alter_session_params.slt new file mode 100644 index 000000000000..ba5cdff21ef1 --- /dev/null +++ b/e2e_test/ddl/alter_session_params.slt @@ -0,0 +1,33 @@ +statement ok +set RW_STREAMING_ENABLE_DELTA_JOIN to true; + +statement error session param query_mode cannot be altered system wide +alter system set query_mode to auto; + +connection other1 +query T +show RW_STREAMING_ENABLE_DELTA_JOIN; +---- +false + +statement ok +set RW_STREAMING_ENABLE_DELTA_JOIN to false; + +statement ok +alter system set rw_streaming_enable_delta_join to true; + +query T +show RW_STREAMING_ENABLE_DELTA_JOIN; +---- +false + +sleep 1s + +connection other2 +query T +show RW_STREAMING_ENABLE_DELTA_JOIN; +---- +true + +statement ok +alter system set RW_STREAMING_ENABLE_DELTA_JOIN to default; diff --git a/e2e_test/error_ui/simple/main.slt b/e2e_test/error_ui/simple/main.slt index cbe2154fa4f6..8ef82e1f0d1c 100644 --- a/e2e_test/error_ui/simple/main.slt +++ b/e2e_test/error_ui/simple/main.slt @@ -81,15 +81,3 @@ Caused by these errors (recent errors listed first): 1: Failed to get/set session config 2: Invalid value `maybe` for `rw_implicit_flush` 3: Invalid bool - - -statement error -set transaction_isolation to 'read committed'; ----- -db error: ERROR: Failed to run the query - -Caused by these errors (recent errors listed first): - 1: Failed to get/set session config - 2: Invalid value `read committed` for `transaction_isolation` - 3: Feature is not yet implemented: isolation level -Tracking issue: https://github.com/risingwavelabs/risingwave/issues/10736 diff --git a/proto/meta.proto b/proto/meta.proto index 6a1460b4453a..e48dc485d495 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -386,6 +386,7 @@ message MetaSnapshot { repeated catalog.Connection connections = 17; repeated catalog.Subscription subscriptions = 19; repeated user.UserInfo users = 8; + GetSessionParamsResponse session_params = 20; // for streaming repeated FragmentParallelUnitMapping parallel_unit_mappings = 9; repeated common.WorkerNode nodes = 10; @@ -432,6 +433,7 @@ message SubscribeResponse { catalog.Schema schema = 5; catalog.Function function = 6; user.UserInfo user = 11; + SetSessionParamRequest session_param = 26; // for streaming FragmentParallelUnitMapping parallel_unit_mapping = 12; common.WorkerNode node = 13; @@ -596,6 +598,28 @@ service SystemParamsService { rpc SetSystemParam(SetSystemParamRequest) returns (SetSystemParamResponse); } +message GetSessionParamsRequest {} + +message GetSessionParamsResponse { + string params = 1; +} + +message SetSessionParamRequest { + string param = 1; + // None means set to default value. + optional string value = 2; +} + +message SetSessionParamResponse { + string param = 1; +} + +// Used for alter system wide default parameters +service SessionParamService { + rpc GetSessionParams(GetSessionParamsRequest) returns (GetSessionParamsResponse); + rpc SetSessionParam(SetSessionParamRequest) returns (SetSessionParamResponse); +} + message GetServingVnodeMappingsRequest {} message GetServingVnodeMappingsResponse { diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index aa1eeed8f0fa..3f18be769752 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -156,7 +156,7 @@ where } Info::HummockSnapshot(_) => true, Info::MetaBackupManifestId(_) => true, - Info::SystemParams(_) => true, + Info::SystemParams(_) | Info::SessionParam(_) => true, Info::ServingParallelUnitMappings(_) => true, Info::Snapshot(_) | Info::HummockWriteLimits(_) => unreachable!(), Info::HummockStats(_) => true, diff --git a/src/common/proc_macro/src/lib.rs b/src/common/proc_macro/src/lib.rs index cb52bf9786fe..ccf7d4c282e2 100644 --- a/src/common/proc_macro/src/lib.rs +++ b/src/common/proc_macro/src/lib.rs @@ -258,6 +258,7 @@ pub fn derive_estimate_size(input: TokenStream) -> TokenStream { /// `flags` options include /// - `SETTER`: to manually write a `set_your_parameter_name` function, in which you should call `set_your_parameter_name_inner`. /// - `REPORT`: to report the parameter through `ConfigReporter` +/// - `NO_ALTER_SYS`: disallow the parameter to be set by `alter system set` #[proc_macro_derive(SessionConfig, attributes(parameter))] #[proc_macro_error] pub fn session_config(input: TokenStream) -> TokenStream { diff --git a/src/common/proc_macro/src/session_config.rs b/src/common/proc_macro/src/session_config.rs index 2d961f360ad7..c86993ebf521 100644 --- a/src/common/proc_macro/src/session_config.rs +++ b/src/common/proc_macro/src/session_config.rs @@ -40,7 +40,9 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { let mut get_match_branches = vec![]; let mut reset_match_branches = vec![]; let mut show_all_list = vec![]; + let mut list_all_list = vec![]; let mut alias_to_entry_name_branches = vec![]; + let mut entry_name_flags = vec![]; for field in fields { let field_ident = field.ident.expect_or_abort("Field need to be named"); @@ -149,7 +151,7 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { &mut self, val: &str, reporter: &mut impl ConfigReporter - ) -> SessionConfigResult<()> { + ) -> SessionConfigResult { let val_t = #parse(val).map_err(|e| { SessionConfigError::InvalidValue { entry: #entry_name, @@ -158,8 +160,7 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { } })?; - self.#set_t_func_name(val_t, reporter)?; - Ok(()) + self.#set_t_func_name(val_t, reporter).map(|val| val.to_string()) } #[doc = #set_t_func_doc] @@ -167,12 +168,12 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { &mut self, val: #ty, reporter: &mut impl ConfigReporter - ) -> SessionConfigResult<()> { + ) -> SessionConfigResult<#ty> { #check_hook #report_hook - self.#field_ident = val; - Ok(()) + self.#field_ident = val.clone(); + Ok(val) } }); @@ -181,10 +182,11 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { struct_impl_reset.push(quote! { #[allow(clippy::useless_conversion)] - pub fn #reset_func_name(&mut self, reporter: &mut impl ConfigReporter) { + pub fn #reset_func_name(&mut self, reporter: &mut impl ConfigReporter) -> String { let val = #default; #report_hook self.#field_ident = val.into(); + self.#field_ident.to_string() } }); @@ -224,20 +226,42 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { #entry_name => Ok(self.#reset_func_name(reporter)), }); - if !flags.contains(&"NO_SHOW_ALL") { - show_all_list.push(quote! { - VariableInfo { - name: #entry_name.to_string(), - setting: self.#field_ident.to_string(), - description : #description.to_string(), - }, + let var_info = quote! { + VariableInfo { + name: #entry_name.to_string(), + setting: self.#field_ident.to_string(), + description : #description.to_string(), + }, + }; + list_all_list.push(var_info.clone()); - }); + let no_show_all = flags.contains(&"NO_SHOW_ALL"); + let no_show_all_flag: TokenStream = no_show_all.to_string().parse().unwrap(); + if !no_show_all { + show_all_list.push(var_info); } + + let no_alter_sys_flag: TokenStream = + flags.contains(&"NO_ALTER_SYS").to_string().parse().unwrap(); + + entry_name_flags.push( + quote! { + (#entry_name, ParamFlags {no_show_all: #no_show_all_flag, no_alter_sys: #no_alter_sys_flag}) + } + ); } let struct_ident = input.ident; quote! { + use std::collections::HashMap; + use std::sync::LazyLock; + static PARAM_NAME_FLAGS: LazyLock> = LazyLock::new(|| HashMap::from([#(#entry_name_flags, )*])); + + struct ParamFlags { + no_show_all: bool, + no_alter_sys: bool, + } + impl Default for #struct_ident { #[allow(clippy::useless_conversion)] fn default() -> Self { @@ -252,11 +276,11 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { Default::default() } - fn alias_to_entry_name(key_name: &str) -> &str { + pub fn alias_to_entry_name(key_name: &str) -> String { match key_name { #(#alias_to_entry_name_branches)* _ => key_name, - } + }.to_ascii_lowercase() } #(#struct_impl_get)* @@ -266,9 +290,9 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { #(#struct_impl_reset)* /// Set a parameter given it's name and value string. - pub fn set(&mut self, key_name: &str, value: String, reporter: &mut impl ConfigReporter) -> SessionConfigResult<()> { + pub fn set(&mut self, key_name: &str, value: String, reporter: &mut impl ConfigReporter) -> SessionConfigResult { let key_name = Self::alias_to_entry_name(key_name); - match key_name.to_ascii_lowercase().as_ref() { + match key_name.as_ref() { #(#set_match_branches)* _ => Err(SessionConfigError::UnrecognizedEntry(key_name.to_string())), } @@ -277,27 +301,47 @@ pub(crate) fn derive_config(input: DeriveInput) -> TokenStream { /// Get a parameter by it's name. pub fn get(&self, key_name: &str) -> SessionConfigResult { let key_name = Self::alias_to_entry_name(key_name); - match key_name.to_ascii_lowercase().as_ref() { + match key_name.as_ref() { #(#get_match_branches)* _ => Err(SessionConfigError::UnrecognizedEntry(key_name.to_string())), } } /// Reset a parameter by it's name. - pub fn reset(&mut self, key_name: &str, reporter: &mut impl ConfigReporter) -> SessionConfigResult<()> { + pub fn reset(&mut self, key_name: &str, reporter: &mut impl ConfigReporter) -> SessionConfigResult { let key_name = Self::alias_to_entry_name(key_name); - match key_name.to_ascii_lowercase().as_ref() { + match key_name.as_ref() { #(#reset_match_branches)* _ => Err(SessionConfigError::UnrecognizedEntry(key_name.to_string())), } } - /// Show all parameters. + /// Show all parameters except those specified `NO_SHOW_ALL`. pub fn show_all(&self) -> Vec { vec![ #(#show_all_list)* ] } + + /// List all parameters + pub fn list_all(&self) -> Vec { + vec![ + #(#list_all_list)* + ] + } + + /// Check if `SessionConfig` has a parameter. + pub fn contains_param(key_name: &str) -> bool { + let key_name = Self::alias_to_entry_name(key_name); + PARAM_NAME_FLAGS.contains_key(key_name.as_str()) + } + + /// Check if `SessionConfig` has a parameter. + pub fn check_no_alter_sys(key_name: &str) -> SessionConfigResult { + let key_name = Self::alias_to_entry_name(key_name); + let flags = PARAM_NAME_FLAGS.get(key_name.as_str()).ok_or_else(|| SessionConfigError::UnrecognizedEntry(key_name.to_string()))?; + Ok(flags.no_alter_sys) + } } } } diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index 8bfd70f6248e..ea3cd4a4f710 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -103,7 +103,7 @@ pub const RW_VERSION: &str = env!("CARGO_PKG_VERSION"); /// Placeholder for unknown git sha. pub const UNKNOWN_GIT_SHA: &str = "unknown"; -// The single source of truth of the pg parameters, Used in ConfigMap and current_cluster_version. +// The single source of truth of the pg parameters, Used in SessionConfig and current_cluster_version. // The version of PostgreSQL that Risingwave claims to be. pub const PG_VERSION: &str = "13.14.0"; /// The version of PostgreSQL that Risingwave claims to be. diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index bc8bb0b1506a..c8e5d4b0fcf9 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -23,8 +23,10 @@ mod visibility_mode; use chrono_tz::Tz; pub use over_window::OverWindowCachePolicy; pub use query_mode::QueryMode; -use risingwave_common_proc_macro::SessionConfig; +use risingwave_common_proc_macro::{ConfigDoc, SessionConfig}; pub use search_path::{SearchPath, USER_NAME_WILD_CARD}; +use serde::{Deserialize, Serialize}; +use serde_with::{serde_as, DisplayFromStr}; use thiserror::Error; use self::non_zero64::ConfigNonZeroU64; @@ -50,9 +52,10 @@ pub enum SessionConfigError { type SessionConfigResult = std::result::Result; +#[serde_as] /// This is the Session Config of RisingWave. -#[derive(SessionConfig)] -pub struct ConfigMap { +#[derive(Clone, Debug, Deserialize, Serialize, SessionConfig, ConfigDoc, PartialEq)] +pub struct SessionConfig { /// If `RW_IMPLICIT_FLUSH` is on, then every INSERT/UPDATE/DELETE statement will block /// until the entire dataflow is refreshed. In other words, every related table & MV will /// be able to see the write. @@ -67,7 +70,8 @@ pub struct ConfigMap { /// A temporary config variable to force query running in either local or distributed mode. /// The default value is auto which means let the system decide to run batch queries in local /// or distributed mode automatically. - #[parameter(default = QueryMode::default())] + #[serde_as(as = "DisplayFromStr")] + #[parameter(default = QueryMode::default(), flags = "NO_ALTER_SYS")] query_mode: QueryMode, /// Sets the number of digits displayed for floating-point values. @@ -106,19 +110,23 @@ pub struct ConfigMap { /// Sets the order in which schemas are searched when an object (table, data type, function, etc.) /// is referenced by a simple name with no schema specified. /// See + #[serde_as(as = "DisplayFromStr")] #[parameter(default = SearchPath::default())] search_path: SearchPath, /// If `VISIBILITY_MODE` is all, we will support querying data without checkpoint. + #[serde_as(as = "DisplayFromStr")] #[parameter(default = VisibilityMode::default())] visibility_mode: VisibilityMode, /// See + #[serde_as(as = "DisplayFromStr")] #[parameter(default = IsolationLevel::default())] transaction_isolation: IsolationLevel, /// Select as of specific epoch. /// Sets the historical epoch for querying data. If 0, querying latest data. + #[serde_as(as = "DisplayFromStr")] #[parameter(default = ConfigNonZeroU64::default())] query_epoch: ConfigNonZeroU64, @@ -128,6 +136,7 @@ pub struct ConfigMap { /// If `STREAMING_PARALLELISM` is non-zero, CREATE MATERIALIZED VIEW/TABLE/INDEX will use it as /// streaming parallelism. + #[serde_as(as = "DisplayFromStr")] #[parameter(default = ConfigNonZeroU64::default())] streaming_parallelism: ConfigNonZeroU64, @@ -177,6 +186,7 @@ pub struct ConfigMap { interval_style: String, /// If `BATCH_PARALLELISM` is non-zero, batch queries will use this parallelism. + #[serde_as(as = "DisplayFromStr")] #[parameter(default = ConfigNonZeroU64::default())] batch_parallelism: ConfigNonZeroU64, @@ -197,6 +207,7 @@ pub struct ConfigMap { client_encoding: String, /// Enable decoupling sink and internal streaming graph or not + #[serde_as(as = "DisplayFromStr")] #[parameter(default = SinkDecouple::default())] sink_decouple: SinkDecouple, @@ -231,11 +242,13 @@ pub struct ConfigMap { standard_conforming_strings: String, /// Set streaming rate limit (rows per second) for each parallelism for mv backfilling + #[serde_as(as = "DisplayFromStr")] #[parameter(default = ConfigNonZeroU64::default())] streaming_rate_limit: ConfigNonZeroU64, /// Cache policy for partition cache in streaming over window. /// Can be "full", "recent", "`recent_first_n`" or "`recent_last_n`". + #[serde_as(as = "DisplayFromStr")] #[parameter(default = OverWindowCachePolicy::default(), rename = "rw_streaming_over_window_cache_policy")] streaming_over_window_cache_policy: OverWindowCachePolicy, @@ -282,17 +295,17 @@ fn check_bytea_output(val: &str) -> Result<(), String> { } } -impl ConfigMap { +impl SessionConfig { pub fn set_force_two_phase_agg( &mut self, val: bool, reporter: &mut impl ConfigReporter, - ) -> SessionConfigResult<()> { - self.set_force_two_phase_agg_inner(val, reporter)?; + ) -> SessionConfigResult { + let set_val = self.set_force_two_phase_agg_inner(val, reporter)?; if self.force_two_phase_agg { self.set_enable_two_phase_agg(true, reporter) } else { - Ok(()) + Ok(set_val) } } @@ -300,12 +313,12 @@ impl ConfigMap { &mut self, val: bool, reporter: &mut impl ConfigReporter, - ) -> SessionConfigResult<()> { - self.set_enable_two_phase_agg_inner(val, reporter)?; + ) -> SessionConfigResult { + let set_val = self.set_enable_two_phase_agg_inner(val, reporter)?; if !self.force_two_phase_agg { self.set_force_two_phase_agg(false, reporter) } else { - Ok(()) + Ok(set_val) } } } @@ -332,7 +345,7 @@ mod test { #[derive(SessionConfig)] struct TestConfig { - #[parameter(default = 1, alias = "test_param_alias" | "alias_param_test")] + #[parameter(default = 1, flags = "NO_ALTER_SYS", alias = "test_param_alias" | "alias_param_test")] test_param: i32, } @@ -345,5 +358,6 @@ mod test { .set("alias_param_test", "3".to_string(), &mut ()) .unwrap(); assert_eq!(config.get("test_param_alias").unwrap(), "3"); + assert!(TestConfig::check_no_alter_sys("test_param").unwrap()); } } diff --git a/src/common/src/session_config/search_path.rs b/src/common/src/session_config/search_path.rs index d0707a8bc48a..4573294ab24c 100644 --- a/src/common/src/session_config/search_path.rs +++ b/src/common/src/session_config/search_path.rs @@ -31,7 +31,7 @@ pub const USER_NAME_WILD_CARD: &str = "\"$user\""; /// valid schema in `search_path`. /// /// 3. when we `create` a `index` or `sink`, it will use the schema of the associated table. -#[derive(Clone)] +#[derive(Clone, Debug, PartialEq)] pub struct SearchPath { origin_str: String, /// The path will implicitly includes `rw_catalog` and `pg_catalog` if user does specify them. diff --git a/src/common/src/session_config/transaction_isolation_level.rs b/src/common/src/session_config/transaction_isolation_level.rs index 953c2f834411..31b9c4c1c1ac 100644 --- a/src/common/src/session_config/transaction_isolation_level.rs +++ b/src/common/src/session_config/transaction_isolation_level.rs @@ -15,8 +15,6 @@ use std::fmt::Formatter; use std::str::FromStr; -use crate::error::{bail_not_implemented, NotImplemented}; - #[derive(Copy, Default, Debug, Clone, PartialEq, Eq)] // Some variants are never constructed so allow dead code here. #[allow(dead_code)] @@ -29,10 +27,18 @@ pub enum IsolationLevel { } impl FromStr for IsolationLevel { - type Err = NotImplemented; + type Err = &'static str; - fn from_str(_s: &str) -> Result { - bail_not_implemented!(issue = 10736, "isolation level"); + fn from_str(s: &str) -> Result { + match s.to_ascii_lowercase().as_str() { + "read committed" => Ok(Self::ReadCommitted), + "read uncommitted" => Ok(Self::ReadUncommitted), + "repeatable read" => Ok(Self::RepeatableRead), + "serializable" => Ok(Self::Serializable), + _ => Err( + "expect one of [read committed, read uncommitted, repeatable read, serializable]", + ), + } } } diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index a9dc6250054c..3648f53d5027 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use itertools::Itertools; use parking_lot::RwLock; -use risingwave_common::session_config::{ConfigMap, SearchPath}; +use risingwave_common::session_config::{SearchPath, SessionConfig}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_sqlparser::ast::{ @@ -107,7 +107,7 @@ pub struct Binder { /// and so on. next_share_id: ShareId, - session_config: Arc>, + session_config: Arc>, search_path: SearchPath, /// The type of binding statement. diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 4f5c8941c80d..81137f402ad0 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -31,7 +31,7 @@ use risingwave_common::catalog::{ MAX_SYS_CATALOG_NUM, SYS_CATALOG_START_ID, }; use risingwave_common::error::BoxedError; -use risingwave_common::session_config::ConfigMap; +use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::types::DataType; use risingwave_pb::meta::list_table_fragment_states_response::TableFragmentState; @@ -110,7 +110,7 @@ pub struct SysCatalogReaderImpl { // Read auth context. auth_context: Arc, // Read config. - config: Arc>, + config: Arc>, // Read system params. system_params: SystemParamsReaderRef, } @@ -122,7 +122,7 @@ impl SysCatalogReaderImpl { worker_node_manager: WorkerNodeManagerRef, meta_client: Arc, auth_context: Arc, - config: Arc>, + config: Arc>, system_params: SystemParamsReaderRef, ) -> Self { Self { diff --git a/src/frontend/src/handler/alter_system.rs b/src/frontend/src/handler/alter_system.rs index 9a186aead78f..97117f5b8977 100644 --- a/src/frontend/src/handler/alter_system.rs +++ b/src/frontend/src/handler/alter_system.rs @@ -13,12 +13,13 @@ // limitations under the License. use pgwire::pg_response::StatementType; +use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_sqlparser::ast::{Ident, SetVariableValue}; use super::variable::set_var_to_param_str; use super::{HandlerArgs, RwPgResponse}; -use crate::error::Result; +use crate::error::{ErrorCode, Result}; // Warn user if barrier_interval_ms is set above 5mins. const NOTICE_BARRIER_INTERVAL_MS: u32 = 300000; @@ -31,23 +32,34 @@ pub async fn handle_alter_system( value: SetVariableValue, ) -> Result { let value = set_var_to_param_str(&value); - let params = handler_args - .session - .env() - .meta_client() - .set_system_param(param.to_string(), value) - .await?; + let param_name = param.to_string(); + let meta_client = handler_args.session.env().meta_client(); let mut builder = RwPgResponse::builder(StatementType::ALTER_SYSTEM); - if let Some(params) = params { - if params.barrier_interval_ms() >= NOTICE_BARRIER_INTERVAL_MS { - builder = builder.notice( - format!("Barrier interval is set to {} ms >= {} ms. This can hurt freshness and potentially cause OOM.", - params.barrier_interval_ms(), NOTICE_BARRIER_INTERVAL_MS)); + + // Currently session params are separated from system params. If the param exist in session params, we set it. Otherwise + // we try to set it as a system param. + if SessionConfig::contains_param(¶m_name) { + if SessionConfig::check_no_alter_sys(¶m_name).unwrap() { + return Err(ErrorCode::InternalError(format!( + "session param {} cannot be altered system wide", + param_name + )) + .into()); } - if params.checkpoint_frequency() >= NOTICE_CHECKPOINT_FREQUENCY { - builder = builder.notice( - format!("Checkpoint frequency is set to {} >= {}. This can hurt freshness and potentially cause OOM.", - params.checkpoint_frequency(), NOTICE_CHECKPOINT_FREQUENCY)); + meta_client.set_session_param(param_name, value).await?; + } else { + let params = meta_client.set_system_param(param_name, value).await?; + if let Some(params) = params { + if params.barrier_interval_ms() >= NOTICE_BARRIER_INTERVAL_MS { + builder = builder.notice( + format!("Barrier interval is set to {} ms >= {} ms. This can hurt freshness and potentially cause OOM.", + params.barrier_interval_ms(), NOTICE_BARRIER_INTERVAL_MS)); + } + if params.checkpoint_frequency() >= NOTICE_CHECKPOINT_FREQUENCY { + builder = builder.notice( + format!("Checkpoint frequency is set to {} >= {}. This can hurt freshness and potentially cause OOM.", + params.checkpoint_frequency(), NOTICE_CHECKPOINT_FREQUENCY)); + } } } Ok(builder.into()) diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index 1bfb05aa25a9..678a1684f731 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -14,6 +14,8 @@ use std::collections::HashMap; +use anyhow::Context; +use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_pb::backup_service::MetaSnapshotMetadata; @@ -79,6 +81,10 @@ pub trait FrontendMetaClient: Send + Sync { value: Option, ) -> Result>; + async fn get_session_params(&self) -> Result; + + async fn set_session_param(&self, param: String, value: Option) -> Result; + async fn list_ddl_progress(&self) -> Result>; async fn get_tables(&self, table_ids: &[u32]) -> Result>; @@ -183,6 +189,17 @@ impl FrontendMetaClient for FrontendMetaClientImpl { self.0.set_system_param(param, value).await } + async fn get_session_params(&self) -> Result { + let session_config: SessionConfig = + serde_json::from_str(&self.0.get_session_params().await?) + .context("failed to parse session config")?; + Ok(session_config) + } + + async fn set_session_param(&self, param: String, value: Option) -> Result { + self.0.set_session_param(param, value).await + } + async fn list_ddl_progress(&self) -> Result> { let ddl_progress = self.0.get_ddl_progress().await?; Ok(ddl_progress) diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index 71b52573b530..ddf6ca489bf0 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -20,6 +20,7 @@ use parking_lot::RwLock; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeManagerRef; use risingwave_common::catalog::CatalogVersion; use risingwave_common::hash::ParallelUnitMapping; +use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::observer_manager::{ObserverState, SubscribeFrontend}; use risingwave_pb::common::WorkerNode; @@ -44,6 +45,7 @@ pub struct FrontendObserverNode { user_info_updated_tx: Sender, hummock_snapshot_manager: HummockSnapshotManagerRef, system_params_manager: LocalSystemParamsManagerRef, + session_params: Arc>, compute_client_pool: ComputeClientPoolRef, } @@ -92,6 +94,12 @@ impl ObserverState for FrontendObserverNode { Info::SystemParams(p) => { self.system_params_manager.try_set_params(p); } + Info::SessionParam(p) => { + self.session_params + .write() + .set(&p.param, p.value().to_string(), &mut ()) + .unwrap(); + } Info::HummockStats(stats) => { self.handle_table_stats_notification(stats); } @@ -132,6 +140,7 @@ impl ObserverState for FrontendObserverNode { hummock_version: _, meta_backup_manifest_id: _, hummock_write_limits: _, + session_params, version, } = snapshot; @@ -185,6 +194,8 @@ impl ObserverState for FrontendObserverNode { self.user_info_updated_tx .send(snapshot_version.catalog_version) .unwrap(); + *self.session_params.write() = + serde_json::from_str(&session_params.unwrap().params).unwrap(); } } @@ -197,6 +208,7 @@ impl FrontendObserverNode { user_info_updated_tx: Sender, hummock_snapshot_manager: HummockSnapshotManagerRef, system_params_manager: LocalSystemParamsManagerRef, + session_params: Arc>, compute_client_pool: ComputeClientPoolRef, ) -> Self { Self { @@ -207,6 +219,7 @@ impl FrontendObserverNode { user_info_updated_tx, hummock_snapshot_manager, system_params_manager, + session_params, compute_client_pool, } } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 89a2347f9a83..9ecff36106e3 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -48,7 +48,7 @@ use risingwave_common::config::{ load_config, BatchConfig, MetaConfig, MetricLevel, StreamingConfig, }; use risingwave_common::memory::MemoryContext; -use risingwave_common::session_config::{ConfigMap, ConfigReporter, VisibilityMode}; +use risingwave_common::session_config::{ConfigReporter, SessionConfig, VisibilityMode}; use risingwave_common::system_param::local_manager::{ LocalSystemParamsManager, LocalSystemParamsManagerRef, }; @@ -129,6 +129,7 @@ pub struct FrontendEnv { query_manager: QueryManager, hummock_snapshot_manager: HummockSnapshotManagerRef, system_params_manager: LocalSystemParamsManagerRef, + session_params: Arc>, server_addr: HostAddr, client_pool: ComputeClientPoolRef, @@ -209,6 +210,7 @@ impl FrontendEnv { query_manager, hummock_snapshot_manager, system_params_manager, + session_params: Default::default(), server_addr, client_pool, sessions_map: Arc::new(RwLock::new(HashMap::new())), @@ -303,6 +305,9 @@ impl FrontendEnv { let system_params_manager = Arc::new(LocalSystemParamsManager::new(system_params_reader.clone())); + + // This `session_params` should be initialized during the initial notification in `observer_manager` + let session_params = Arc::new(RwLock::new(SessionConfig::default())); let frontend_observer_node = FrontendObserverNode::new( worker_node_manager.clone(), catalog, @@ -311,6 +316,7 @@ impl FrontendEnv { user_info_updated_tx, hummock_snapshot_manager.clone(), system_params_manager.clone(), + session_params.clone(), compute_client_pool, ); let observer_manager = @@ -411,6 +417,7 @@ impl FrontendEnv { query_manager, hummock_snapshot_manager, system_params_manager, + session_params, server_addr: frontend_address, client_pool, frontend_metrics, @@ -482,6 +489,10 @@ impl FrontendEnv { &self.system_params_manager } + pub fn session_params_snapshot(&self) -> SessionConfig { + self.session_params.read_recursive().clone() + } + pub fn server_address(&self) -> &HostAddr { &self.server_addr } @@ -571,7 +582,7 @@ pub struct SessionImpl { /// Used for user authentication. user_authenticator: UserAuthenticator, /// Stores the value of configurations. - config_map: Arc>, + config_map: Arc>, /// buffer the Notices to users, notices: RwLock>, @@ -625,12 +636,13 @@ impl SessionImpl { user_authenticator: UserAuthenticator, id: SessionId, peer_addr: AddressRef, + session_config: SessionConfig, ) -> Self { Self { env, auth_context, user_authenticator, - config_map: Default::default(), + config_map: Arc::new(RwLock::new(session_config)), id, peer_addr, txn: Default::default(), @@ -689,15 +701,15 @@ impl SessionImpl { self.auth_context.user_id } - pub fn shared_config(&self) -> Arc> { + pub fn shared_config(&self) -> Arc> { Arc::clone(&self.config_map) } - pub fn config(&self) -> RwLockReadGuard<'_, ConfigMap> { + pub fn config(&self) -> RwLockReadGuard<'_, SessionConfig> { self.config_map.read() } - pub fn set_config(&self, key: &str, value: String) -> Result<()> { + pub fn set_config(&self, key: &str, value: String) -> Result { self.config_map .write() .set(key, value, &mut ()) @@ -709,7 +721,7 @@ impl SessionImpl { key: &str, value: Option, mut reporter: impl ConfigReporter, - ) -> Result<()> { + ) -> Result { if let Some(value) = value { self.config_map .write() @@ -1021,6 +1033,9 @@ impl SessionManager for SessionManagerImpl { let secret_key = self.number.fetch_add(1, Ordering::Relaxed); // Use a trivial strategy: process_id and secret_key are equal. let id = (secret_key, secret_key); + // Read session params snapshot from frontend env. + let session_config = self.env.session_params_snapshot(); + let session_impl: Arc = SessionImpl::new( self.env.clone(), Arc::new(AuthContext::new( @@ -1031,6 +1046,7 @@ impl SessionManager for SessionManagerImpl { user_authenticator, id, peer_addr, + session_config, ) .into(); self.insert_session(session_impl.clone()); @@ -1193,7 +1209,7 @@ impl Session for SessionImpl { } } - fn set_config(&self, key: &str, value: String) -> std::result::Result<(), BoxedError> { + fn set_config(&self, key: &str, value: String) -> std::result::Result { Self::set_config(self, key, value).map_err(Into::into) } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index fdbd8fc2912e..fb8cc650b297 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -28,6 +28,7 @@ use risingwave_common::catalog::{ FunctionId, IndexId, TableId, DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, DEFAULT_SUPER_USER, DEFAULT_SUPER_USER_ID, NON_RESERVED_USER_ID, PG_CATALOG_SCHEMA_NAME, RW_CATALOG_SCHEMA_NAME, }; +use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; @@ -194,6 +195,7 @@ impl LocalFrontend { 6666, )) .into(), + Default::default(), )) } } @@ -986,6 +988,14 @@ impl FrontendMetaClient for MockFrontendMetaClient { Ok(Some(SystemParams::default().into())) } + async fn get_session_params(&self) -> RpcResult { + Ok(Default::default()) + } + + async fn set_session_param(&self, _param: String, _value: Option) -> RpcResult { + Ok("".to_string()) + } + async fn list_ddl_progress(&self) -> RpcResult> { Ok(vec![]) } diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index 82fb785b8565..c82a54cbc0e6 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -6,6 +6,7 @@ mod m20230908_072257_init; mod m20231008_020431_hummock; mod m20240304_074901_subscription; mod m20240410_082733_with_version_column_migration; +mod m20240410_154406_session_params; pub struct Migrator; @@ -17,6 +18,7 @@ impl MigratorTrait for Migrator { Box::new(m20231008_020431_hummock::Migration), Box::new(m20240304_074901_subscription::Migration), Box::new(m20240410_082733_with_version_column_migration::Migration), + Box::new(m20240410_154406_session_params::Migration), ] } } diff --git a/src/meta/model_v2/migration/src/m20240410_154406_session_params.rs b/src/meta/model_v2/migration/src/m20240410_154406_session_params.rs new file mode 100644 index 000000000000..e08a34752101 --- /dev/null +++ b/src/meta/model_v2/migration/src/m20240410_154406_session_params.rs @@ -0,0 +1,43 @@ +use sea_orm_migration::prelude::*; + +use crate::{assert_not_has_tables, drop_tables}; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + assert_not_has_tables!(manager, SessionParameter); + manager + .create_table( + Table::create() + .table(SessionParameter::Table) + .col( + ColumnDef::new(SessionParameter::Name) + .string() + .primary_key() + .not_null(), + ) + .col(ColumnDef::new(SessionParameter::Value).string().not_null()) + .col(ColumnDef::new(SessionParameter::Description).string()) + .to_owned(), + ) + .await?; + Ok(()) + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // drop tables cascade. + drop_tables!(manager, SessionParameter); + Ok(()) + } +} + +#[derive(DeriveIden)] +enum SessionParameter { + Table, + Name, + Value, + Description, +} diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index a2eb99c16aed..05fb29cb70ba 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -43,6 +43,7 @@ pub mod index; pub mod object; pub mod object_dependency; pub mod schema; +pub mod session_parameter; pub mod sink; pub mod source; pub mod streaming_job; diff --git a/src/meta/model_v2/src/prelude.rs b/src/meta/model_v2/src/prelude.rs index a8a65cb3b840..7d3c8cde7acb 100644 --- a/src/meta/model_v2/src/prelude.rs +++ b/src/meta/model_v2/src/prelude.rs @@ -32,6 +32,7 @@ pub use super::index::Entity as Index; pub use super::object::Entity as Object; pub use super::object_dependency::Entity as ObjectDependency; pub use super::schema::Entity as Schema; +pub use super::session_parameter::Entity as SessionParameter; pub use super::sink::Entity as Sink; pub use super::source::Entity as Source; pub use super::streaming_job::Entity as StreamingJob; diff --git a/src/meta/model_v2/src/session_parameter.rs b/src/meta/model_v2/src/session_parameter.rs new file mode 100644 index 000000000000..b0623270982f --- /dev/null +++ b/src/meta/model_v2/src/session_parameter.rs @@ -0,0 +1,29 @@ +// 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 sea_orm::entity::prelude::*; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[sea_orm(table_name = "session_parameter")] +pub struct Model { + #[sea_orm(primary_key, auto_increment = false)] + pub name: String, + pub value: String, + pub description: Option, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation {} + +impl ActiveModelBehavior for ActiveModel {} diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 8ec0568c1bd5..134d3843b4cc 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -364,6 +364,7 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { object_store_config: config.storage.object_store, }, config.system.into_init_system_params(), + Default::default(), ) .await .unwrap(); diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index c91ffffe872c..e53759349568 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -22,6 +22,7 @@ use futures::future::join_all; use otlp_embedded::TraceServiceServer; use regex::Regex; use risingwave_common::monitor::connection::{RouterExt, TcpConfig}; +use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::telemetry_env_enabled; @@ -48,6 +49,7 @@ use risingwave_meta_service::meta_member_service::MetaMemberServiceImpl; use risingwave_meta_service::notification_service::NotificationServiceImpl; use risingwave_meta_service::scale_service::ScaleServiceImpl; use risingwave_meta_service::serving_service::ServingServiceImpl; +use risingwave_meta_service::session_config::SessionParamsServiceImpl; use risingwave_meta_service::sink_coordination_service::SinkCoordinationServiceImpl; use risingwave_meta_service::stream_service::StreamServiceImpl; use risingwave_meta_service::system_params_service::SystemParamsServiceImpl; @@ -67,6 +69,7 @@ use risingwave_pb::meta::meta_member_service_server::MetaMemberServiceServer; use risingwave_pb::meta::notification_service_server::NotificationServiceServer; use risingwave_pb::meta::scale_service_server::ScaleServiceServer; use risingwave_pb::meta::serving_service_server::ServingServiceServer; +use risingwave_pb::meta::session_param_service_server::SessionParamServiceServer; use risingwave_pb::meta::stream_manager_service_server::StreamManagerServiceServer; use risingwave_pb::meta::system_params_service_server::SystemParamsServiceServer; use risingwave_pb::meta::telemetry_info_service_server::TelemetryInfoServiceServer; @@ -111,6 +114,7 @@ pub async fn rpc_serve( lease_interval_secs: u64, opts: MetaOpts, init_system_params: SystemParams, + init_session_config: SessionConfig, ) -> MetaResult<(JoinHandle<()>, Option>, WatchSender<()>)> { match meta_store_backend { MetaStoreBackend::Etcd { @@ -153,6 +157,7 @@ pub async fn rpc_serve( lease_interval_secs, opts, init_system_params, + init_session_config, ) } MetaStoreBackend::Mem => { @@ -165,6 +170,7 @@ pub async fn rpc_serve( lease_interval_secs, opts, init_system_params, + init_session_config, ) } MetaStoreBackend::Sql { endpoint } => { @@ -208,6 +214,7 @@ pub async fn rpc_serve( lease_interval_secs, opts, init_system_params, + init_session_config, ) } } @@ -222,6 +229,7 @@ pub fn rpc_serve_with_store( lease_interval_secs: u64, opts: MetaOpts, init_system_params: SystemParams, + init_session_config: SessionConfig, ) -> MetaResult<(JoinHandle<()>, Option>, WatchSender<()>)> { let (svc_shutdown_tx, svc_shutdown_rx) = watch::channel(()); @@ -302,6 +310,7 @@ pub fn rpc_serve_with_store( max_cluster_heartbeat_interval, opts, init_system_params, + init_session_config, election_client, svc_shutdown_rx, ) @@ -373,6 +382,7 @@ pub async fn start_service_as_election_leader( max_cluster_heartbeat_interval: Duration, opts: MetaOpts, init_system_params: SystemParams, + init_session_config: SessionConfig, election_client: Option, mut svc_shutdown_rx: WatchReceiver<()>, ) -> MetaResult<()> { @@ -384,7 +394,13 @@ pub async fn start_service_as_election_leader( .expect("Failed to upgrade models in meta store"); } - let env = MetaSrvEnv::new(opts.clone(), init_system_params, meta_store_impl).await?; + let env = MetaSrvEnv::new( + opts.clone(), + init_system_params, + init_session_config, + meta_store_impl, + ) + .await?; let system_params_reader = env.system_params_reader().await; let data_directory = system_params_reader.data_directory(); @@ -619,6 +635,7 @@ pub async fn start_service_as_election_leader( let backup_srv = BackupServiceImpl::new(backup_manager); let telemetry_srv = TelemetryInfoServiceImpl::new(env.meta_store_ref()); let system_params_srv = SystemParamsServiceImpl::new(env.system_params_manager_impl_ref()); + let session_params_srv = SessionParamsServiceImpl::new(env.session_params_manager_impl_ref()); let serving_srv = ServingServiceImpl::new(serving_vnode_mapping.clone(), metadata_manager.clone()); let cloud_srv = CloudServiceImpl::new(metadata_manager.clone(), aws_cli); @@ -777,6 +794,7 @@ pub async fn start_service_as_election_leader( .add_service(HealthServer::new(health_srv)) .add_service(BackupServiceServer::new(backup_srv)) .add_service(SystemParamsServiceServer::new(system_params_srv)) + .add_service(SessionParamServiceServer::new(session_params_srv)) .add_service(TelemetryInfoServiceServer::new(telemetry_srv)) .add_service(ServingServiceServer::new(serving_srv)) .add_service(CloudServiceServer::new(cloud_srv)) diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 654ecee0e6a3..6e00e6eb4318 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -28,6 +28,7 @@ risingwave_meta = { workspace = true } risingwave_meta_model_v2 = { workspace = true } risingwave_pb = { workspace = true } sea-orm = { workspace = true } +serde_json = "1" sync-point = { path = "../../utils/sync-point" } thiserror-ext = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio", features = [ diff --git a/src/meta/service/src/lib.rs b/src/meta/service/src/lib.rs index 23037be13467..80a83349f2cc 100644 --- a/src/meta/service/src/lib.rs +++ b/src/meta/service/src/lib.rs @@ -32,6 +32,7 @@ pub mod meta_member_service; pub mod notification_service; pub mod scale_service; pub mod serving_service; +pub mod session_config; pub mod sink_coordination_service; pub mod stream_service; pub mod system_params_service; diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index e668ee107354..e9a5e4a017ad 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -12,8 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::Context; use itertools::Itertools; -use risingwave_meta::manager::MetadataManager; +use risingwave_meta::manager::{MetadataManager, SessionParamsManagerImpl}; use risingwave_meta::MetaResult; use risingwave_pb::backup_service::MetaBackupManifestId; use risingwave_pb::catalog::Table; @@ -23,7 +24,8 @@ use risingwave_pb::hummock::WriteLimits; use risingwave_pb::meta::meta_snapshot::SnapshotVersion; use risingwave_pb::meta::notification_service_server::NotificationService; use risingwave_pb::meta::{ - FragmentParallelUnitMapping, MetaSnapshot, SubscribeRequest, SubscribeType, + FragmentParallelUnitMapping, GetSessionParamsResponse, MetaSnapshot, SubscribeRequest, + SubscribeType, }; use risingwave_pb::user::UserInfo; use tokio::sync::mpsc; @@ -246,6 +248,16 @@ impl NotificationServiceImpl { let hummock_snapshot = Some(self.hummock_manager.latest_snapshot()); + let session_params = match self.env.session_params_manager_impl_ref() { + SessionParamsManagerImpl::Kv(manager) => manager.get_params().await, + SessionParamsManagerImpl::Sql(controller) => controller.get_params().await, + }; + + let session_params = Some(GetSessionParamsResponse { + params: serde_json::to_string(&session_params) + .context("failed to encode session params")?, + }); + Ok(MetaSnapshot { databases, schemas, @@ -267,6 +279,7 @@ impl NotificationServiceImpl { parallel_unit_mapping_version, worker_node_version, }), + session_params, ..Default::default() }) } diff --git a/src/meta/service/src/session_config.rs b/src/meta/service/src/session_config.rs new file mode 100644 index 000000000000..a4a7d1b70591 --- /dev/null +++ b/src/meta/service/src/session_config.rs @@ -0,0 +1,77 @@ +// 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 async_trait::async_trait; +use risingwave_meta::manager::SessionParamsManagerImpl; +use risingwave_pb::meta::session_param_service_server::SessionParamService; +use risingwave_pb::meta::{ + GetSessionParamsRequest, GetSessionParamsResponse, SetSessionParamRequest, + SetSessionParamResponse, +}; +use serde_json; +use thiserror_ext::AsReport; +use tonic::{Request, Response, Status}; + +pub struct SessionParamsServiceImpl { + session_params_manager: SessionParamsManagerImpl, +} + +impl SessionParamsServiceImpl { + pub fn new(session_params_manager: SessionParamsManagerImpl) -> Self { + Self { + session_params_manager, + } + } +} + +#[async_trait] +impl SessionParamService for SessionParamsServiceImpl { + async fn get_session_params( + &self, + _request: Request, + ) -> Result, Status> { + let params = match &self.session_params_manager { + SessionParamsManagerImpl::Kv(controller) => controller.get_params().await, + SessionParamsManagerImpl::Sql(manager) => manager.get_params().await, + }; + let params_str = serde_json::to_string(¶ms).map_err(|e| { + Status::internal(format!("Failed to parse session config: {}", e.as_report())) + })?; + + Ok(Response::new(GetSessionParamsResponse { + params: params_str, + })) + } + + async fn set_session_param( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + let req_param = req.get_param(); + + let param_value = match &self.session_params_manager { + SessionParamsManagerImpl::Kv(controller) => { + controller.set_param(req_param, req.value.clone()).await + } + SessionParamsManagerImpl::Sql(manager) => { + manager.set_param(req_param, req.value.clone()).await + } + }; + + Ok(Response::new(SetSessionParamResponse { + param: param_value?, + })) + } +} diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 3d43c77d655a..645a580d8445 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -33,6 +33,7 @@ pub mod cluster; pub mod fragment; pub mod id; pub mod rename; +pub mod session_params; pub mod streaming_job; pub mod system_param; pub mod user; diff --git a/src/meta/src/controller/session_params.rs b/src/meta/src/controller/session_params.rs new file mode 100644 index 000000000000..4a27967fa2b0 --- /dev/null +++ b/src/meta/src/controller/session_params.rs @@ -0,0 +1,211 @@ +// 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::sync::Arc; + +use itertools::Itertools; +use risingwave_common::session_config::{SessionConfig, SessionConfigError}; +use risingwave_meta_model_v2::prelude::SessionParameter; +use risingwave_meta_model_v2::session_parameter; +use risingwave_pb::meta::subscribe_response::{Info, Operation}; +use risingwave_pb::meta::SetSessionParamRequest; +use sea_orm::ActiveValue::Set; +use sea_orm::{ActiveModelTrait, DatabaseConnection, EntityTrait, TransactionTrait}; +use thiserror_ext::AsReport; +use tokio::sync::RwLock; +use tracing::info; + +use crate::controller::SqlMetaStore; +use crate::manager::NotificationManagerRef; +use crate::{MetaError, MetaResult}; + +pub type SessionParamsControllerRef = Arc; + +/// Manages the global default session params on meta. +/// Note that the session params in each session will be initialized from the default value here. +pub struct SessionParamsController { + db: DatabaseConnection, + // Cached parameters. + params: RwLock, + notification_manager: NotificationManagerRef, +} + +impl SessionParamsController { + pub async fn new( + sql_meta_store: SqlMetaStore, + notification_manager: NotificationManagerRef, + mut init_params: SessionConfig, + ) -> MetaResult { + let db = sql_meta_store.conn; + let params = SessionParameter::find().all(&db).await?; + for param in params { + if let Err(e) = init_params.set(¶m.name, param.value, &mut ()) { + match e { + SessionConfigError::InvalidValue { .. } => { + tracing::error!(error = %e.as_report(), "failed to set parameter from meta database, using default value {}", init_params.get(¶m.name)?) + } + SessionConfigError::UnrecognizedEntry(_) => { + tracing::error!(error = %e.as_report(), "failed to set parameter from meta database") + } + } + } + } + + info!(?init_params, "session parameters"); + + let ctl = Self { + db, + params: RwLock::new(init_params.clone()), + notification_manager, + }; + // flush to db. + ctl.flush_params().await?; + + Ok(ctl) + } + + pub async fn get_params(&self) -> SessionConfig { + self.params.read().await.clone() + } + + async fn flush_params(&self) -> MetaResult<()> { + let params = self.params.read().await.list_all(); + let models = params + .into_iter() + .map(|param| session_parameter::ActiveModel { + name: Set(param.name), + value: Set(param.setting), + description: Set(Some(param.description)), + }) + .collect_vec(); + let txn = self.db.begin().await?; + // delete all params first and then insert all params. It follows the same logic + // as the old code, we'd better change it to another way later to keep consistency. + SessionParameter::delete_many().exec(&txn).await?; + SessionParameter::insert_many(models).exec(&txn).await?; + txn.commit().await?; + Ok(()) + } + + pub async fn set_param(&self, name: &str, value: Option) -> MetaResult { + let mut params_guard = self.params.write().await; + let name = SessionConfig::alias_to_entry_name(name); + let Some(param) = SessionParameter::find_by_id(name.clone()) + .one(&self.db) + .await? + else { + return Err(MetaError::system_params(format!( + "unrecognized session parameter {:?}", + name + ))); + }; + // FIXME: use a real reporter + let reporter = &mut (); + let new_param = if let Some(value) = value { + params_guard.set(&name, value, reporter)? + } else { + params_guard.reset(&name, reporter)? + }; + + let mut param: session_parameter::ActiveModel = param.into(); + param.value = Set(new_param.clone()); + param.update(&self.db).await?; + + self.notify_workers(name.to_string(), new_param.clone()); + + Ok(new_param) + } + + pub fn notify_workers(&self, name: String, value: String) { + self.notification_manager.notify_frontend_without_version( + Operation::Update, + Info::SessionParam(SetSessionParamRequest { + param: name, + value: Some(value), + }), + ); + } +} + +#[cfg(test)] +mod tests { + use sea_orm::ColumnTrait; + + use super::*; + use crate::manager::MetaSrvEnv; + + #[tokio::test] + #[cfg(not(madsim))] + async fn test_session_params() { + use sea_orm::QueryFilter; + + let env = MetaSrvEnv::for_test_with_sql_meta_store().await; + let meta_store = env.meta_store().as_sql(); + let init_params = SessionConfig::default(); + + // init system parameter controller as first launch. + let session_param_ctl = SessionParamsController::new( + meta_store.clone(), + env.notification_manager_ref(), + init_params.clone(), + ) + .await + .unwrap(); + let params = session_param_ctl.get_params().await; + assert_eq!(params, init_params); + + // set parameter. + let new_params = session_param_ctl + .set_param("rw_implicit_flush", Some("true".into())) + .await + .unwrap(); + + // insert deprecated params. + let deprecated_param = session_parameter::ActiveModel { + name: Set("deprecated_param".into()), + value: Set("foo".into()), + description: Set(None), + }; + deprecated_param + .insert(&session_param_ctl.db) + .await + .unwrap(); + + // init system parameter controller as not first launch. + let session_param_ctl = SessionParamsController::new( + meta_store.clone(), + env.notification_manager_ref(), + init_params.clone(), + ) + .await + .unwrap(); + // check deprecated params are cleaned up. + assert!(SessionParameter::find_by_id("deprecated_param".to_string()) + .one(&session_param_ctl.db) + .await + .unwrap() + .is_none()); + // check new params are set. + let params = session_param_ctl.get_params().await; + assert_eq!(params.get("rw_implicit_flush").unwrap(), new_params); + // check db consistency. + let models = SessionParameter::find() + .filter(session_parameter::Column::Name.eq("rw_implicit_flush")) + .one(&session_param_ctl.db) + .await + .unwrap() + .unwrap(); + assert_eq!(models.value, params.get("rw_implicit_flush").unwrap()); + } +} diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 3583e8993454..702f02a4ca0a 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -13,6 +13,7 @@ // limitations under the License. use risingwave_common::error::BoxedError; +use risingwave_common::session_config::SessionConfigError; use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; use risingwave_pb::PbFieldNotFound; @@ -88,6 +89,13 @@ pub enum MetaErrorInner { #[error("SystemParams error: {0}")] SystemParams(String), + #[error("SessionParams error: {0}")] + SessionConfig( + #[from] + #[backtrace] + SessionConfigError, + ), + #[error(transparent)] Connector( #[from] diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index f7e16abacb2f..f7c0d34d7a55 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -18,16 +18,20 @@ use std::sync::Arc; use risingwave_common::config::{ CompactionConfig, DefaultParallelism, MetaBackend, ObjectStoreConfig, }; +use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_meta_model_v2::prelude::Cluster; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::{StreamClientPool, StreamClientPoolRef}; use sea_orm::EntityTrait; -use super::{SystemParamsManager, SystemParamsManagerRef}; +use super::{ + SessionParamsManager, SessionParamsManagerRef, SystemParamsManager, SystemParamsManagerRef, +}; use crate::controller::id::{ IdGeneratorManager as SqlIdGeneratorManager, IdGeneratorManagerRef as SqlIdGeneratorManagerRef, }; +use crate::controller::session_params::{SessionParamsController, SessionParamsControllerRef}; use crate::controller::system_param::{SystemParamsController, SystemParamsControllerRef}; use crate::controller::SqlMetaStore; use crate::hummock::sequence::SequenceGenerator; @@ -90,6 +94,12 @@ pub enum SystemParamsManagerImpl { Sql(SystemParamsControllerRef), } +#[derive(Clone)] +pub enum SessionParamsManagerImpl { + Kv(SessionParamsManagerRef), + Sql(SessionParamsControllerRef), +} + /// [`MetaSrvEnv`] is the global environment in Meta service. The instance will be shared by all /// kind of managers inside Meta. #[derive(Clone)] @@ -100,6 +110,9 @@ pub struct MetaSrvEnv { /// system param manager. system_param_manager_impl: SystemParamsManagerImpl, + /// session param manager. + session_param_manager_impl: SessionParamsManagerImpl, + /// meta store. meta_store_impl: MetaStoreImpl, @@ -318,6 +331,7 @@ impl MetaSrvEnv { pub async fn new( opts: MetaOpts, init_system_params: SystemParams, + init_session_config: SessionConfig, meta_store_impl: MetaStoreImpl, ) -> MetaResult { let notification_manager = @@ -347,15 +361,28 @@ impl MetaSrvEnv { ) .await?, ); + let session_params_manager = Arc::new( + SessionParamsManager::new( + meta_store.clone(), + init_session_config.clone(), + notification_manager.clone(), + cluster_first_launch, + ) + .await?, + ); // Persist params before starting services so that invalid params that cause meta node // to crash will not be persisted. system_params_manager.flush_params().await?; + session_params_manager.flush_params().await?; cluster_id.put_at_meta_store(meta_store).await?; Self { id_gen_manager_impl: IdGenManagerImpl::Kv(id_gen_manager), system_param_manager_impl: SystemParamsManagerImpl::Kv(system_params_manager), + session_param_manager_impl: SessionParamsManagerImpl::Kv( + session_params_manager, + ), meta_store_impl: meta_store_impl.clone(), notification_manager, stream_client_pool, @@ -380,7 +407,14 @@ impl MetaSrvEnv { ) .await?, ); - + let session_param_controller = Arc::new( + SessionParamsController::new( + sql_meta_store.clone(), + notification_manager.clone(), + init_session_config, + ) + .await?, + ); Self { id_gen_manager_impl: IdGenManagerImpl::Sql(Arc::new( SqlIdGeneratorManager::new(&sql_meta_store.conn).await?, @@ -388,6 +422,9 @@ impl MetaSrvEnv { system_param_manager_impl: SystemParamsManagerImpl::Sql( system_param_controller, ), + session_param_manager_impl: SessionParamsManagerImpl::Sql( + session_param_controller, + ), meta_store_impl: meta_store_impl.clone(), notification_manager, stream_client_pool, @@ -443,6 +480,10 @@ impl MetaSrvEnv { self.system_param_manager_impl.clone() } + pub fn session_params_manager_impl_ref(&self) -> SessionParamsManagerImpl { + self.session_param_manager_impl.clone() + } + pub fn stream_client_pool_ref(&self) -> StreamClientPoolRef { self.stream_client_pool.clone() } @@ -473,6 +514,7 @@ impl MetaSrvEnv { Self::new( MetaOpts::test(false), risingwave_common::system_param::system_params_for_test(), + Default::default(), MetaStoreImpl::Sql(SqlMetaStore::for_test().await), ) .await @@ -483,6 +525,7 @@ impl MetaSrvEnv { Self::new( opts, risingwave_common::system_param::system_params_for_test(), + Default::default(), MetaStoreImpl::Kv(MemStore::default().into_ref()), ) .await diff --git a/src/meta/src/manager/mod.rs b/src/meta/src/manager/mod.rs index 24c981008425..a7368c72c033 100644 --- a/src/meta/src/manager/mod.rs +++ b/src/meta/src/manager/mod.rs @@ -22,6 +22,7 @@ mod idle; mod metadata; mod notification; mod notification_version; +mod session_params; pub mod sink_coordination; mod streaming_job; mod system_param; @@ -34,5 +35,6 @@ pub use idle::*; pub use metadata::*; pub use notification::{LocalNotification, MessageStatus, NotificationManagerRef, *}; pub use risingwave_meta_model_v2::prelude; +pub use session_params::*; pub use streaming_job::*; pub use system_param::*; diff --git a/src/meta/src/manager/session_params.rs b/src/meta/src/manager/session_params.rs new file mode 100644 index 000000000000..b8fca4f47043 --- /dev/null +++ b/src/meta/src/manager/session_params.rs @@ -0,0 +1,204 @@ +// 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::ops::{Deref, DerefMut}; +use std::sync::Arc; + +use risingwave_common::session_config::{SessionConfig, SessionConfigError}; +use risingwave_pb::meta::subscribe_response::{Info, Operation}; +use risingwave_pb::meta::SetSessionParamRequest; +use thiserror_ext::AsReport; +use tokio::sync::RwLock; +use tracing::info; + +use crate::model::{ValTransaction, VarTransaction}; +use crate::storage::{MetaStore, MetaStoreRef, Snapshot, Transaction}; +use crate::{MetaError, MetaResult}; + +pub type SessionParamsManagerRef = Arc; + +/// Manages the global default session params on meta. +/// Note that the session params in each session will be initialized from the default value here. +pub struct SessionParamsManager { + meta_store: MetaStoreRef, + // Cached parameters. + params: RwLock, + notification_manager: NotificationManagerRef, +} + +impl SessionParamsManager { + /// Return error if `init_params` conflict with persisted system params. + pub async fn new( + meta_store: MetaStoreRef, + init_params: SessionConfig, + notification_manager: NotificationManagerRef, + cluster_first_launch: bool, + ) -> MetaResult { + let params = if cluster_first_launch { + init_params + } else if let Some(params) = + ::get(&meta_store, init_params).await? + { + params + } else { + return Err(MetaError::system_params( + "cluster is not newly created but no session parameters can be found", + )); + }; + + info!(?params, "session parameters"); + + Ok(Self { + meta_store, + params: RwLock::new(params.clone()), + notification_manager, + }) + } + + pub async fn get_params(&self) -> SessionConfig { + self.params.read().await.clone() + } + + pub async fn set_param(&self, name: &str, value: Option) -> MetaResult { + let mut params_guard = self.params.write().await; + let params = params_guard.deref_mut(); + let mut mem_txn = VarTransaction::new(params); + + // FIXME: use a real reporter + let reporter = &mut (); + let new_param = if let Some(value) = value { + mem_txn.set(name, value, reporter)? + } else { + mem_txn.reset(name, reporter)? + }; + let mut store_txn = Transaction::default(); + mem_txn.apply_to_txn(&mut store_txn).await?; + self.meta_store.txn(store_txn).await?; + + mem_txn.commit(); + self.notify_workers(name.to_string(), new_param.clone()); + + Ok(new_param) + } + + /// Flush the cached params to meta store. + pub async fn flush_params(&self) -> MetaResult<()> { + Ok(SessionConfig::insert(self.params.read().await.deref(), &self.meta_store).await?) + } + + pub fn notify_workers(&self, name: String, value: String) { + self.notification_manager.notify_frontend_without_version( + Operation::Update, + Info::SessionParam(SetSessionParamRequest { + param: name, + value: Some(value), + }), + ); + } +} + +use async_trait::async_trait; + +use super::NotificationManagerRef; +use crate::model::{MetadataModelResult, Transactional}; + +const SESSION_PARAMS_CF_NAME: &str = "cf/session_params"; + +// A dummy trait to implement custom methods on `SessionParams`. +#[async_trait] +pub trait SessionParamsModel: Sized { + fn cf_name() -> String; + async fn get( + store: &S, + init_params: SessionConfig, + ) -> MetadataModelResult>; + async fn get_at_snapshot( + store: &S::Snapshot, + init_params: SessionConfig, + ) -> MetadataModelResult>; + async fn insert(&self, store: &S) -> MetadataModelResult<()>; +} + +#[async_trait] +impl SessionParamsModel for SessionConfig { + fn cf_name() -> String { + SESSION_PARAMS_CF_NAME.to_string() + } + + /// Return error if there are missing or unrecognized fields. + async fn get(store: &S, init_params: SessionConfig) -> MetadataModelResult> + where + S: MetaStore, + { + Self::get_at_snapshot::(&store.snapshot().await, init_params).await + } + + async fn get_at_snapshot( + snapshot: &S::Snapshot, + mut init_params: SessionConfig, + ) -> MetadataModelResult> + where + S: MetaStore, + { + let kvs = snapshot.list_cf(&Self::cf_name()).await?; + if kvs.is_empty() { + Ok(None) + } else { + for (k, v) in kvs { + let k = std::str::from_utf8(k.as_ref()).unwrap(); + let v = std::str::from_utf8(v.as_ref()).unwrap(); + if let Err(e) = init_params.set(k, v.to_string(), &mut ()) { + match e { + SessionConfigError::InvalidValue { .. } => { + tracing::error!(error = %e.as_report(), "failed to set parameter from meta database, using default value {}", init_params.get(k).unwrap()) + } + SessionConfigError::UnrecognizedEntry(_) => { + tracing::error!(error = %e.as_report(), "failed to set parameter from meta database") + } + } + } + } + Ok(Some(init_params)) + } + } + + /// All undeprecated fields must be `Some`. + /// Return error if there are missing fields. + async fn insert(&self, store: &S) -> MetadataModelResult<()> + where + S: MetaStore, + { + let mut txn = Transaction::default(); + self.upsert_in_transaction(&mut txn).await?; + Ok(store.txn(txn).await?) + } +} + +#[async_trait] +impl Transactional for SessionConfig { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + for (k, v) in self + .list_all() + .iter() + .map(|info| (info.name.clone(), info.setting.clone())) + { + trx.put(Self::cf_name(), k.into_bytes(), v.into_bytes()); + } + Ok(()) + } + + async fn delete_in_transaction(&self, _trx: &mut Transaction) -> MetadataModelResult<()> { + unreachable!() + } +} diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 0f7bfeb6bccb..eae7cd5019cf 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -77,6 +77,7 @@ use risingwave_pb::meta::meta_member_service_client::MetaMemberServiceClient; use risingwave_pb::meta::notification_service_client::NotificationServiceClient; use risingwave_pb::meta::scale_service_client::ScaleServiceClient; use risingwave_pb::meta::serving_service_client::ServingServiceClient; +use risingwave_pb::meta::session_param_service_client::SessionParamServiceClient; use risingwave_pb::meta::stream_manager_service_client::StreamManagerServiceClient; use risingwave_pb::meta::system_params_service_client::SystemParamsServiceClient; use risingwave_pb::meta::telemetry_info_service_client::TelemetryInfoServiceClient; @@ -1110,6 +1111,18 @@ impl MetaClient { Ok(resp.params.map(SystemParamsReader::from)) } + pub async fn get_session_params(&self) -> Result { + let req = GetSessionParamsRequest {}; + let resp = self.inner.get_session_params(req).await?; + Ok(resp.params) + } + + pub async fn set_session_param(&self, param: String, value: Option) -> Result { + let req = SetSessionParamRequest { param, value }; + let resp = self.inner.set_session_param(req).await?; + Ok(resp.param) + } + pub async fn get_ddl_progress(&self) -> Result> { let req = GetDdlProgressRequest {}; let resp = self.inner.get_ddl_progress(req).await?; @@ -1500,6 +1513,7 @@ struct GrpcMetaClientCore { backup_client: BackupServiceClient, telemetry_client: TelemetryInfoServiceClient, system_params_client: SystemParamsServiceClient, + session_params_client: SessionParamServiceClient, serving_client: ServingServiceClient, cloud_client: CloudServiceClient, sink_coordinate_client: SinkCoordinationRpcClient, @@ -1526,6 +1540,7 @@ impl GrpcMetaClientCore { let telemetry_client = TelemetryInfoServiceClient::new(channel.clone()).max_decoding_message_size(usize::MAX); let system_params_client = SystemParamsServiceClient::new(channel.clone()); + let session_params_client = SessionParamServiceClient::new(channel.clone()); let serving_client = ServingServiceClient::new(channel.clone()); let cloud_client = CloudServiceClient::new(channel.clone()); let sink_coordinate_client = SinkCoordinationServiceClient::new(channel.clone()); @@ -1544,6 +1559,7 @@ impl GrpcMetaClientCore { backup_client, telemetry_client, system_params_client, + session_params_client, serving_client, cloud_client, sink_coordinate_client, @@ -1973,6 +1989,8 @@ macro_rules! for_all_meta_rpc { ,{ telemetry_client, get_telemetry_info, GetTelemetryInfoRequest, TelemetryInfoResponse} ,{ system_params_client, get_system_params, GetSystemParamsRequest, GetSystemParamsResponse } ,{ system_params_client, set_system_param, SetSystemParamRequest, SetSystemParamResponse } + ,{ session_params_client, get_session_params, GetSessionParamsRequest, GetSessionParamsResponse } + ,{ session_params_client, set_session_param, SetSessionParamRequest, SetSessionParamResponse } ,{ serving_client, get_serving_vnode_mappings, GetServingVnodeMappingsRequest, GetServingVnodeMappingsResponse } ,{ cloud_client, rw_cloud_validate_source, RwCloudValidateSourceRequest, RwCloudValidateSourceResponse } ,{ event_log_client, list_event_log, ListEventLogRequest, ListEventLogResponse } diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index 7f6dd41368d4..0538217de9ac 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -113,7 +113,7 @@ pub trait Session: Send + Sync { fn id(&self) -> SessionId; - fn set_config(&self, key: &str, value: String) -> Result<(), BoxedError>; + fn set_config(&self, key: &str, value: String) -> Result; fn transaction_status(&self) -> TransactionStatus; @@ -437,8 +437,8 @@ mod tests { (0, 0) } - fn set_config(&self, _key: &str, _value: String) -> Result<(), BoxedError> { - Ok(()) + fn set_config(&self, _key: &str, _value: String) -> Result { + Ok("".to_string()) } fn take_notices(self: Arc) -> Vec { From b5c02b77c5d9fa9d92b7e230b74eaf8a7cb980d0 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 15 Apr 2024 14:30:34 +0800 Subject: [PATCH 16/64] fix(postgres-cdc): set default ssl.mode to disable (#16299) --- src/connector/src/source/cdc/external/mod.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index ed549fd2e5e2..bd2b623731c7 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -251,7 +251,7 @@ pub struct ExternalTableConfig { pub table: String, /// `ssl.mode` specifies the SSL/TLS encryption level for secure communication with Postgres. /// Choices include `disable`, `prefer`, and `require`. - /// This field is optional. `prefer` is used if not specified. + /// This field is optional. #[serde(rename = "ssl.mode", default = "Default::default")] pub sslmode: SslMode, } @@ -266,7 +266,8 @@ pub enum SslMode { impl Default for SslMode { fn default() -> Self { - Self::Prefer + // default to `disable` for backward compatibility + Self::Disable } } From 72c8becb2e7687cec8a2d763d1dfe0308e029ace Mon Sep 17 00:00:00 2001 From: Li0k Date: Mon, 15 Apr 2024 16:18:37 +0800 Subject: [PATCH 17/64] fix(storage): refactor emergency picker (#15954) --- src/meta/src/hummock/compaction/mod.rs | 30 ++++++++++++--- src/meta/src/hummock/manager/compaction.rs | 43 +++++++++++++++++++++- src/meta/src/hummock/manager/mod.rs | 34 ++--------------- src/meta/src/hummock/manager/versioning.rs | 19 ++++------ 4 files changed, 77 insertions(+), 49 deletions(-) diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index 197a565090a3..49eeaa5778ab 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -32,7 +32,8 @@ use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::{CompactTask, CompactionConfig, LevelType}; pub use selector::CompactionSelector; -use self::selector::LocalSelectorStatistic; +use self::selector::{EmergencySelector, LocalSelectorStatistic}; +use super::check_cg_write_limit; use crate::hummock::compaction::overlap_strategy::{OverlapStrategy, RangeOverlapStrategy}; use crate::hummock::compaction::picker::CompactionInput; use crate::hummock::level_handler::LevelHandler; @@ -101,15 +102,34 @@ impl CompactStatus { // When we compact the files, we must make the result of compaction meet the following // conditions, for any user key, the epoch of it in the file existing in the lower // layer must be larger. - selector.pick_compaction( + if let Some(task) = selector.pick_compaction( task_id, group, levels, &mut self.level_handlers, stats, - table_id_to_options, - developer_config, - ) + table_id_to_options.clone(), + developer_config.clone(), + ) { + return Some(task); + } else { + let compaction_group_config = &group.compaction_config; + if check_cg_write_limit(levels, compaction_group_config.as_ref()).is_write_stop() + && compaction_group_config.enable_emergency_picker + { + return EmergencySelector::default().pick_compaction( + task_id, + group, + levels, + &mut self.level_handlers, + stats, + table_id_to_options, + developer_config, + ); + } + } + + None } pub fn is_trivial_move_task(task: &CompactTask) -> bool { diff --git a/src/meta/src/hummock/manager/compaction.rs b/src/meta/src/hummock/manager/compaction.rs index 065e9745f7db..39dd44565865 100644 --- a/src/meta/src/hummock/manager/compaction.rs +++ b/src/meta/src/hummock/manager/compaction.rs @@ -20,13 +20,16 @@ use futures::future::Shared; use itertools::Itertools; use risingwave_hummock_sdk::{CompactionGroupId, HummockCompactionTaskId}; use risingwave_pb::hummock::compact_task::{TaskStatus, TaskType}; +use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::subscribe_compaction_event_request::{ self, Event as RequestEvent, PullTask, }; use risingwave_pb::hummock::subscribe_compaction_event_response::{ Event as ResponseEvent, PullTaskAck, }; -use risingwave_pb::hummock::{CompactStatus as PbCompactStatus, CompactTaskAssignment}; +use risingwave_pb::hummock::{ + CompactStatus as PbCompactStatus, CompactTaskAssignment, CompactionConfig, +}; use thiserror_ext::AsReport; use tokio::sync::mpsc::UnboundedReceiver; use tokio::sync::oneshot::Receiver as OneShotReceiver; @@ -257,3 +260,41 @@ impl HummockManager { } } } + +pub fn check_cg_write_limit( + levels: &Levels, + compaction_config: &CompactionConfig, +) -> WriteLimitType { + let threshold = compaction_config.level0_stop_write_threshold_sub_level_number as usize; + let l0_sub_level_number = levels.l0.as_ref().unwrap().sub_levels.len(); + if threshold < l0_sub_level_number { + return WriteLimitType::WriteStop(l0_sub_level_number, threshold); + } + + WriteLimitType::Unlimited +} + +pub enum WriteLimitType { + Unlimited, + + // (l0_level_count, threshold) + WriteStop(usize, usize), +} + +impl WriteLimitType { + pub fn as_str(&self) -> String { + match self { + Self::Unlimited => "Unlimited".to_string(), + Self::WriteStop(l0_level_count, threshold) => { + format!( + "WriteStop(l0_level_count: {}, threshold: {}) too many L0 sub levels", + l0_level_count, threshold + ) + } + } + } + + pub fn is_write_stop(&self) -> bool { + matches!(self, Self::WriteStop(_, _)) + } +} diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index eec99e2cdb9a..55d51d1d0d4c 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -114,6 +114,7 @@ mod utils; mod worker; use compaction::*; +pub use compaction::{check_cg_write_limit, WriteLimitType}; pub(crate) use utils::*; type Snapshot = ArcSwap; @@ -2871,6 +2872,7 @@ impl HummockManager { *self.group_to_table_vnode_partition.write() = group_to_table_vnode_partition; } + /// dedicated event runtime for CPU/IO bound event pub fn compaction_event_loop( hummock_manager: Arc, mut compactor_streams_change_rx: UnboundedReceiver<( @@ -3078,39 +3080,14 @@ impl HummockManager { /// This method will return all compaction group id in a random order and task type. If there are any group block by `write_limit`, it will return a single array with `TaskType::Emergency`. /// If these groups get different task-type, it will return all group id with `TaskType::Dynamic` if the first group get `TaskType::Dynamic`, otherwise it will return the single group with other task type. - #[named] pub async fn auto_pick_compaction_groups_and_type( &self, ) -> (Vec, compact_task::TaskType) { - let versioning_guard = read_lock!(self, versioning).await; - let versioning = versioning_guard.deref(); - let mut compaction_group_ids = - get_compaction_group_ids(&versioning.current_version).collect_vec(); + let mut compaction_group_ids = self.compaction_group_ids().await; compaction_group_ids.shuffle(&mut thread_rng()); let mut normal_groups = vec![]; for cg_id in compaction_group_ids { - if versioning.write_limit.contains_key(&cg_id) { - let enable_emergency_picker = match self - .compaction_group_manager - .read() - .await - .try_get_compaction_group_config(cg_id) - { - Some(config) => config.compaction_config.enable_emergency_picker, - None => { - unreachable!("compaction-group {} not exist", cg_id) - } - }; - - if enable_emergency_picker { - if normal_groups.is_empty() { - return (vec![cg_id], TaskType::Emergency); - } else { - break; - } - } - } if let Some(pick_type) = self.compaction_state.auto_pick_type(cg_id) { if pick_type == TaskType::Dynamic { normal_groups.push(cg_id); @@ -3491,10 +3468,6 @@ fn init_selectors() -> HashMap::default(), ); - compaction_selectors.insert( - compact_task::TaskType::Emergency, - Box::::default(), - ); compaction_selectors } @@ -3503,7 +3476,6 @@ use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::HummockVersion; use tokio::sync::mpsc::error::SendError; -use super::compaction::selector::EmergencySelector; use super::compaction::CompactionSelector; use crate::hummock::manager::checkpoint::HummockVersionCheckpoint; use crate::hummock::sequence::next_sstable_object_id; diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index 72fe2867eccd..be510d2aa755 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -35,6 +35,7 @@ use risingwave_pb::hummock::{ }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; +use super::check_cg_write_limit; use crate::hummock::error::Result; use crate::hummock::manager::checkpoint::HummockVersionCheckpoint; use crate::hummock::manager::worker::{HummockManagerEvent, HummockManagerEventSender}; @@ -321,20 +322,14 @@ pub(super) fn calc_new_write_limits( } Some(levels) => levels, }; - // Add write limit conditions here. - let threshold = config - .compaction_config - .level0_stop_write_threshold_sub_level_number as usize; - let l0_sub_level_number = levels.l0.as_ref().unwrap().sub_levels.len(); - if threshold < l0_sub_level_number { + + let write_limit_type = check_cg_write_limit(levels, config.compaction_config.as_ref()); + if write_limit_type.is_write_stop() { new_write_limits.insert( *id, WriteLimit { table_ids: levels.member_table_ids.clone(), - reason: format!( - "too many L0 sub levels: {} > {}", - l0_sub_level_number, threshold - ), + reason: write_limit_type.as_str(), }, ); continue; @@ -519,7 +514,7 @@ mod tests { ); assert_eq!( new_write_limits.get(&1).as_ref().unwrap().reason, - "too many L0 sub levels: 11 > 10" + "WriteStop(l0_level_count: 11, threshold: 10) too many L0 sub levels" ); assert_eq!(new_write_limits.len(), 2); @@ -540,7 +535,7 @@ mod tests { ); assert_eq!( new_write_limits.get(&1).as_ref().unwrap().reason, - "too many L0 sub levels: 11 > 5" + "WriteStop(l0_level_count: 11, threshold: 5) too many L0 sub levels" ); } From f776f107a2786ca471a1abab239c5ba1da9e9c34 Mon Sep 17 00:00:00 2001 From: August Date: Mon, 15 Apr 2024 18:24:43 +0800 Subject: [PATCH 18/64] fix(sql-backend): fix setval function call for postgreSQL backend in migration (#16306) --- src/ctl/src/cmd_impl/meta/migration.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index db3e166c07a6..66fde163eff5 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -816,14 +816,14 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an .conn .execute(Statement::from_string( DatabaseBackend::Postgres, - "SELECT setval('object_oid_seq', SELECT MAX(oid) FROM object);", + "SELECT setval('object_oid_seq', (SELECT MAX(oid) FROM object));", )) .await?; meta_store_sql .conn .execute(Statement::from_string( DatabaseBackend::Postgres, - "SELECT setval('user_user_id_seq', SELECT MAX(user_id) FROM \"user\");", + "SELECT setval('user_user_id_seq', (SELECT MAX(user_id) FROM \"user\"));", )) .await?; } From 00d89589baaa8de70499bcd5b8fcaa450a2349be Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 15 Apr 2024 23:10:20 +0800 Subject: [PATCH 19/64] fix(sqlsmith): ban failing functions and aggregations (#16295) --- ci/scripts/pr-fuzz-test.sh | 1 - ci/scripts/run-deterministic-fuzz-test.sh | 33 +++++++++++++++++++ ci/scripts/run-fuzz-test.sh | 9 ----- ci/workflows/pull-request.yml | 15 +++++++++ src/tests/sqlsmith/scripts/extract_queries.sh | 2 +- src/tests/sqlsmith/src/sql_gen/types.rs | 5 +++ 6 files changed, 54 insertions(+), 11 deletions(-) create mode 100755 ci/scripts/run-deterministic-fuzz-test.sh diff --git a/ci/scripts/pr-fuzz-test.sh b/ci/scripts/pr-fuzz-test.sh index 66923c4fb8a7..f33986e51e45 100755 --- a/ci/scripts/pr-fuzz-test.sh +++ b/ci/scripts/pr-fuzz-test.sh @@ -13,7 +13,6 @@ source ci/scripts/common.sh export RUN_SQLSMITH_FRONTEND=0 export RUN_SQLSMITH=1 export SQLSMITH_COUNT=100 -export TEST_NUM=32 echo "Enabled Sqlsmith tests." source ci/scripts/run-fuzz-test.sh diff --git a/ci/scripts/run-deterministic-fuzz-test.sh b/ci/scripts/run-deterministic-fuzz-test.sh new file mode 100755 index 000000000000..c3b8cb2821fd --- /dev/null +++ b/ci/scripts/run-deterministic-fuzz-test.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash + +# Exits as soon as any line fails. +set -euo pipefail + +export LOGDIR=.risingwave/log +export RUST_LOG=info +mkdir -p $LOGDIR + +while getopts 'p:' opt; do + case ${opt} in + p ) + profile=$OPTARG + ;; + \? ) + echo "Invalid Option: -$OPTARG" 1>&2 + exit 1 + ;; + : ) + echo "Invalid option: $OPTARG requires an argument" 1>&2 + ;; + esac +done +shift $((OPTIND -1)) + +source ci/scripts/common.sh + +echo "--- Download artifacts" +download-and-decompress-artifact risingwave_simulation . +chmod +x ./risingwave_simulation + +echo "--- deterministic simulation e2e, ci-3cn-2fe, fuzzing (seed)" +seq 32 | parallel MADSIM_TEST_SEED={} './risingwave_simulation --sqlsmith 100 ./src/tests/sqlsmith/tests/testdata 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' \ No newline at end of file diff --git a/ci/scripts/run-fuzz-test.sh b/ci/scripts/run-fuzz-test.sh index 441b2797b6a4..067c9b2e5e36 100755 --- a/ci/scripts/run-fuzz-test.sh +++ b/ci/scripts/run-fuzz-test.sh @@ -38,10 +38,6 @@ if [[ "$RUN_SQLSMITH" -eq "1" ]]; then download_and_prepare_rw "$profile" common - # echo "--- Download artifacts" - # download-and-decompress-artifact risingwave_simulation . - # chmod +x ./risingwave_simulation - echo "--- Download sqlsmith e2e bin" download-and-decompress-artifact sqlsmith-"$profile" target/debug/ mv target/debug/sqlsmith-"$profile" target/debug/sqlsmith @@ -70,9 +66,4 @@ if [[ "$RUN_SQLSMITH" -eq "1" ]]; then # Use that to reproduce logs on local machine. echo "--- Kill cluster" risedev kill - - # NOTE(Noel): This is disabled because once it fails, it keeps failing. - # That blocks PRs from getting through. - # echo "--- deterministic simulation e2e, ci-3cn-2fe, fuzzing (seed)" - # seq $TEST_NUM | parallel MADSIM_TEST_SEED={} './risingwave_simulation --sqlsmith 100 ./src/tests/sqlsmith/tests/testdata 2> $LOGDIR/fuzzing-{}.log && rm $LOGDIR/fuzzing-{}.log' fi diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index e0af290a42b0..c19b9a774fc1 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -754,6 +754,21 @@ steps: timeout_in_minutes: 15 retry: *auto-retry + - label: "deterministic fuzz test" + command: "ci/scripts/run-deterministic-fuzz-test.sh -p ci-dev" + if: build.pull_request.labels includes "ci/run-deterministic-sqlsmith-fuzzing-tests" || build.env("CI_STEPS") =~ /(^|,)deterministic-sqlsmith-fuzzing-tests?(,|$$)/ + depends_on: + - "build-simulation" + plugins: + - ./ci/plugins/swapfile + - docker-compose#v5.1.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 15 + retry: *auto-retry + - label: "enable ci/skip-ci only in draft PRs" if: build.pull_request.labels includes "ci/skip-ci" && !build.pull_request.draft commands: diff --git a/src/tests/sqlsmith/scripts/extract_queries.sh b/src/tests/sqlsmith/scripts/extract_queries.sh index 9abac600296a..0dd4c6e72004 100755 --- a/src/tests/sqlsmith/scripts/extract_queries.sh +++ b/src/tests/sqlsmith/scripts/extract_queries.sh @@ -14,7 +14,7 @@ OUTPUT_FILE="$2" SHRUNK_OUTPUT_FILE="$2".shrunk echo "--- Extracting queries" -cat "$LOG_FILE" | rg "\[EXECUTING .*\]" | sed 's/.*\[EXECUTING .*\]: //' | sed 's/$/;/' > "$OUTPUT_FILE" +cat "$LOG_FILE" | rg "(\[EXECUTING|\[TEST)" | sed 's/.*\[EXECUTING .*\]: //' | sed 's/.*\[TEST .*\]: //' | sed 's/$/;/' > "$OUTPUT_FILE" echo "--- Extracted queries to $OUTPUT_FILE" echo "--- Shrinking queries" diff --git a/src/tests/sqlsmith/src/sql_gen/types.rs b/src/tests/sqlsmith/src/sql_gen/types.rs index b4f9d01d294a..4591a91c8383 100644 --- a/src/tests/sqlsmith/src/sql_gen/types.rs +++ b/src/tests/sqlsmith/src/sql_gen/types.rs @@ -112,6 +112,10 @@ static FUNC_BAN_LIST: LazyLock> = LazyLock::new(|| { ExprType::Repeat, // The format argument needs to be handled specially. It is still generated in `gen_special_func`. ExprType::Decode, + // ENABLE: https://github.com/risingwavelabs/risingwave/issues/16293 + ExprType::Sqrt, + // ENABLE: https://github.com/risingwavelabs/risingwave/issues/16293 + ExprType::Pow, ] .into_iter() .collect() @@ -181,6 +185,7 @@ pub(crate) static AGG_FUNC_TABLE: LazyLock Date: Mon, 15 Apr 2024 23:33:45 +0800 Subject: [PATCH 20/64] style: minor improvements for source test and risedev (#16289) Signed-off-by: xxchan --- Makefile.toml | 38 +++++------ ci/scripts/e2e-source-test.sh | 49 ++++++-------- .../scripts/e2e-full-standalone-demo.sh | 4 +- src/risedevtool/src/risedev_env.rs | 67 ++++++++++--------- 4 files changed, 76 insertions(+), 82 deletions(-) diff --git a/Makefile.toml b/Makefile.toml index 37d11266f6d6..604b7b2b4e44 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -558,8 +558,8 @@ env_files = ["${PREFIX_CONFIG}/risedev-env"] script = ''' #!/usr/bin/env bash cat < "${PREFIX_CONFIG}/psql-env" -export PGHOST=$RW_FRONTEND_LISTEN_ADDRESS -export PGPORT=$RW_FRONTEND_PORT +export PGHOST=$RISEDEV_RW_FRONTEND_LISTEN_ADDRESS +export PGPORT=$RISEDEV_RW_FRONTEND_PORT export PGUSER=root export PGDATABASE=dev EOF @@ -576,7 +576,7 @@ dependencies = ["check-risedev-env-file"] env_files = ["${PREFIX_CONFIG}/risedev-env"] script = ''' #!/usr/bin/env bash -psql -h $RW_FRONTEND_LISTEN_ADDRESS -p $RW_FRONTEND_PORT -U root -d dev "$@" +psql -h $RISEDEV_RW_FRONTEND_LISTEN_ADDRESS -p $RISEDEV_RW_FRONTEND_PORT -U root -d dev "$@" ''' [tasks.ctl] @@ -883,7 +883,7 @@ cargo build \ """ [tasks.stest] -category = "RiseDev - Deterministic Simulation" +category = "RiseDev - Test - Deterministic Simulation" description = "Run unit tests in deterministic simulation mode" dependencies = ["install-nextest"] env = { CARGO_TARGET_DIR = "target/sim" } @@ -908,7 +908,7 @@ cargo nextest run \ """ [tasks.sit-test] -category = "RiseDev - Deterministic Simulation" +category = "RiseDev - Test - Deterministic Simulation" description = "Run integration tests in deterministic simulation mode" dependencies = ["install-nextest"] env = { CARGO_TARGET_DIR = "target/sim" } @@ -923,7 +923,7 @@ cargo nextest run \ """ [tasks.sarchive-it-test] -category = "RiseDev - Deterministic Simulation" +category = "RiseDev - Test - Deterministic Simulation" description = "Archive integration tests in deterministic simulation mode" dependencies = ["install-nextest"] env = { CARGO_TARGET_DIR = "target/sim" } @@ -939,7 +939,7 @@ cargo nextest archive \ """ [tasks.scheck] -category = "RiseDev - Deterministic Simulation" +category = "RiseDev - Test - Deterministic Simulation" description = "Run cargo check in deterministic simulation mode" dependencies = ["warn-on-missing-tools"] env = { CARGO_TARGET_DIR = "target/sim" } @@ -964,7 +964,7 @@ cargo check \ """ [tasks.sslt] -category = "RiseDev - Deterministic Simulation" +category = "RiseDev - Test - Deterministic Simulation" description = "Run e2e tests in deterministic simulation mode" dependencies = ["warn-on-missing-tools"] env = { CARGO_TARGET_DIR = "target/sim" } @@ -978,7 +978,7 @@ cargo run \ """ [tasks.sslt-build-all] -category = "RiseDev - Deterministic Simulation" +category = "RiseDev - Test - Deterministic Simulation" description = "Build deterministic simulation runner and tests" dependencies = ["warn-on-missing-tools"] env = { CARGO_TARGET_DIR = "target/sim" } @@ -993,7 +993,7 @@ cargo build \ """ [tasks.sslt-cov] -category = "RiseDev - Deterministic Simulation" +category = "RiseDev - Test - Deterministic Simulation" description = "Run e2e tests in deterministic simulation mode and report code coverage" dependencies = ["install-llvm-cov"] env = { CARGO_TARGET_DIR = "target/sim-cov" } @@ -1290,42 +1290,42 @@ echo "All processes has exited." """ [tasks.slt] -category = "RiseDev - SQLLogicTest" -install_crate = { version = "0.19.1", crate_name = "sqllogictest-bin", binary = "sqllogictest", test_arg = [ +env = { SLT_HOST = "${RISEDEV_RW_FRONTEND_LISTEN_ADDRESS}", SLT_PORT = "${RISEDEV_RW_FRONTEND_PORT}", SLT_DB = "dev" } +category = "RiseDev - Test - SQLLogicTest" +install_crate = { version = "0.20.0", crate_name = "sqllogictest-bin", binary = "sqllogictest", test_arg = [ "--help", ], install_command = "binstall" } dependencies = ["check-risedev-env-file"] env_files = ["${PREFIX_CONFIG}/risedev-env"] -env = { SLT_HOST = "${RW_FRONTEND_LISTEN_ADDRESS}", SLT_PORT = "${RW_FRONTEND_PORT}", SLT_DB = "dev" } command = "sqllogictest" args = ["${@}"] description = "🌟 Run SQLLogicTest" [tasks.slt-streaming] -category = "RiseDev - SQLLogicTest" +category = "RiseDev - Test - SQLLogicTest" extend = "slt" args = ["${@}", "./e2e_test/streaming/**/*.slt"] description = "Run all streaming e2e tests" [tasks.slt-batch] -category = "RiseDev - SQLLogicTest" +category = "RiseDev - Test - SQLLogicTest" extend = "slt" args = ["${@}", "./e2e_test/batch/*.slt"] description = "Run all batch e2e tests" [tasks.slt-generated] -category = "RiseDev - SQLLogicTest" +category = "RiseDev - Test - SQLLogicTest" extend = "slt" args = ["${@}", "./e2e_test/generated/**/*.slt"] description = "Run all generated e2e tests" [tasks.slt-all] -category = "RiseDev - SQLLogicTest" +category = "RiseDev - Test - SQLLogicTest" run_task = { name = ["slt-streaming", "slt-batch", "slt-generated"] } description = "Run all e2e tests" [tasks.docslt] -category = "RiseDev - SQLLogicTest" +category = "RiseDev - Test - SQLLogicTest" description = "Extract SQL examples written in SQLLogicTest syntax from Rust doc comments" script = ''' #!/usr/bin/env bash @@ -1400,7 +1400,7 @@ UPDATE_EXPECT=1 cargo test -p risingwave_connector tests::test_with_options_yaml ''' [tasks.backwards-compat-test] -category = "RiseDev - Backwards Compatibility Test" +category = "RiseDev - Test - Backwards Compatibility Test" description = "Run backwards compatibility test" script = "./backwards-compat-tests/scripts/run_local.sh" diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index f3c99c404f6e..97e104096860 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -5,15 +5,6 @@ set -euo pipefail source ci/scripts/common.sh -# Arguments: -# $1: subject name -# $2: schema file path -function register_schema_registry() { - curl -X POST http://message_queue:8081/subjects/"$1"/versions \ - -H ‘Content-Type: application/vnd.schemaregistry.v1+json’ \ - --data-binary @<(jq -n --arg schema “$(cat "$2")” ‘{schemaType: “PROTOBUF”, schema: "$schema"}’) -} - # prepare environment export CONNECTOR_LIBS_PATH="./connector-node/libs" @@ -73,26 +64,26 @@ echo 'db.runCommand({ping: 1})' | mongo mongodb://mongodb:27017 echo '> rs config' echo 'rs.conf()' | mongo mongodb://mongodb:27017 echo '> run test..' -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/mongodb/**/*.slt' +risedev slt './e2e_test/source/cdc/mongodb/**/*.slt' echo "--- inline cdc test" export MYSQL_HOST=mysql MYSQL_TCP_PORT=3306 MYSQL_PWD=123456 -sqllogictest -p 4566 -d dev './e2e_test/source/cdc_inline/**/*.slt' +risedev slt './e2e_test/source/cdc_inline/**/*.slt' echo "--- opendal source test" -sqllogictest -p 4566 -d dev './e2e_test/source/opendal/**/*.slt' +risedev slt './e2e_test/source/opendal/**/*.slt' echo "--- mysql & postgres cdc validate test" -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.validate.mysql.slt' -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.validate.postgres.slt' +risedev slt './e2e_test/source/cdc/cdc.validate.mysql.slt' +risedev slt './e2e_test/source/cdc/cdc.validate.postgres.slt' echo "--- cdc share source test" # cdc share stream test cases export MYSQL_HOST=mysql MYSQL_TCP_PORT=3306 MYSQL_PWD=123456 -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.share_stream.slt' +risedev slt './e2e_test/source/cdc/cdc.share_stream.slt' # create a share source and check whether heartbeat message is received -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.create_source_job.slt' +risedev slt './e2e_test/source/cdc/cdc.create_source_job.slt' table_id=$(psql -U root -h localhost -p 4566 -d dev -t -c "select id from rw_internal_tables where name like '%mysql_source%';" | xargs); table_count=$(psql -U root -h localhost -p 4566 -d dev -t -c "select count(*) from rw_table(${table_id}, public);" | xargs); if [ "$table_count" -eq 0 ]; then @@ -101,10 +92,10 @@ if [ "$table_count" -eq 0 ]; then fi echo "--- mysql & postgres load and check" -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.load.slt' +risedev slt './e2e_test/source/cdc/cdc.load.slt' # wait for cdc loading sleep 10 -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.check.slt' +risedev slt './e2e_test/source/cdc/cdc.check.slt' # kill cluster risedev kill @@ -135,10 +126,10 @@ echo "> wait for cluster recovery finish" sleep 20 echo "> check mviews after cluster recovery" # check results -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.check_new_rows.slt' +risedev slt './e2e_test/source/cdc/cdc.check_new_rows.slt' # drop relations -sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc_share_stream_drop.slt' +risedev slt './e2e_test/source/cdc/cdc_share_stream_drop.slt' echo "--- Kill cluster" risedev ci-kill @@ -153,8 +144,8 @@ echo "make sure google/protobuf/source_context.proto is NOT in schema registry" curl --silent 'http://message_queue:8081/subjects'; echo # curl --silent --head -X GET 'http://message_queue:8081/subjects/google%2Fprotobuf%2Fsource_context.proto/versions' | grep 404 curl --silent 'http://message_queue:8081/subjects' | grep -v 'google/protobuf/source_context.proto' -sqllogictest -p 4566 -d dev './e2e_test/schema_registry/pb.slt' -sqllogictest -p 4566 -d dev './e2e_test/schema_registry/alter_sr.slt' +risedev slt './e2e_test/schema_registry/pb.slt' +risedev slt './e2e_test/schema_registry/alter_sr.slt' echo "--- Kill cluster" risedev ci-kill @@ -164,19 +155,19 @@ RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=i risedev ci-start ci-pubsub ./scripts/source/prepare_ci_kafka.sh cargo run --bin prepare_ci_pubsub -sqllogictest -p 4566 -d dev './e2e_test/source/basic/*.slt' -sqllogictest -p 4566 -d dev './e2e_test/source/basic/old_row_format_syntax/*.slt' -sqllogictest -p 4566 -d dev './e2e_test/source/basic/alter/kafka.slt' +risedev slt './e2e_test/source/basic/*.slt' +risedev slt './e2e_test/source/basic/old_row_format_syntax/*.slt' +risedev slt './e2e_test/source/basic/alter/kafka.slt' echo "--- e2e, kafka alter source" chmod +x ./scripts/source/prepare_data_after_alter.sh ./scripts/source/prepare_data_after_alter.sh 2 -sqllogictest -p 4566 -d dev './e2e_test/source/basic/alter/kafka_after_new_data.slt' +risedev slt './e2e_test/source/basic/alter/kafka_after_new_data.slt' echo "--- e2e, kafka alter source again" ./scripts/source/prepare_data_after_alter.sh 3 -sqllogictest -p 4566 -d dev './e2e_test/source/basic/alter/kafka_after_new_data_2.slt' +risedev slt './e2e_test/source/basic/alter/kafka_after_new_data_2.slt' echo "--- Run CH-benCHmark" -risedev slt -p 4566 -d dev './e2e_test/ch_benchmark/batch/ch_benchmark.slt' -risedev slt -p 4566 -d dev './e2e_test/ch_benchmark/streaming/*.slt' +risedev slt './e2e_test/ch_benchmark/batch/ch_benchmark.slt' +risedev slt './e2e_test/ch_benchmark/streaming/*.slt' diff --git a/src/cmd_all/scripts/e2e-full-standalone-demo.sh b/src/cmd_all/scripts/e2e-full-standalone-demo.sh index 6c8c01740731..28469aaddbe7 100755 --- a/src/cmd_all/scripts/e2e-full-standalone-demo.sh +++ b/src/cmd_all/scripts/e2e-full-standalone-demo.sh @@ -67,8 +67,8 @@ sleep 15 # FIXME: Integrate standalone into risedev, so we can reuse risedev-env functionality here. cat << EOF > "$RW_PREFIX"/config/risedev-env RW_META_ADDR="http://0.0.0.0:5690" -RW_FRONTEND_LISTEN_ADDRESS="0.0.0.0" -RW_FRONTEND_PORT="4566" +RISEDEV_RW_FRONTEND_LISTEN_ADDRESS="0.0.0.0" +RISEDEV_RW_FRONTEND_PORT="4566" EOF echo "--- Setting up table" diff --git a/src/risedevtool/src/risedev_env.rs b/src/risedevtool/src/risedev_env.rs index f4866353dff7..077f1ce51f82 100644 --- a/src/risedevtool/src/risedev_env.rs +++ b/src/risedevtool/src/risedev_env.rs @@ -25,50 +25,53 @@ use crate::{add_hummock_backend, HummockInMemoryStrategy, ServiceConfig}; pub fn generate_risedev_env(services: &Vec) -> String { let mut env = String::new(); for item in services { - if let ServiceConfig::ComputeNode(c) = item { - // RW_HUMMOCK_URL - // If the cluster is launched without a shared storage, we will skip this. - { - let mut cmd = Command::new("compute-node"); - if add_hummock_backend( - "dummy", - c.provide_opendal.as_ref().unwrap(), - c.provide_minio.as_ref().unwrap(), - c.provide_aws_s3.as_ref().unwrap(), - HummockInMemoryStrategy::Disallowed, - &mut cmd, - ) - .is_ok() + match item { + ServiceConfig::ComputeNode(c) => { + // RW_HUMMOCK_URL + // If the cluster is launched without a shared storage, we will skip this. + { + let mut cmd = Command::new("compute-node"); + if add_hummock_backend( + "dummy", + c.provide_opendal.as_ref().unwrap(), + c.provide_minio.as_ref().unwrap(), + c.provide_aws_s3.as_ref().unwrap(), + HummockInMemoryStrategy::Disallowed, + &mut cmd, + ) + .is_ok() + { + writeln!( + env, + "RW_HUMMOCK_URL=\"{}\"", + cmd.get_args().nth(1).unwrap().to_str().unwrap() + ) + .unwrap(); + } + } + + // RW_META_ADDR { + let meta_node = &c.provide_meta_node.as_ref().unwrap()[0]; writeln!( env, - "RW_HUMMOCK_URL=\"{}\"", - cmd.get_args().nth(1).unwrap().to_str().unwrap() + "RW_META_ADDR=\"http://{}:{}\"", + meta_node.address, meta_node.port ) .unwrap(); } } - - // RW_META_ADDR - { - let meta_node = &c.provide_meta_node.as_ref().unwrap()[0]; + ServiceConfig::Frontend(c) => { + let listen_address = &c.listen_address; writeln!( env, - "RW_META_ADDR=\"http://{}:{}\"", - meta_node.address, meta_node.port + "RISEDEV_RW_FRONTEND_LISTEN_ADDRESS=\"{listen_address}\"", ) .unwrap(); + let port = &c.port; + writeln!(env, "RISEDEV_RW_FRONTEND_PORT=\"{port}\"",).unwrap(); } - break; - } - } - for item in services { - if let ServiceConfig::Frontend(c) = item { - let listen_address = &c.listen_address; - writeln!(env, "RW_FRONTEND_LISTEN_ADDRESS=\"{listen_address}\"",).unwrap(); - let port = &c.port; - writeln!(env, "RW_FRONTEND_PORT=\"{port}\"",).unwrap(); - break; + _ => {} } } env From cb0ea416756d5f59dae9555196ba715836f57d63 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Mon, 15 Apr 2024 20:21:08 -0500 Subject: [PATCH 21/64] fix(compatibility): backwards compatibility of session params (#16310) --- src/meta/src/manager/session_params.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/meta/src/manager/session_params.rs b/src/meta/src/manager/session_params.rs index b8fca4f47043..662eae162a92 100644 --- a/src/meta/src/manager/session_params.rs +++ b/src/meta/src/manager/session_params.rs @@ -24,7 +24,7 @@ use tracing::info; use crate::model::{ValTransaction, VarTransaction}; use crate::storage::{MetaStore, MetaStoreRef, Snapshot, Transaction}; -use crate::{MetaError, MetaResult}; +use crate::MetaResult; pub type SessionParamsManagerRef = Arc; @@ -48,13 +48,13 @@ impl SessionParamsManager { let params = if cluster_first_launch { init_params } else if let Some(params) = - ::get(&meta_store, init_params).await? + ::get(&meta_store, init_params.clone()).await? { params } else { - return Err(MetaError::system_params( - "cluster is not newly created but no session parameters can be found", - )); + tracing::warn!("Cluster is not newly created but no session parameters can be found. \ + Possibly caused by upgrading from a version where system wide session parameter was not supported"); + init_params }; info!(?params, "session parameters"); From 211b9b93ecbd18c32b3e2f9b4bc1974453a9d6ce Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 16 Apr 2024 10:14:45 +0800 Subject: [PATCH 22/64] feat(frontend, meta): support `RECOVER` command to trigger recovery (#16259) --- proto/meta.proto | 5 ++ src/frontend/src/handler/mod.rs | 2 + src/frontend/src/handler/recover.rs | 38 ++++++++++++++ src/frontend/src/meta_client.rs | 6 +++ src/frontend/src/test_utils.rs | 4 ++ src/meta/service/src/stream_service.rs | 14 ++++- src/meta/src/barrier/mod.rs | 51 ++++++++++++++++--- src/meta/src/error.rs | 4 ++ src/meta/src/manager/notification.rs | 1 + src/rpc_client/src/meta_client.rs | 7 +++ src/sqlparser/src/ast/mod.rs | 6 +++ src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 1 + .../tests/integration_tests/backfill_tests.rs | 25 +++++++++ src/utils/pgwire/src/pg_response.rs | 1 + 15 files changed, 158 insertions(+), 8 deletions(-) create mode 100644 src/frontend/src/handler/recover.rs diff --git a/proto/meta.proto b/proto/meta.proto index e48dc485d495..dadc5b364c62 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -266,6 +266,10 @@ message ApplyThrottleResponse { common.Status status = 1; } +message RecoverRequest {} + +message RecoverResponse {} + service StreamManagerService { rpc Flush(FlushRequest) returns (FlushResponse); rpc Pause(PauseRequest) returns (PauseResponse); @@ -277,6 +281,7 @@ service StreamManagerService { rpc ListActorStates(ListActorStatesRequest) returns (ListActorStatesResponse); rpc ListObjectDependencies(ListObjectDependenciesRequest) returns (ListObjectDependenciesResponse); rpc ApplyThrottle(ApplyThrottleRequest) returns (ApplyThrottleResponse); + rpc Recover(RecoverRequest) returns (RecoverResponse); } // Below for cluster service. diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 4d5f8527fb19..8b60eeeeef2b 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -87,6 +87,7 @@ pub mod handle_privilege; mod kill_process; pub mod privilege; pub mod query; +mod recover; pub mod show; mod transaction; pub mod util; @@ -509,6 +510,7 @@ pub async fn handle( } Statement::Flush => flush::handle_flush(handler_args).await, Statement::Wait => wait::handle_wait(handler_args).await, + Statement::Recover => recover::handle_recover(handler_args).await, Statement::SetVariable { local: _, variable, diff --git a/src/frontend/src/handler/recover.rs b/src/frontend/src/handler/recover.rs new file mode 100644 index 000000000000..0a2c7d1837d3 --- /dev/null +++ b/src/frontend/src/handler/recover.rs @@ -0,0 +1,38 @@ +// 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 pgwire::pg_response::{PgResponse, StatementType}; + +use super::RwPgResponse; +use crate::error::{ErrorCode, Result}; +use crate::handler::HandlerArgs; +use crate::session::SessionImpl; + +pub(super) async fn handle_recover(handler_args: HandlerArgs) -> Result { + // Only permit recovery for super users. + if !handler_args.session.is_super_user() { + return Err(ErrorCode::PermissionDenied( + "only superusers can trigger adhoc recovery".to_string(), + ) + .into()); + } + do_recover(&handler_args.session).await?; + Ok(PgResponse::empty_result(StatementType::RECOVER)) +} + +pub(crate) async fn do_recover(session: &SessionImpl) -> Result<()> { + let client = session.env().meta_client(); + client.recover().await?; + Ok(()) +} diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index 678a1684f731..3cc7e22cf8b2 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -52,6 +52,8 @@ pub trait FrontendMetaClient: Send + Sync { async fn wait(&self) -> Result<()>; + async fn recover(&self) -> Result<()>; + async fn cancel_creating_jobs(&self, jobs: PbJobs) -> Result>; async fn list_table_fragments( @@ -137,6 +139,10 @@ impl FrontendMetaClient for FrontendMetaClientImpl { self.0.wait().await } + async fn recover(&self) -> Result<()> { + self.0.recover().await + } + async fn cancel_creating_jobs(&self, infos: PbJobs) -> Result> { self.0.cancel_creating_jobs(infos).await } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index fb8cc650b297..bda668379725 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -1055,6 +1055,10 @@ impl FrontendMetaClient for MockFrontendMetaClient { async fn list_compact_task_progress(&self) -> RpcResult> { unimplemented!() } + + async fn recover(&self) -> RpcResult<()> { + unimplemented!() + } } #[cfg(test)] diff --git a/src/meta/service/src/stream_service.rs b/src/meta/service/src/stream_service.rs index 6c2d40e2c581..cf9a8b1a3e48 100644 --- a/src/meta/service/src/stream_service.rs +++ b/src/meta/service/src/stream_service.rs @@ -16,7 +16,7 @@ use std::collections::{HashMap, HashSet}; use itertools::Itertools; use risingwave_common::catalog::TableId; -use risingwave_meta::manager::MetadataManager; +use risingwave_meta::manager::{LocalNotification, MetadataManager}; use risingwave_meta::model; use risingwave_meta::model::ActorId; use risingwave_meta::stream::ThrottleConfig; @@ -411,4 +411,16 @@ impl StreamManagerService for StreamServiceImpl { dependencies, })) } + + #[cfg_attr(coverage, coverage(off))] + async fn recover( + &self, + _request: Request, + ) -> Result, Status> { + self.env + .notification_manager() + .notify_local_subscribers(LocalNotification::AdhocRecovery) + .await; + Ok(Response::new(RecoverResponse {})) + } } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 63df19631adc..c99940fbc32f 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -57,6 +57,7 @@ use crate::barrier::notifier::BarrierInfo; use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::rpc::ControlStreamManager; use crate::barrier::state::BarrierManagerState; +use crate::error::MetaErrorInner; use crate::hummock::{CommitEpochInfo, HummockManagerRef, NewTableFragmentInfo}; use crate::manager::sink_coordination::SinkCoordinatorManager; use crate::manager::{ @@ -118,6 +119,8 @@ enum RecoveryReason { Bootstrap, /// After failure. Failover(MetaError), + /// Manually triggered + Adhoc, } /// Status of barrier manager. @@ -651,14 +654,20 @@ impl GlobalBarrierManager { } } - // Checkpoint frequency changes. notification = local_notification_rx.recv() => { let notification = notification.unwrap(); - // Handle barrier interval and checkpoint frequency changes - if let LocalNotification::SystemParamsChange(p) = ¬ification { - self.scheduled_barriers.set_min_interval(Duration::from_millis(p.barrier_interval_ms() as u64)); - self.scheduled_barriers - .set_checkpoint_frequency(p.checkpoint_frequency() as usize) + match notification { + // Handle barrier interval and checkpoint frequency changes. + LocalNotification::SystemParamsChange(p) => { + self.scheduled_barriers.set_min_interval(Duration::from_millis(p.barrier_interval_ms() as u64)); + self.scheduled_barriers + .set_checkpoint_frequency(p.checkpoint_frequency() as usize) + }, + // Handle adhoc recovery triggered by user. + LocalNotification::AdhocRecovery => { + self.adhoc_recovery().await; + } + _ => {} } } resp_result = self.control_stream_manager.next_response() => { @@ -788,7 +797,7 @@ impl GlobalBarrierManager { err.clone(), ))); let latest_snapshot = self.context.hummock_manager.latest_snapshot(); - let prev_epoch = TracedEpoch::new(latest_snapshot.committed_epoch.into()); // we can only recovery from the committed epoch + let prev_epoch = TracedEpoch::new(latest_snapshot.committed_epoch.into()); // we can only recover from the committed epoch let span = tracing::info_span!( "failure_recovery", error = %err.as_report(), @@ -803,6 +812,31 @@ impl GlobalBarrierManager { panic!("failed to execute barrier: {}", err.as_report()); } } + + async fn adhoc_recovery(&mut self) { + let err = MetaErrorInner::AdhocRecovery.into(); + self.context.tracker.lock().await.abort_all(&err); + self.checkpoint_control.clear_on_err(&err).await; + + if self.enable_recovery { + self.context + .set_status(BarrierManagerStatus::Recovering(RecoveryReason::Adhoc)); + let latest_snapshot = self.context.hummock_manager.latest_snapshot(); + let prev_epoch = TracedEpoch::new(latest_snapshot.committed_epoch.into()); // we can only recover from the committed epoch + let span = tracing::info_span!( + "adhoc_recovery", + error = %err.as_report(), + prev_epoch = prev_epoch.value().0 + ); + + // No need to clean dirty tables for barrier recovery, + // The foreground stream job should cleanup their own tables. + self.recovery(None).instrument(span).await; + self.context.set_status(BarrierManagerStatus::Running); + } else { + panic!("failed to execute barrier: {}", err.as_report()); + } + } } impl GlobalBarrierManagerContext { @@ -1031,6 +1065,9 @@ impl GlobalBarrierManagerContext { BarrierManagerStatus::Recovering(RecoveryReason::Failover(e)) => { Err(anyhow::anyhow!(e.clone()).context("The cluster is recovering"))? } + BarrierManagerStatus::Recovering(RecoveryReason::Adhoc) => { + bail!("The cluster is recovering-adhoc") + } BarrierManagerStatus::Running => Ok(()), } } diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 702f02a4ca0a..72eb9de2f67d 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -119,6 +119,10 @@ pub enum MetaErrorInner { #[backtrace] anyhow::Error, ), + + // Indicates that recovery was triggered manually. + #[error("adhoc recovery triggered")] + AdhocRecovery, } impl MetaError { diff --git a/src/meta/src/manager/notification.rs b/src/meta/src/manager/notification.rs index dc54c1e1b12e..0ce47608cdfd 100644 --- a/src/meta/src/manager/notification.rs +++ b/src/meta/src/manager/notification.rs @@ -47,6 +47,7 @@ pub enum LocalNotification { SystemParamsChange(SystemParamsReader), FragmentMappingsUpsert(Vec), FragmentMappingsDelete(Vec), + AdhocRecovery, } #[derive(Debug)] diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index eae7cd5019cf..d438c13ba7a0 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -798,6 +798,12 @@ impl MetaClient { Ok(()) } + pub async fn recover(&self) -> Result<()> { + let request = RecoverRequest {}; + self.inner.recover(request).await?; + Ok(()) + } + pub async fn cancel_creating_jobs(&self, jobs: PbJobs) -> Result> { let request = CancelCreatingJobsRequest { jobs: Some(jobs) }; let resp = self.inner.cancel_creating_jobs(request).await?; @@ -1903,6 +1909,7 @@ macro_rules! for_all_meta_rpc { ,{ stream_client, list_fragment_distribution, ListFragmentDistributionRequest, ListFragmentDistributionResponse } ,{ stream_client, list_actor_states, ListActorStatesRequest, ListActorStatesResponse } ,{ stream_client, list_object_dependencies, ListObjectDependenciesRequest, ListObjectDependenciesResponse } + ,{ stream_client, recover, RecoverRequest, RecoverResponse } ,{ ddl_client, create_table, CreateTableRequest, CreateTableResponse } ,{ ddl_client, alter_name, AlterNameRequest, AlterNameResponse } ,{ ddl_client, alter_owner, AlterOwnerRequest, AlterOwnerResponse } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index eeeeb85a932b..82b59d1bf462 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1519,6 +1519,8 @@ pub enum Statement { /// WAIT for ALL running stream jobs to finish. /// It will block the current session the condition is met. Wait, + /// Trigger stream job recover + Recover, } impl fmt::Display for Statement { @@ -2108,6 +2110,10 @@ impl fmt::Display for Statement { write!(f, "KILL {}", process_id)?; Ok(()) } + Statement::Recover => { + write!(f, "RECOVER")?; + Ok(()) + } } } } diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index c5cd8adbe73a..208a642eb484 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -402,6 +402,7 @@ define_keywords!( READ, READS, REAL, + RECOVER, RECURSIVE, REF, REFERENCES, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index fbef8bac45a5..aaf400b43879 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -283,6 +283,7 @@ impl Parser { Keyword::CLOSE => Ok(self.parse_close_cursor()?), Keyword::FLUSH => Ok(Statement::Flush), Keyword::WAIT => Ok(Statement::Wait), + Keyword::RECOVER => Ok(Statement::Recover), _ => self.expected( "an SQL statement", Token::Word(w).with_location(token.location), diff --git a/src/tests/simulation/tests/integration_tests/backfill_tests.rs b/src/tests/simulation/tests/integration_tests/backfill_tests.rs index a97a52a095da..bcc271bcb4dc 100644 --- a/src/tests/simulation/tests/integration_tests/backfill_tests.rs +++ b/src/tests/simulation/tests/integration_tests/backfill_tests.rs @@ -305,3 +305,28 @@ async fn test_enable_arrangement_backfill() -> Result<()> { assert!(!result.contains("ArrangementBackfill")); Ok(()) } + +#[tokio::test] +async fn test_recovery_cancels_foreground_ddl() -> Result<()> { + let mut cluster = Cluster::start(Configuration::enable_arrangement_backfill()).await?; + let mut session = cluster.start_session(); + session.run("SET STREAMING_RATE_LIMIT=1").await?; + session.run("CREATE TABLE t(v1 int);").await?; + session + .run("INSERT INTO t select * from generate_series(1, 100000);") + .await?; + let handle = tokio::spawn(async move { + session + .run("CREATE MATERIALIZED VIEW m1 AS SELECT * FROM t;") + .await + }); + sleep(Duration::from_secs(2)).await; + cluster.run("RECOVER").await?; + match handle.await? { + Ok(_) => panic!("create m1 should fail"), + Err(e) => { + assert!(e.to_string().contains("adhoc recovery triggered")); + } + } + Ok(()) +} diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 0742ac62f728..1431909f5366 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -105,6 +105,7 @@ pub enum StatementType { CLOSE_CURSOR, WAIT, KILL, + RECOVER, } impl std::fmt::Display for StatementType { From c1ef666eb4c435138011ecd6426523832292c943 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Tue, 16 Apr 2024 11:06:25 +0800 Subject: [PATCH 23/64] refactor: move rate limit inside executor (#15948) Signed-off-by: tabVersion Co-authored-by: Noel Kwan --- src/meta/src/controller/streaming_job.rs | 17 ++- src/meta/src/manager/catalog/fragment.rs | 15 ++- .../executor/backfill/arrangement_backfill.rs | 42 +++--- .../src/executor/backfill/cdc/cdc_backfill.rs | 11 ++ .../backfill/cdc/upstream_table/snapshot.rs | 24 +++- .../executor/backfill/no_shuffle_backfill.rs | 76 ++++++----- src/stream/src/executor/backfill/utils.rs | 19 +++ src/stream/src/executor/flow_control.rs | 123 ------------------ src/stream/src/executor/mod.rs | 2 - .../src/executor/source/fetch_executor.rs | 19 ++- .../src/executor/source/fs_source_executor.rs | 34 ++++- .../source/source_backfill_executor.rs | 6 +- .../src/executor/source/source_executor.rs | 66 +++++++++- src/stream/src/from_proto/source/fs_fetch.rs | 8 +- .../src/from_proto/source/trad_source.rs | 8 +- src/stream/src/from_proto/stream_cdc_scan.rs | 10 +- src/stream/src/from_proto/stream_scan.rs | 12 +- .../tests/integration_tests/backfill_tests.rs | 2 +- 18 files changed, 274 insertions(+), 220 deletions(-) delete mode 100644 src/stream/src/executor/flow_control.rs diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 68bbc5fd025d..271860ee4956 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -951,12 +951,21 @@ impl CatalogController { fragments.retain_mut(|(_, fragment_type_mask, stream_node)| { let mut found = false; - if *fragment_type_mask & PbFragmentTypeFlag::StreamScan as i32 != 0 { - visit_stream_node(stream_node, |node| { - if let PbNodeBody::StreamScan(node) = node { + if (*fragment_type_mask & PbFragmentTypeFlag::StreamScan as i32 != 0) + || (*fragment_type_mask & PbFragmentTypeFlag::Source as i32 != 0) + { + visit_stream_node(stream_node, |node| match node { + PbNodeBody::StreamScan(node) => { node.rate_limit = rate_limit; found = true; } + PbNodeBody::Source(node) => { + if let Some(inner) = node.source_inner.as_mut() { + inner.rate_limit = rate_limit; + found = true; + } + } + _ => {} }); } found @@ -964,7 +973,7 @@ impl CatalogController { if fragments.is_empty() { return Err(MetaError::invalid_parameter(format!( - "stream scan node not found in job id {job_id}" + "stream scan node or source node not found in job id {job_id}" ))); } let fragment_ids = fragments.iter().map(|(id, _, _)| *id).collect_vec(); diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 97fce87ebfb8..feb3dc3026db 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -952,15 +952,24 @@ impl FragmentManager { let mut fragment_to_apply = HashMap::new(); for fragment in fragment.fragments.values_mut() { - if (fragment.get_fragment_type_mask() & FragmentTypeFlag::StreamScan as u32) != 0 { + if (fragment.get_fragment_type_mask() & FragmentTypeFlag::StreamScan as u32) != 0 + || (fragment.get_fragment_type_mask() & FragmentTypeFlag::Source as u32) != 0 + { let mut actor_to_apply = Vec::new(); for actor in &mut fragment.actors { if let Some(node) = actor.nodes.as_mut() { - visit_stream_node(node, |node_body| { - if let NodeBody::StreamScan(ref mut node) = node_body { + visit_stream_node(node, |node_body| match node_body { + NodeBody::StreamScan(ref mut node) => { node.rate_limit = rate_limit; actor_to_apply.push(actor.actor_id); } + NodeBody::Source(ref mut node) => { + if let Some(ref mut node_inner) = node.source_inner { + node_inner.rate_limit = rate_limit; + actor_to_apply.push(actor.actor_id); + } + } + _ => {} }) }; } diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 6a4c11717426..2fea4b313c88 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -33,9 +33,9 @@ use crate::common::table::state_table::{ReplicatedStateTable, StateTable}; #[cfg(debug_assertions)] use crate::executor::backfill::utils::METADATA_STATE_LEN; use crate::executor::backfill::utils::{ - compute_bounds, create_builder, get_progress_per_vnode, mapping_chunk, mapping_message, - mark_chunk_ref_by_vnode, owned_row_iter, persist_state_per_vnode, update_pos_by_vnode, - BackfillProgressPerVnode, BackfillState, + compute_bounds, create_builder, create_limiter, get_progress_per_vnode, mapping_chunk, + mapping_message, mark_chunk_ref_by_vnode, owned_row_iter, persist_state_per_vnode, + update_pos_by_vnode, BackfillProgressPerVnode, BackfillRateLimiter, BackfillState, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -215,6 +215,9 @@ where let mut upstream_chunk_buffer: Vec = vec![]; let mut pending_barrier: Option = None; + let rate_limiter = self.rate_limit.and_then(create_limiter); + let rate_limit = self.rate_limit; + let backfill_snapshot_read_row_count_metric = self .metrics .backfill_snapshot_read_row_count @@ -243,10 +246,14 @@ where { let left_upstream = upstream.by_ref().map(Either::Left); + // Check if stream paused + let paused = paused || matches!(rate_limit, Some(0)); + // Create the snapshot stream let right_snapshot = pin!(Self::make_snapshot_stream( &upstream_table, backfill_state.clone(), // FIXME: Use mutable reference instead. paused, + &rate_limiter, ) .map(Either::Right)); @@ -290,7 +297,7 @@ where // Consume remaining rows in the builder. for (vnode, builder) in &mut builders { if let Some(data_chunk) = builder.consume_all() { - yield Self::handle_snapshot_chunk( + yield Message::Chunk(Self::handle_snapshot_chunk( data_chunk, *vnode, &pk_in_output_indices, @@ -298,7 +305,7 @@ where &mut cur_barrier_snapshot_processed_rows, &mut total_snapshot_processed_rows, &self.output_indices, - )?; + )?); } } @@ -326,7 +333,7 @@ where Some((vnode, row)) => { let builder = builders.get_mut(&vnode).unwrap(); if let Some(chunk) = builder.append_one_row(row) { - yield Self::handle_snapshot_chunk( + yield Message::Chunk(Self::handle_snapshot_chunk( chunk, vnode, &pk_in_output_indices, @@ -334,7 +341,7 @@ where &mut cur_barrier_snapshot_processed_rows, &mut total_snapshot_processed_rows, &self.output_indices, - )?; + )?); } } } @@ -366,7 +373,7 @@ where Some((vnode, row)) => { let builder = builders.get_mut(&vnode).unwrap(); if let Some(chunk) = builder.append_one_row(row) { - yield Self::handle_snapshot_chunk( + yield Message::Chunk(Self::handle_snapshot_chunk( chunk, vnode, &pk_in_output_indices, @@ -374,7 +381,7 @@ where &mut cur_barrier_snapshot_processed_rows, &mut total_snapshot_processed_rows, &self.output_indices, - )?; + )?); } break; @@ -585,10 +592,11 @@ where } #[try_stream(ok = Option<(VirtualNode, OwnedRow)>, error = StreamExecutorError)] - async fn make_snapshot_stream( - upstream_table: &ReplicatedStateTable, + async fn make_snapshot_stream<'a>( + upstream_table: &'a ReplicatedStateTable, backfill_state: BackfillState, paused: bool, + rate_limiter: &'a Option, ) { if paused { #[for_await] @@ -596,9 +604,14 @@ where yield None; } } else { + // Checked the rate limit is not zero. #[for_await] for r in Self::snapshot_read_per_vnode(upstream_table, backfill_state) { - yield r?; + let r = r?; + if let Some(rate_limit) = rate_limiter { + rate_limit.until_ready().await; + } + yield r; } } } @@ -611,7 +624,7 @@ where cur_barrier_snapshot_processed_rows: &mut u64, total_snapshot_processed_rows: &mut u64, output_indices: &[usize], - ) -> StreamExecutorResult { + ) -> StreamExecutorResult { let chunk = StreamChunk::from_parts(vec![Op::Insert; chunk.capacity()], chunk); // Raise the current position. // As snapshot read streams are ordered by pk, so we can @@ -628,8 +641,7 @@ where let chunk_cardinality = chunk.cardinality() as u64; *cur_barrier_snapshot_processed_rows += chunk_cardinality; *total_snapshot_processed_rows += chunk_cardinality; - let chunk = Message::Chunk(mapping_chunk(chunk, output_indices)); - Ok(chunk) + Ok(mapping_chunk(chunk, output_indices)) } /// Read snapshot per vnode. diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 9ccaa74ce6f3..234eceea9fc9 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -272,6 +272,17 @@ impl CdcBackfillExecutor { paused = false; valve.resume(); } + Mutation::Throttle(some) => { + if let Some(rate_limit) = + some.get(&self.actor_ctx.id) + { + self.chunk_size = rate_limit + .map(|x| x as usize) + .unwrap_or(self.chunk_size); + // rebuild the new reader stream with new chunk size + continue 'backfill_loop; + } + } _ => (), } } diff --git a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs index 6835b7cd2c77..be98a6822035 100644 --- a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs +++ b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs @@ -13,9 +13,12 @@ // limitations under the License. use std::future::Future; +use std::num::NonZeroU32; use futures::{pin_mut, Stream}; use futures_async_stream::try_stream; +use governor::clock::MonotonicClock; +use governor::{Quota, RateLimiter}; use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::row::OwnedRow; @@ -104,9 +107,28 @@ impl UpstreamTableRead for UpstreamTableReader { let mut builder = DataChunkBuilder::new(self.inner.schema().data_types(), args.chunk_size); let chunk_stream = iter_chunks(row_stream, &mut builder); + + if args.chunk_size == 0 { + // If limit is 0, we should not read any data from the upstream table. + // Keep waiting util the stream is rebuilt. + let future = futures::future::pending::<()>(); + future.await; + } + let limiter = { + let quota = Quota::per_second(NonZeroU32::new(args.chunk_size as u32).unwrap()); + let clock = MonotonicClock; + RateLimiter::direct_with_clock(quota, &clock) + }; #[for_await] for chunk in chunk_stream { - yield Some(chunk?); + let chunk = chunk?; + if chunk.cardinality() != 0 { + limiter + .until_n_ready(NonZeroU32::new(chunk.cardinality() as u32).unwrap()) + .await + .unwrap(); + } + yield Some(chunk); } yield None; } diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 46597d941427..b409e183f5a9 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -33,8 +33,9 @@ use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; use crate::executor::backfill::utils; use crate::executor::backfill::utils::{ - compute_bounds, construct_initial_finished_state, create_builder, get_new_pos, mapping_chunk, - mapping_message, mark_chunk, owned_row_iter, METADATA_STATE_LEN, + compute_bounds, construct_initial_finished_state, create_builder, create_limiter, get_new_pos, + mapping_chunk, mapping_message, mark_chunk, owned_row_iter, BackfillRateLimiter, + METADATA_STATE_LEN, }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ @@ -165,6 +166,8 @@ where tracing::trace!(is_finished, row_count, "backfill state recovered"); let data_types = self.upstream_table.schema().data_types(); + + // Chunk builder will be instantiated with min(rate_limit, self.chunk_size) as the chunk's max size. let mut builder = create_builder(rate_limit, self.chunk_size, data_types.clone()); // Use this buffer to construct state, @@ -216,6 +219,7 @@ where if !is_finished { let mut upstream_chunk_buffer: Vec = vec![]; let mut pending_barrier: Option = None; + let mut rate_limiter = rate_limit.and_then(create_limiter); let backfill_snapshot_read_row_count_metric = self .metrics @@ -244,12 +248,13 @@ where { let left_upstream = upstream.by_ref().map(Either::Left); - + let paused = paused || matches!(rate_limit, Some(0)); let right_snapshot = pin!(Self::make_snapshot_stream( &self.upstream_table, snapshot_read_epoch, current_pos.clone(), - paused + paused, + &rate_limiter, ) .map(Either::Right)); @@ -293,14 +298,14 @@ where None => { // Consume remaining rows in the builder. if let Some(data_chunk) = builder.consume_all() { - yield Self::handle_snapshot_chunk( + yield Message::Chunk(Self::handle_snapshot_chunk( data_chunk, &mut current_pos, &mut cur_barrier_snapshot_processed_rows, &mut total_snapshot_processed_rows, &pk_indices, &self.output_indices, - ); + )); } // End of the snapshot read stream. @@ -327,14 +332,14 @@ where Some(record) => { // Buffer the snapshot read row. if let Some(data_chunk) = builder.append_one_row(record) { - yield Self::handle_snapshot_chunk( + yield Message::Chunk(Self::handle_snapshot_chunk( data_chunk, &mut current_pos, &mut cur_barrier_snapshot_processed_rows, &mut total_snapshot_processed_rows, &pk_indices, &self.output_indices, - ); + )); } } } @@ -367,14 +372,14 @@ where } Some(row) => { let chunk = DataChunk::from_rows(&[row], &data_types); - yield Self::handle_snapshot_chunk( + yield Message::Chunk(Self::handle_snapshot_chunk( chunk, &mut current_pos, &mut cur_barrier_snapshot_processed_rows, &mut total_snapshot_processed_rows, &pk_indices, &self.output_indices, - ); + )); break; } } @@ -397,14 +402,14 @@ where // Consume snapshot rows left in builder let chunk = builder.consume_all(); if let Some(chunk) = chunk { - yield Self::handle_snapshot_chunk( + yield Message::Chunk(Self::handle_snapshot_chunk( chunk, &mut current_pos, &mut cur_barrier_snapshot_processed_rows, &mut total_snapshot_processed_rows, &pk_indices, &self.output_indices, - ); + )); } // Consume upstream buffer chunk @@ -467,18 +472,26 @@ where Mutation::Throttle(actor_to_apply) => { let new_rate_limit_entry = actor_to_apply.get(&self.actor_id); if let Some(new_rate_limit) = new_rate_limit_entry { - rate_limit = new_rate_limit.as_ref().map(|x| *x as _); - tracing::info!( - id = self.actor_id, - new_rate_limit = ?self.rate_limit, - "actor rate limit changed", - ); - assert!(builder.is_empty()); - builder = create_builder( - rate_limit, - self.chunk_size, - self.upstream_table.schema().data_types(), - ); + let new_rate_limit = new_rate_limit.as_ref().map(|x| *x as _); + if new_rate_limit != rate_limit { + rate_limit = new_rate_limit; + tracing::info!( + id = self.actor_id, + new_rate_limit = ?self.rate_limit, + "actor rate limit changed", + ); + // The builder is emptied above via `DataChunkBuilder::consume_all`. + assert!( + builder.is_empty(), + "builder should already be emptied" + ); + builder = create_builder( + rate_limit, + self.chunk_size, + self.upstream_table.schema().data_types(), + ); + rate_limiter = new_rate_limit.and_then(create_limiter); + } } } _ => (), @@ -633,11 +646,12 @@ where } #[try_stream(ok = Option, error = StreamExecutorError)] - async fn make_snapshot_stream( - upstream_table: &StorageTable, + async fn make_snapshot_stream<'a>( + upstream_table: &'a StorageTable, epoch: u64, current_pos: Option, paused: bool, + rate_limiter: &'a Option, ) { if paused { #[for_await] @@ -645,8 +659,12 @@ where yield None; } } else { + // Checked the rate limit is not zero. #[for_await] for r in Self::snapshot_read(upstream_table, epoch, current_pos) { + if let Some(rate_limit) = &rate_limiter { + rate_limit.until_ready().await; + } yield r?; } } @@ -721,7 +739,7 @@ where /// 2. Update the current position. /// 3. Update Metrics /// 4. Map the chunk according to output indices, return - /// the stream message to be yielded downstream. + /// the stream chunk and do wrapping outside. fn handle_snapshot_chunk( data_chunk: DataChunk, current_pos: &mut Option, @@ -729,7 +747,7 @@ where total_snapshot_processed_rows: &mut u64, pk_indices: &[usize], output_indices: &[usize], - ) -> Message { + ) -> StreamChunk { let ops = vec![Op::Insert; data_chunk.capacity()]; let chunk = StreamChunk::from_parts(ops, data_chunk); // Raise the current position. @@ -741,7 +759,7 @@ where *cur_barrier_snapshot_processed_rows += chunk_cardinality; *total_snapshot_processed_rows += chunk_cardinality; - Message::Chunk(mapping_chunk(chunk, output_indices)) + mapping_chunk(chunk, output_indices) } } diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index a0e85f33c810..1d469b311fd2 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -14,13 +14,19 @@ use std::cmp::Ordering; use std::collections::HashMap; +use std::num::NonZeroU32; use std::ops::Bound; +use std::time::Instant; use await_tree::InstrumentAwait; use bytes::Bytes; use futures::future::try_join_all; use futures::{pin_mut, Stream, StreamExt}; use futures_async_stream::try_stream; +use governor::clock::MonotonicClock; +use governor::middleware::NoOpMiddleware; +use governor::state::{InMemoryState, NotKeyed}; +use governor::{Quota, RateLimiter}; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bail; @@ -799,6 +805,9 @@ pub(crate) async fn persist_state( Ok(()) } +pub type BackfillRateLimiter = + RateLimiter>; + /// Creates a data chunk builder for snapshot read. /// If the `rate_limit` is smaller than `chunk_size`, it will take precedence. /// This is so we can partition snapshot read into smaller chunks than chunk size. @@ -809,9 +818,19 @@ pub fn create_builder( ) -> DataChunkBuilder { if let Some(rate_limit) = rate_limit && rate_limit < chunk_size + && rate_limit > 0 { DataChunkBuilder::new(data_types, rate_limit) } else { DataChunkBuilder::new(data_types, chunk_size) } } + +pub fn create_limiter(rate_limit: usize) -> Option { + if rate_limit == 0 { + return None; + } + let quota = Quota::per_second(NonZeroU32::new(rate_limit as u32).unwrap()); + let clock = MonotonicClock; + Some(RateLimiter::direct_with_clock(quota, &clock)) +} diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs deleted file mode 100644 index 984dddab9fb3..000000000000 --- a/src/stream/src/executor/flow_control.rs +++ /dev/null @@ -1,123 +0,0 @@ -// 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::fmt::{Debug, Formatter}; -use std::num::NonZeroU32; - -use futures::StreamExt; -use futures_async_stream::try_stream; -use governor::clock::MonotonicClock; -use governor::{Quota, RateLimiter}; - -use super::{ - ActorContextRef, BoxedMessageStream, Execute, Executor, Message, Mutation, StreamExecutorError, -}; - -/// Flow Control Executor is used to control the rate of the input executor. -/// -/// Currently it is placed after the `BackfillExecutor`: -/// upstream `MaterializeExecutor` -> `BackfillExecutor` -> `FlowControlExecutor` -/// -/// The rate limit is set statically at the moment, and cannot be changed in a running -/// stream graph. -/// -/// It is used to throttle problematic MVs that are consuming too much resources. -pub struct FlowControlExecutor { - input: Executor, - actor_ctx: ActorContextRef, - rate_limit: Option, -} - -impl FlowControlExecutor { - pub fn new(input: Executor, actor_ctx: ActorContextRef, rate_limit: Option) -> Self { - Self { - input, - actor_ctx, - rate_limit, - } - } - - #[try_stream(ok = Message, error = StreamExecutorError)] - async fn execute_inner(mut self) { - let get_rate_limiter = |rate_limit: u32| { - let quota = Quota::per_second(NonZeroU32::new(rate_limit).unwrap()); - let clock = MonotonicClock; - RateLimiter::direct_with_clock(quota, &clock) - }; - let mut rate_limiter = self.rate_limit.map(get_rate_limiter); - if self.rate_limit.is_some() { - tracing::info!(rate_limit = ?self.rate_limit, "actor starts with rate limit"); - } - - #[for_await] - for msg in self.input.execute() { - let msg = msg?; - match msg { - Message::Chunk(chunk) => { - let chunk_cardinality = chunk.cardinality(); - let Some(n) = NonZeroU32::new(chunk_cardinality as u32) else { - // Handle case where chunk is empty - continue; - }; - if let Some(rate_limiter) = &rate_limiter { - let limit = NonZeroU32::new(self.rate_limit.unwrap()).unwrap(); - if n <= limit { - // `InsufficientCapacity` should never happen because we have done the check - rate_limiter.until_n_ready(n).await.unwrap(); - yield Message::Chunk(chunk); - } else { - // Cut the chunk into smaller chunks - for chunk in chunk.split(limit.get() as usize) { - let n = NonZeroU32::new(chunk.cardinality() as u32).unwrap(); - // Ditto. - rate_limiter.until_n_ready(n).await.unwrap(); - yield Message::Chunk(chunk); - } - } - } else { - yield Message::Chunk(chunk); - } - } - Message::Barrier(barrier) => { - if let Some(mutation) = barrier.mutation.as_ref() { - if let Mutation::Throttle(actor_to_apply) = mutation.as_ref() { - if let Some(limit) = actor_to_apply.get(&self.actor_ctx.id) { - self.rate_limit = *limit; - rate_limiter = self.rate_limit.map(get_rate_limiter); - tracing::info!(new_rate_limit = ?self.rate_limit, "actor rate limit changed"); - } - } - } - - yield Message::Barrier(barrier); - } - _ => yield msg, - } - } - } -} - -impl Debug for FlowControlExecutor { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("FlowControlExecutor") - .field("rate_limit", &self.rate_limit) - .finish() - } -} - -impl Execute for FlowControlExecutor { - fn execute(self: Box) -> BoxedMessageStream { - self.execute_inner().boxed() - } -} diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index a8878763422b..ab53b02ef7ce 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -65,7 +65,6 @@ mod dynamic_filter; pub mod error; mod expand; mod filter; -mod flow_control; mod hash_agg; pub mod hash_join; mod hop_window; @@ -119,7 +118,6 @@ pub use dynamic_filter::DynamicFilterExecutor; pub use error::{StreamExecutorError, StreamExecutorResult}; pub use expand::ExpandExecutor; pub use filter::FilterExecutor; -pub use flow_control::FlowControlExecutor; pub use hash_agg::HashAggExecutor; pub use hash_join::*; pub use hop_window::HopWindowExecutor; diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 3c6d89fe59a5..1d6e409c63a2 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -39,6 +39,7 @@ use risingwave_storage::StateStore; use thiserror_ext::AsReport; use super::{get_split_offset_col_idx, SourceStateTableHandler}; +use crate::executor::source_executor::apply_rate_limit; use crate::executor::stream_reader::StreamReaderWithPause; use crate::executor::{ expect_first_barrier, get_split_offset_mapping_from_chunk, prune_additional_cols, @@ -152,11 +153,13 @@ impl FsFetchExecutor { source_desc: &SourceDesc, batch: SplitBatch, ) -> StreamExecutorResult { - source_desc + let rate_limit = source_ctx.source_ctrl_opts.rate_limit; + let stream = source_desc .source .to_stream(batch, column_ids, Arc::new(source_ctx)) .await - .map_err(StreamExecutorError::connector_error) + .map_err(StreamExecutorError::connector_error)?; + Ok(apply_rate_limit(stream, rate_limit).boxed()) } fn build_source_ctx( @@ -234,10 +237,20 @@ impl FsFetchExecutor { Either::Left(msg) => { match &msg { Message::Barrier(barrier) => { + let mut need_rebuild_reader = false; + if let Some(mutation) = barrier.mutation.as_deref() { match mutation { Mutation::Pause => stream.pause_stream(), Mutation::Resume => stream.resume_stream(), + Mutation::Throttle(actor_to_apply) => { + if let Some(throttle) = + actor_to_apply.get(&self.actor_ctx.id) + { + self.source_ctrl_opts.rate_limit = *throttle; + need_rebuild_reader = true; + } + } _ => (), } } @@ -261,7 +274,7 @@ impl FsFetchExecutor { } } - if splits_on_fetch == 0 { + if splits_on_fetch == 0 || need_rebuild_reader { Self::replace_with_new_batch_reader( &mut splits_on_fetch, &state_store_handler, diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 73b76c247864..13b876c05c2b 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -38,6 +38,7 @@ use super::executor_core::StreamSourceCore; use crate::error::StreamResult; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; +use crate::executor::source_executor::apply_rate_limit; use crate::executor::stream_reader::StreamReaderWithPause; use crate::executor::*; @@ -104,11 +105,33 @@ impl FsSourceExecutor { source_desc.source.config.clone(), self.stream_source_core.source_name.clone(), ); - source_desc + let stream = source_desc .source .to_stream(state, column_ids, Arc::new(source_ctx)) .await - .map_err(StreamExecutorError::connector_error) + .map_err(StreamExecutorError::connector_error)?; + + Ok(apply_rate_limit(stream, self.source_ctrl_opts.rate_limit).boxed()) + } + + async fn rebuild_stream_reader( + &mut self, + source_desc: &FsSourceDesc, + stream: &mut StreamReaderWithPause, + ) -> StreamExecutorResult<()> { + let target_state: Vec = self + .stream_source_core + .latest_split_info + .values() + .cloned() + .collect(); + let reader = self + .build_stream_source_reader(source_desc, Some(target_state)) + .await? + .map_err(StreamExecutorError::connector_error); + stream.replace_data_stream(reader); + + Ok(()) } async fn apply_split_change( @@ -387,6 +410,13 @@ impl FsSourceExecutor { ) .await?; } + Mutation::Throttle(actor_to_apply) => { + if let Some(throttle) = actor_to_apply.get(&self.actor_ctx.id) { + self.source_ctrl_opts.rate_limit = *throttle; + self.rebuild_stream_reader(&source_desc, &mut stream) + .await?; + } + } _ => {} } } diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index 6f8fca84bafd..be752b4598a3 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -41,6 +41,7 @@ use thiserror_ext::AsReport; use super::executor_core::StreamSourceCore; use super::source_backfill_state_table::BackfillStateTableHandler; use crate::executor::monitor::StreamingMetrics; +use crate::executor::source_executor::apply_rate_limit; use crate::executor::*; #[derive(Clone, Debug, Deserialize, Serialize)] @@ -214,11 +215,12 @@ impl SourceBackfillExecutorInner { source_desc.source.config.clone(), self.stream_source_core.source_name.clone(), ); - source_desc + let stream = source_desc .source .to_stream(Some(splits), column_ids, Arc::new(source_ctx)) .await - .map_err(StreamExecutorError::connector_error) + .map_err(StreamExecutorError::connector_error)?; + Ok(apply_rate_limit(stream, self.source_ctrl_opts.rate_limit).boxed()) } #[try_stream(ok = Message, error = StreamExecutorError)] diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 65635d0e4074..d86a85ec5bde 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::fmt::Formatter; +use std::num::NonZeroU32; use std::str::FromStr; use std::time::Duration; @@ -20,9 +21,12 @@ use anyhow::anyhow; use either::Either; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; +use governor::clock::MonotonicClock; +use governor::{Quota, RateLimiter}; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; +use risingwave_connector::error::ConnectorError; use risingwave_connector::source::cdc::jni_source; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ @@ -63,6 +67,54 @@ pub struct SourceExecutor { source_ctrl_opts: SourceCtrlOpts, } +#[try_stream(ok = StreamChunk, error = ConnectorError)] +pub async fn apply_rate_limit(stream: BoxChunkSourceStream, rate_limit: Option) { + if let Some(limit) = rate_limit + && limit == 0 + { + // block the stream until the rate limit is reset + let future = futures::future::pending::<()>(); + future.await; + } + let get_rate_limiter = |rate_limit: u32| { + let quota = Quota::per_second(NonZeroU32::new(rate_limit).unwrap()); + let clock = MonotonicClock; + RateLimiter::direct_with_clock(quota, &clock) + }; + let limiter = rate_limit.map(get_rate_limiter); + if rate_limit.is_some() { + tracing::info!(rate_limit = ?rate_limit, "applied rate limit"); + } + #[for_await] + for batch in stream { + let chunk: StreamChunk = batch?; + let chunk_cardinality = chunk.cardinality(); + let Some(n) = NonZeroU32::new(chunk_cardinality as u32) else { + // pass empty chunk + yield chunk; + continue; + }; + if let Some(limiter) = &limiter { + let limit = NonZeroU32::new(rate_limit.unwrap()).unwrap(); + if n <= limit { + // `InsufficientCapacity` should never happen because we have done the check + limiter.until_n_ready(n).await.unwrap(); + yield chunk; + } else { + // Cut the chunk into smaller chunks + for chunk in chunk.split(limit.get() as usize) { + let n = NonZeroU32::new(chunk.cardinality() as u32).unwrap(); + // Ditto. + limiter.until_n_ready(n).await.unwrap(); + yield chunk; + } + } + } else { + yield chunk; + } + } +} + impl SourceExecutor { pub fn new( actor_ctx: ActorContextRef, @@ -117,11 +169,13 @@ impl SourceExecutor { .source_name .clone(), ); - source_desc + let stream = source_desc .source .to_stream(state, column_ids, Arc::new(source_ctx)) .await - .map_err(StreamExecutorError::connector_error) + .map_err(StreamExecutorError::connector_error); + + Ok(apply_rate_limit(stream?, self.source_ctrl_opts.rate_limit).boxed()) } /// `source_id | source_name | actor_id | fragment_id` @@ -500,6 +554,14 @@ impl SourceExecutor { ) .await?; } + Mutation::Throttle(actor_to_apply) => { + if let Some(throttle) = actor_to_apply.get(&self.actor_ctx.id) { + self.source_ctrl_opts.rate_limit = *throttle; + // recreate from latest_split_info + self.rebuild_stream_reader(&source_desc, &mut stream) + .await?; + } + } _ => {} } } diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index d3aa7e0be40a..e07670fb97a6 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -25,8 +25,7 @@ use risingwave_storage::StateStore; use crate::error::StreamResult; use crate::executor::{ - Execute, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler, - StreamSourceCore, + Execute, Executor, FsFetchExecutor, SourceStateTableHandler, StreamSourceCore, }; use crate::from_proto::ExecutorBuilder; use crate::task::ExecutorParams; @@ -119,11 +118,6 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { } _ => unreachable!(), }; - let mut info = params.info.clone(); - info.identity = format!("{} (flow controlled)", info.identity); - - let rate_limit = source.rate_limit.map(|x| x as _); - let exec = FlowControlExecutor::new((info, exec).into(), params.actor_context, rate_limit); Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 7f8bff9d3c6f..ca057ced81f6 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -38,7 +38,7 @@ use super::*; use crate::executor::source::{FsListExecutor, StreamSourceCore}; use crate::executor::source_executor::SourceExecutor; use crate::executor::state_table_handler::SourceStateTableHandler; -use crate::executor::{FlowControlExecutor, TroublemakerExecutor}; +use crate::executor::TroublemakerExecutor; const FS_CONNECTORS: &[&str] = &["s3"]; pub struct SourceExecutorBuilder; @@ -244,12 +244,6 @@ impl ExecutorBuilder for SourceExecutorBuilder { .boxed() } }; - let mut info = params.info.clone(); - info.identity = format!("{} (flow controlled)", info.identity); - - let rate_limit = source.rate_limit.map(|x| x as _); - let exec = - FlowControlExecutor::new((info, exec).into(), params.actor_context, rate_limit); if crate::consistency::insane() { let mut info = params.info.clone(); diff --git a/src/stream/src/from_proto/stream_cdc_scan.rs b/src/stream/src/from_proto/stream_cdc_scan.rs index 2736fdd712cb..2f035fc4f270 100644 --- a/src/stream/src/from_proto/stream_cdc_scan.rs +++ b/src/stream/src/from_proto/stream_cdc_scan.rs @@ -23,7 +23,7 @@ use risingwave_pb::stream_plan::StreamCdcScanNode; use super::*; use crate::common::table::state_table::StateTable; -use crate::executor::{CdcBackfillExecutor, Executor, ExternalStorageTable, FlowControlExecutor}; +use crate::executor::{CdcBackfillExecutor, Executor, ExternalStorageTable}; pub struct StreamCdcScanExecutorBuilder; @@ -106,14 +106,6 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { backfill_chunk_size, disable_backfill, ); - let mut info = params.info.clone(); - info.identity = format!("{} (flow controlled)", info.identity); - - let exec = FlowControlExecutor::new( - (info, exec).into(), - params.actor_context, - node.rate_limit.map(|x| x as _), - ); Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index 669b9a247245..56ee7bd36f6e 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -24,8 +24,8 @@ use risingwave_storage::table::batch_table::storage_table::StorageTable; use super::*; use crate::common::table::state_table::{ReplicatedStateTable, StateTable}; use crate::executor::{ - ArrangementBackfillExecutor, BackfillExecutor, ChainExecutor, FlowControlExecutor, - RearrangedChainExecutor, TroublemakerExecutor, + ArrangementBackfillExecutor, BackfillExecutor, ChainExecutor, RearrangedChainExecutor, + TroublemakerExecutor, }; pub struct StreamScanExecutorBuilder; @@ -145,14 +145,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { } StreamScanType::Unspecified => unreachable!(), }; - let mut info = params.info.clone(); - info.identity = format!("{} (flow controlled)", info.identity); - - let exec = FlowControlExecutor::new( - (info, exec).into(), - params.actor_context, - node.rate_limit.map(|x| x as _), - ); if crate::consistency::insane() { let mut info = params.info.clone(); diff --git a/src/tests/simulation/tests/integration_tests/backfill_tests.rs b/src/tests/simulation/tests/integration_tests/backfill_tests.rs index bcc271bcb4dc..ba34f45e6af6 100644 --- a/src/tests/simulation/tests/integration_tests/backfill_tests.rs +++ b/src/tests/simulation/tests/integration_tests/backfill_tests.rs @@ -264,7 +264,7 @@ async fn test_arrangement_backfill_progress() -> Result<()> { let progress = progress.replace('%', ""); let progress = progress.parse::().unwrap(); assert!( - (1.0..2.0).contains(&progress), + (1.0..10.0).contains(&progress), "progress not within bounds {}", progress ); From bd930b44e6809a4d4d877e57bb34ed6d27828576 Mon Sep 17 00:00:00 2001 From: xfz <73645462+xuefengze@users.noreply.github.com> Date: Tue, 16 Apr 2024 11:43:35 +0800 Subject: [PATCH 24/64] test(sink): add compatibility starrocks sink test (#16317) --- .../starrocks-sink/create_sink.sql | 15 +++++++++ .../starrocks-sink/create_source.sql | 33 +++++++++++++++++++ .../starrocks-sink/sink_check.py | 2 +- .../starrocks-sink/starrocks_prepare.sql | 31 +++++++++++++++++ 4 files changed, 80 insertions(+), 1 deletion(-) diff --git a/integration_tests/starrocks-sink/create_sink.sql b/integration_tests/starrocks-sink/create_sink.sql index 5cd98ba1ec17..0e96e1f20789 100644 --- a/integration_tests/starrocks-sink/create_sink.sql +++ b/integration_tests/starrocks-sink/create_sink.sql @@ -72,3 +72,18 @@ FROM starrocks.table = 'upsert_table', primary_key = 'user_id' ); + +CREATE SINK starrocks_types_sink +FROM + starrocks_types WITH ( + connector = 'starrocks', + type = 'append-only', + starrocks.host = 'starrocks-fe', + starrocks.mysqlport = '9030', + starrocks.httpport = '8030', + starrocks.user = 'users', + starrocks.password = '123456', + starrocks.database = 'demo', + starrocks.table = 'starrocks_types', + force_append_only='true' +); diff --git a/integration_tests/starrocks-sink/create_source.sql b/integration_tests/starrocks-sink/create_source.sql index 0e4230851112..e462eb873df8 100644 --- a/integration_tests/starrocks-sink/create_source.sql +++ b/integration_tests/starrocks-sink/create_source.sql @@ -31,3 +31,36 @@ INSERT INTO upsert_user_behaviors VALUES (2,'2','2','2020-01-01T01:01:02Z','2','2','2'), (3,'3','3','2020-01-01T01:01:03Z','3','3','3'), (4,'4','4','2020-01-01T01:01:04Z','4','4','4'); + +CREATE TABLE starrocks_types ( + types_id INT, + c_boolean boolean, + c_smallint smallint, + c_integer integer, + c_bigint bigint, + c_decimal decimal, + c_real real, + c_double_precision double precision, + c_varchar varchar, + c_date date, + c_timestamp timestamp, + c_jsonb jsonb, + c_boolean_array boolean[], + c_smallint_array smallint[], + c_integer_array integer[], + c_bigint_array bigint[], + c_decimal_array decimal[], + c_real_array real[], + c_double_precision_array double precision[], + c_varchar_array varchar[], + c_date_array date[], + c_timestamp_array timestamp[], + c_jsonb_array jsonb[], + PRIMARY KEY (types_id) +); + +INSERT INTO starrocks_types VALUES (1, False, 0, 0, 0, 0, 0, 0, '', '0001-01-01', '0001-01-01 00:00:00.123456'::timestamp, '{}', array[]::boolean[], array[]::smallint[], array[]::integer[], array[]::bigint[], array[]::decimal[], array[]::real[], array[]::double precision[], array[]::varchar[], array[]::date[], array[]::timestamp[], array[]::jsonb[]); + +INSERT INTO starrocks_types VALUES (2, False, -32767, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'aa', '1970-01-01', '1970-01-01 00:00:00', '{}', array[False::boolean]::boolean[], array[-32767::smallint]::smallint[], array[-2147483647::integer]::integer[], array[-9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[-9999.999999::real]::real[], array[-10000.0::double precision]::double precision[], array['cc'::varchar]::varchar[], array['0001-01-01']::date[], array['0001-01-01 00:00:00']::timestamp[], array['{}'::jsonb]::jsonb[]); + +INSERT INTO starrocks_types VALUES (3, True, 32767, 2147483647, 9223372036854775807, -10.0, 9999.999999, 10000.0, 'zzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzz', '9999-12-31', '9999-12-31 23:59:59'::timestamp, '{"whatever":"meaningless"}', array[True::boolean]::boolean[], array[32767::smallint]::smallint[], array[2147483647::integer]::integer[], array[9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[9999.999999::real]::real[], array[10000.0::double precision]::double precision[], array['zzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzz'::varchar]::varchar[], array['9999-12-31'::date]::date[], array['9999-12-31 23:59:59'::timestamp::timestamp]::timestamp[], array['{"whatever":"meaningless"}'::jsonb]::jsonb[]); diff --git a/integration_tests/starrocks-sink/sink_check.py b/integration_tests/starrocks-sink/sink_check.py index bbe76e02a48d..0dba7084b21a 100644 --- a/integration_tests/starrocks-sink/sink_check.py +++ b/integration_tests/starrocks-sink/sink_check.py @@ -1,7 +1,7 @@ import subprocess import sys -relations = ['demo.demo_primary_table','demo.demo_duplicate_table','demo.demo_aggregate_table','demo.demo_unique_table', 'demo.upsert_table'] +relations = ['demo.demo_primary_table','demo.demo_duplicate_table','demo.demo_aggregate_table','demo.demo_unique_table', 'demo.upsert_table', 'demo.starrocks_types'] failed_cases = [] for rel in relations: diff --git a/integration_tests/starrocks-sink/starrocks_prepare.sql b/integration_tests/starrocks-sink/starrocks_prepare.sql index 7556b6db1dcf..c1d5e23238e3 100644 --- a/integration_tests/starrocks-sink/starrocks_prepare.sql +++ b/integration_tests/starrocks-sink/starrocks_prepare.sql @@ -41,5 +41,36 @@ CREATE table demo_unique_table( UNIQUE KEY(`user_id`) DISTRIBUTED BY HASH(`user_id`) properties("replication_num" = "1"); +CREATE table starrocks_types( + types_id int, + c_boolean boolean, + c_smallint smallint, + c_integer int, + c_bigint bigint, + c_decimal decimal, + c_real float, + c_double_precision double, + c_varchar varchar(65533), + c_date date, + c_timestamp datetime, + c_jsonb JSON, + c_boolean_array ARRAY, + c_smallint_array ARRAY, + c_integer_array ARRAY, + c_bigint_array ARRAY, + c_decimal_array ARRAY, + c_real_array ARRAY, + c_double_precision_array ARRAY, + c_varchar_array ARRAY, + c_date_array ARRAY, + c_timestamp_array ARRAY, + c_jsonb_array ARRAY +) ENGINE=OLAP +UNIQUE KEY(`types_id`) +DISTRIBUTED BY HASH(`types_id`) BUCKETS 1 +PROPERTIES ( + "replication_num" = "1" +); + CREATE USER 'users'@'%' IDENTIFIED BY '123456'; GRANT ALL ON *.* TO 'users'@'%'; From addc87d3aa5900efb1ae99b271e255fc5aac53b9 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 16 Apr 2024 12:44:42 +0800 Subject: [PATCH 25/64] fix(meta): fix potentially removing sst objects between min pinned and checkpoint version (#16260) --- src/meta/src/hummock/manager/checkpoint.rs | 70 ++++++++++--------- src/meta/src/hummock/manager/gc.rs | 30 ++++++-- src/meta/src/hummock/manager/mod.rs | 5 +- src/meta/src/hummock/manager/tests.rs | 26 +++++-- src/meta/src/manager/env.rs | 4 +- src/storage/backup/src/lib.rs | 6 +- .../compaction_group/hummock_version_ext.rs | 5 +- 7 files changed, 95 insertions(+), 51 deletions(-) diff --git a/src/meta/src/hummock/manager/checkpoint.rs b/src/meta/src/hummock/manager/checkpoint.rs index 8f3a7cdff695..3e162807262b 100644 --- a/src/meta/src/hummock/manager/checkpoint.rs +++ b/src/meta/src/hummock/manager/checkpoint.rs @@ -23,7 +23,9 @@ use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ }; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::HummockVersionId; -use risingwave_pb::hummock::hummock_version_checkpoint::{PbStaleObjects, StaleObjects}; +use risingwave_pb::hummock::hummock_version_checkpoint::{ + StaleObjects as PbStaleObjects, StaleObjects, +}; use risingwave_pb::hummock::{PbHummockVersionArchive, PbHummockVersionCheckpoint}; use thiserror_ext::AsReport; @@ -127,39 +129,38 @@ impl HummockManager { } let mut archive: Option = None; let mut stale_objects = old_checkpoint.stale_objects.clone(); - if !self.env.opts.enable_hummock_data_archive { - // `object_sizes` is used to calculate size of stale objects. - let mut object_sizes = object_size_map(&old_checkpoint.version); - for (_, version_delta) in versioning - .hummock_version_deltas - .range((Excluded(old_checkpoint_id), Included(new_checkpoint_id))) - { - for group_deltas in version_delta.group_deltas.values() { - let summary = summarize_group_deltas(group_deltas); - object_sizes.extend( - summary - .insert_table_infos - .iter() - .map(|t| (t.object_id, t.file_size)), - ); - } - let removed_object_ids = version_delta.gc_object_ids.clone(); - if removed_object_ids.is_empty() { - continue; - } - let total_file_size = removed_object_ids - .iter() - .map(|t| object_sizes.get(t).copied().unwrap()) - .sum::(); - stale_objects.insert( - version_delta.id, - StaleObjects { - id: removed_object_ids, - total_file_size, - }, + // `object_sizes` is used to calculate size of stale objects. + let mut object_sizes = object_size_map(&old_checkpoint.version); + for (_, version_delta) in versioning + .hummock_version_deltas + .range((Excluded(old_checkpoint_id), Included(new_checkpoint_id))) + { + for group_deltas in version_delta.group_deltas.values() { + let summary = summarize_group_deltas(group_deltas); + object_sizes.extend( + summary + .insert_table_infos + .iter() + .map(|t| (t.object_id, t.file_size)), ); } - } else { + let removed_object_ids = version_delta.gc_object_ids.clone(); + if removed_object_ids.is_empty() { + continue; + } + let total_file_size = removed_object_ids + .iter() + .map(|t| object_sizes.get(t).copied().unwrap()) + .sum::(); + stale_objects.insert( + version_delta.id, + StaleObjects { + id: removed_object_ids, + total_file_size, + }, + ); + } + if self.env.opts.enable_hummock_data_archive { archive = Some(PbHummockVersionArchive { version: Some(old_checkpoint.version.to_protobuf()), version_deltas: versioning @@ -190,7 +191,10 @@ impl HummockManager { let versioning = versioning_guard.deref_mut(); assert!(new_checkpoint.version.id >= versioning.checkpoint.version.id); versioning.checkpoint = new_checkpoint; - versioning.mark_objects_for_deletion(); + // Not delete stale objects when archive is enabled + if !self.env.opts.enable_hummock_data_archive { + versioning.mark_objects_for_deletion(); + } let min_pinned_version_id = versioning.min_pinned_version_id(); trigger_gc_stat(&self.metrics, &versioning.checkpoint, min_pinned_version_id); diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index 67d6fd404db6..ac10c58d78cf 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -14,6 +14,7 @@ use std::cmp; use std::collections::HashSet; +use std::ops::Bound::{Excluded, Included}; use std::ops::DerefMut; use std::time::Duration; @@ -28,6 +29,7 @@ use risingwave_pb::hummock::subscribe_compaction_event_response::Event as Respon use risingwave_pb::hummock::FullScanTask; use crate::hummock::error::{Error, Result}; +use crate::hummock::manager::versioning::Versioning; use crate::hummock::manager::{commit_multi_var, create_trx_wrapper, read_lock, write_lock}; use crate::hummock::HummockManager; use crate::manager::MetadataManager; @@ -117,20 +119,38 @@ impl HummockManager { ) -> usize { let tracked_object_ids: HashSet = { let versioning_guard = read_lock!(self, versioning).await; - let mut tracked_object_ids = - HashSet::from_iter(versioning_guard.current_version.get_object_ids()); - for delta in versioning_guard.hummock_version_deltas.values() { + let versioning: &Versioning = &versioning_guard; + + // object ids in current version + let mut tracked_object_ids = versioning.current_version.get_object_ids(); + // add object ids removed between checkpoint version and current version + for (_, delta) in versioning.hummock_version_deltas.range(( + Excluded(versioning.checkpoint.version.id), + Included(versioning.current_version.id), + )) { tracked_object_ids.extend(delta.gc_object_ids.iter().cloned()); } + // add stale object ids before the checkpoint version + let min_pinned_version_id = versioning.min_pinned_version_id(); + tracked_object_ids.extend( + versioning + .checkpoint + .stale_objects + .iter() + .filter(|(version_id, _)| **version_id >= min_pinned_version_id) + .flat_map(|(_, objects)| objects.id.iter()) + .cloned(), + ); tracked_object_ids }; let to_delete = object_ids .iter() - .filter(|object_id| !tracked_object_ids.contains(object_id)); + .filter(|object_id| !tracked_object_ids.contains(object_id)) + .collect_vec(); let mut versioning_guard = write_lock!(self, versioning).await; versioning_guard.objects_to_delete.extend(to_delete.clone()); drop(versioning_guard); - to_delete.count() + to_delete.len() } /// Starts a full GC. diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 55d51d1d0d4c..b3031e24a7ce 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -610,7 +610,10 @@ impl HummockManager { }; versioning_guard.objects_to_delete.clear(); - versioning_guard.mark_objects_for_deletion(); + // Not delete stale objects when archive is enabled + if !self.env.opts.enable_hummock_data_archive { + versioning_guard.mark_objects_for_deletion(); + } self.initial_compaction_group_config_after_load(versioning_guard) .await?; diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 6710262d8978..d3417f76f537 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -1108,6 +1108,7 @@ async fn test_hummock_compaction_task_heartbeat_removal_on_node_removal() { async fn test_extend_objects_to_delete() { let (_env, hummock_manager, _cluster_manager, worker_node) = setup_compute_env(80).await; let context_id = worker_node.id; + let _pinned_version1 = hummock_manager.pin_version(context_id).await.unwrap(); let sst_infos = add_test_tables(hummock_manager.as_ref(), context_id).await; let max_committed_object_id = sst_infos .iter() @@ -1120,7 +1121,7 @@ async fn test_extend_objects_to_delete() { .max() .unwrap(); let orphan_sst_num = 10; - let orphan_object_ids = sst_infos + let all_object_ids = sst_infos .iter() .flatten() .map(|s| s.get_object_id()) @@ -1129,7 +1130,7 @@ async fn test_extend_objects_to_delete() { assert!(hummock_manager.get_objects_to_delete().await.is_empty()); assert_eq!( hummock_manager - .extend_objects_to_delete_from_scan(&orphan_object_ids) + .extend_objects_to_delete_from_scan(&all_object_ids) .await, orphan_sst_num as usize ); @@ -1143,16 +1144,33 @@ async fn test_extend_objects_to_delete() { hummock_manager.create_version_checkpoint(1).await.unwrap(), 6 ); + assert_eq!( + hummock_manager.get_objects_to_delete().await.len(), + orphan_sst_num as usize + ); + // since version1 is still pinned, the sst removed in compaction can not be reclaimed. assert_eq!( hummock_manager - .extend_objects_to_delete_from_scan(&orphan_object_ids) + .extend_objects_to_delete_from_scan(&all_object_ids) .await, orphan_sst_num as usize ); + let objects_to_delete = hummock_manager.get_objects_to_delete().await; + assert_eq!(objects_to_delete.len(), orphan_sst_num as usize); + let pinned_version2 = hummock_manager.pin_version(context_id).await.unwrap(); + hummock_manager + .unpin_version_before(context_id, pinned_version2.id) + .await + .unwrap(); + // version1 is unpin, and then the sst removed in compaction can be reclaimed assert_eq!( - hummock_manager.get_objects_to_delete().await.len(), + hummock_manager + .extend_objects_to_delete_from_scan(&all_object_ids) + .await, orphan_sst_num as usize + 3 ); + let objects_to_delete = hummock_manager.get_objects_to_delete().await; + assert_eq!(objects_to_delete.len(), orphan_sst_num as usize + 3); } #[tokio::test] diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index f7c0d34d7a55..ca99f5de03de 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -41,7 +41,7 @@ use crate::manager::{ NotificationManagerRef, }; use crate::model::ClusterId; -use crate::storage::{MemStore, MetaStore, MetaStoreBoxExt, MetaStoreRef}; +use crate::storage::{MetaStore, MetaStoreRef}; use crate::MetaResult; #[derive(Clone)] @@ -522,6 +522,8 @@ impl MetaSrvEnv { } pub async fn for_test_opts(opts: MetaOpts) -> Self { + use crate::storage::{MemStore, MetaStoreBoxExt}; + Self::new( opts, risingwave_common::system_param::system_params_for_test(), diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index c5716e39b85d..a1acfde20e40 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -51,7 +51,7 @@ pub type MetaBackupJobId = u64; pub struct MetaSnapshotMetadata { pub id: MetaSnapshotId, pub hummock_version_id: HummockVersionId, - pub ssts: Vec, + pub ssts: HashSet, pub max_committed_epoch: u64, pub safe_epoch: u64, #[serde(default)] @@ -69,9 +69,7 @@ impl MetaSnapshotMetadata { Self { id, hummock_version_id: v.id, - ssts: HashSet::::from_iter(v.get_object_ids()) - .into_iter() - .collect_vec(), + ssts: v.get_object_ids(), max_committed_epoch: v.max_committed_epoch, safe_epoch: v.safe_epoch, format_version, 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 84228a92e395..560205a128b8 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 @@ -157,8 +157,7 @@ impl HummockVersion { }) } - /// This function does NOT dedup. - pub fn get_object_ids(&self) -> Vec { + pub fn get_object_ids(&self) -> HashSet { self.get_combined_levels() .flat_map(|level| { level @@ -166,7 +165,7 @@ impl HummockVersion { .iter() .map(|table_info| table_info.get_object_id()) }) - .collect_vec() + .collect() } pub fn level_iter bool>( From 2e90b648b91b3101b8ddd02edcda8f3ab66ea5c8 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 16 Apr 2024 13:34:05 +0800 Subject: [PATCH 26/64] fix(error): fix regression of error reporting for actor failures (#16285) Signed-off-by: Bugen Zhao --- src/meta/src/barrier/rpc.rs | 3 ++- src/stream/src/task/barrier_manager.rs | 34 ++++++++++++++++---------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index 3484d2ff6942..a098627afcd0 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -192,7 +192,8 @@ impl ControlStreamManager { .nodes .remove(&worker_id) .expect("should exist when get collect resp"); - warn!(node = ?node.worker, err = ?err.as_report(), "get error from response stream"); + // Note: No need to use `?` as the backtrace is from meta and not useful. + warn!(node = ?node.worker, err = %err.as_report(), "get error from response stream"); if let Some(command) = node.inflight_barriers.pop_front() { self.context.report_collect_failure(&command, &err); break Err(err); diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 5cd76eda6215..0f3a90ad81f4 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -25,13 +25,14 @@ use parking_lot::Mutex; use risingwave_pb::stream_service::barrier_complete_response::{ GroupedSstableInfo, PbCreateMviewProgress, }; +use risingwave_rpc_client::error::{ToTonicStatus, TonicStatusWrapper}; use rw_futures_util::{pending_on_none, AttachedFuture}; use thiserror_ext::AsReport; use tokio::select; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; use tokio::sync::oneshot; use tokio::task::JoinHandle; -use tonic::Status; +use tonic::{Code, Status}; use self::managed_state::ManagedBarrierState; use crate::error::{IntoUnexpectedExit, StreamError, StreamResult}; @@ -102,7 +103,11 @@ impl ControlStreamHandle { fn reset_stream_with_err(&mut self, err: Status) { if let Some((sender, _)) = self.pair.take() { - warn!("control stream reset with: {:?}", err.as_report()); + // Note: `TonicStatusWrapper` provides a better error report. + let err = TonicStatusWrapper::new(err); + warn!(error = %err.as_report(), "control stream reset with error"); + + let err = err.into_inner(); if sender.send(Err(err)).is_err() { warn!("failed to notify finish of control stream"); } @@ -111,7 +116,7 @@ impl ControlStreamHandle { fn inspect_result(&mut self, result: StreamResult<()>) { if let Err(e) = result { - self.reset_stream_with_err(Status::internal(format!("get error: {:?}", e.as_report()))); + self.reset_stream_with_err(e.to_status_unnamed(Code::Internal)); } } @@ -132,10 +137,11 @@ impl ControlStreamHandle { Some(Ok(request)) => { return request; } - Some(Err(e)) => self.reset_stream_with_err(Status::internal(format!( - "failed to get request: {:?}", - e.as_report() - ))), + Some(Err(e)) => self.reset_stream_with_err( + anyhow!(TonicStatusWrapper::new(e)) // wrap the status to provide better error report + .context("failed to get request") + .to_status_unnamed(Code::Internal), + ), None => self.reset_stream_with_err(Status::internal("end of stream")), } } @@ -691,12 +697,14 @@ impl LocalBarrierWorker { let root_err = self.try_find_root_failure(err).await; let failed_epochs = self.state.epochs_await_on_actor(actor_id).collect_vec(); if !failed_epochs.is_empty() { - self.control_stream_handle - .reset_stream_with_err(Status::internal(format!( - "failed to collect barrier. epoch: {:?}, err: {:?}", - failed_epochs, - root_err.as_report() - ))); + self.control_stream_handle.reset_stream_with_err( + anyhow!(root_err) + .context(format!( + "failed to collect barrier for epoch {:?}", + failed_epochs + )) + .to_status_unnamed(Code::Internal), + ); } } From 627715747263777b2ff6ce4a017891f175a49a1b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 16 Apr 2024 13:55:25 +0800 Subject: [PATCH 27/64] chore(deps): Bump either from 1.10.0 to 1.11.0 (#16313) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- src/rpc_client/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4e7ae1d81200..e3dc1c359ca3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3779,9 +3779,9 @@ dependencies = [ [[package]] name = "either" -version = "1.10.0" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11157ac094ffbdde99aa67b23417ebdd801842852b500e395a45a9c0aac03e4a" +checksum = "a47c1c47d2f5964e29c61246e81db715514cd532db6b5116a25ea3c03d6780a2" dependencies = [ "serde", ] diff --git a/src/rpc_client/Cargo.toml b/src/rpc_client/Cargo.toml index 6852fb11f717..c6e31e8eb0d7 100644 --- a/src/rpc_client/Cargo.toml +++ b/src/rpc_client/Cargo.toml @@ -17,7 +17,7 @@ normal = ["workspace-hack"] anyhow = "1" async-trait = "0.1" easy-ext = "1" -either = "1.10.0" +either = "1.11.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } http = "0.2" hyper = "0.14" # required by tonic From 1346a22c8afc06542746cf6fc9bd0940cc596d60 Mon Sep 17 00:00:00 2001 From: August Date: Tue, 16 Apr 2024 14:23:24 +0800 Subject: [PATCH 28/64] fix: make max_heartbeat_interval_secs default to 60s (#16283) --- src/common/src/config.rs | 2 +- src/config/ci-compaction-test-meta.toml | 2 +- src/config/ci-compaction-test.toml | 2 +- src/config/ci-iceberg-test.toml | 2 +- src/config/ci-mem.toml | 2 +- src/config/ci-recovery.toml | 2 +- src/config/ci.toml | 2 +- src/config/docs.md | 2 +- src/config/example.toml | 2 +- src/config/full-iceberg-bench.toml | 2 +- src/tests/simulation/tests/integration_tests/batch/mod.rs | 2 +- 11 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 0868c785eedc..e1e8ad8cfcd8 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -1115,7 +1115,7 @@ pub mod default { } pub fn max_heartbeat_interval_sec() -> u32 { - 300 + 60 } pub fn meta_leader_lease_secs() -> u64 { diff --git a/src/config/ci-compaction-test-meta.toml b/src/config/ci-compaction-test-meta.toml index af4b9d2dde75..d6f0dae62d66 100644 --- a/src/config/ci-compaction-test-meta.toml +++ b/src/config/ci-compaction-test-meta.toml @@ -1,6 +1,6 @@ [meta] enable_compaction_deterministic = true -max_heartbeat_interval_secs = 600 +max_heartbeat_interval_secs = 60 [streaming] in_flight_barrier_nums = 40 diff --git a/src/config/ci-compaction-test.toml b/src/config/ci-compaction-test.toml index f7e23bbd0d62..15754b90ca9b 100644 --- a/src/config/ci-compaction-test.toml +++ b/src/config/ci-compaction-test.toml @@ -1,6 +1,6 @@ [meta] enable_compaction_deterministic = true -max_heartbeat_interval_secs = 600 +max_heartbeat_interval_secs = 60 [streaming] in_flight_barrier_nums = 40 diff --git a/src/config/ci-iceberg-test.toml b/src/config/ci-iceberg-test.toml index 457707f486af..8033ad846ed1 100644 --- a/src/config/ci-iceberg-test.toml +++ b/src/config/ci-iceberg-test.toml @@ -1,6 +1,6 @@ [meta] disable_recovery = true -max_heartbeat_interval_secs = 600 +max_heartbeat_interval_secs = 60 [streaming] in_flight_barrier_nums = 10 diff --git a/src/config/ci-mem.toml b/src/config/ci-mem.toml index a441190bac65..433a7768061e 100644 --- a/src/config/ci-mem.toml +++ b/src/config/ci-mem.toml @@ -1,6 +1,6 @@ [meta] disable_recovery = true -max_heartbeat_interval_secs = 600 +max_heartbeat_interval_secs = 60 [streaming] in_flight_barrier_nums = 10 diff --git a/src/config/ci-recovery.toml b/src/config/ci-recovery.toml index 84aa0c8a8e7b..95e583462111 100644 --- a/src/config/ci-recovery.toml +++ b/src/config/ci-recovery.toml @@ -1,6 +1,6 @@ [meta] disable_recovery = false -max_heartbeat_interval_secs = 600 +max_heartbeat_interval_secs = 60 [streaming] in_flight_barrier_nums = 10 diff --git a/src/config/ci.toml b/src/config/ci.toml index db207ebf4441..02e1509546f2 100644 --- a/src/config/ci.toml +++ b/src/config/ci.toml @@ -1,6 +1,6 @@ [meta] disable_recovery = true -max_heartbeat_interval_secs = 600 +max_heartbeat_interval_secs = 60 [meta.compaction_config] level0_tier_compact_file_number = 6 diff --git a/src/config/docs.md b/src/config/docs.md index d925a428fb21..eacd8039a5d7 100644 --- a/src/config/docs.md +++ b/src/config/docs.md @@ -37,7 +37,7 @@ This page is automatically generated by `./risedev generate-example-config` | full_gc_interval_sec | Interval of automatic hummock full GC. | 86400 | | hummock_version_checkpoint_interval_sec | Interval of hummock version checkpoint. | 30 | | hybird_partition_vnode_count | | 4 | -| max_heartbeat_interval_secs | Maximum allowed heartbeat interval in seconds. | 300 | +| max_heartbeat_interval_secs | Maximum allowed heartbeat interval in seconds. | 60 | | meta_leader_lease_secs | | 30 | | min_delta_log_num_for_hummock_version_checkpoint | The minimum delta log number a new checkpoint should compact, otherwise the checkpoint attempt is rejected. | 10 | | min_sst_retention_time_sec | Objects within `min_sst_retention_time_sec` won't be deleted by hummock full GC, even they are dangling. | 86400 | diff --git a/src/config/example.toml b/src/config/example.toml index b8a1dcc27def..397bf9a89f69 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -23,7 +23,7 @@ vacuum_spin_interval_ms = 10 hummock_version_checkpoint_interval_sec = 30 enable_hummock_data_archive = false min_delta_log_num_for_hummock_version_checkpoint = 10 -max_heartbeat_interval_secs = 300 +max_heartbeat_interval_secs = 60 disable_recovery = false disable_automatic_parallelism_control = false parallelism_control_batch_size = 10 diff --git a/src/config/full-iceberg-bench.toml b/src/config/full-iceberg-bench.toml index 8c7cae6cd090..bfe731f23e37 100644 --- a/src/config/full-iceberg-bench.toml +++ b/src/config/full-iceberg-bench.toml @@ -19,7 +19,7 @@ vacuum_interval_sec = 30 vacuum_spin_interval_ms = 10 hummock_version_checkpoint_interval_sec = 30 min_delta_log_num_for_hummock_version_checkpoint = 10 -max_heartbeat_interval_secs = 300 +max_heartbeat_interval_secs = 60 disable_recovery = false meta_leader_lease_secs = 30 default_parallelism = "Full" diff --git a/src/tests/simulation/tests/integration_tests/batch/mod.rs b/src/tests/simulation/tests/integration_tests/batch/mod.rs index fb3658bef44d..ef530efa9f46 100644 --- a/src/tests/simulation/tests/integration_tests/batch/mod.rs +++ b/src/tests/simulation/tests/integration_tests/batch/mod.rs @@ -54,7 +54,7 @@ fn cluster_config_no_compute_nodes() -> Configuration { file.write_all( "\ [meta] -max_heartbeat_interval_secs = 300 +max_heartbeat_interval_secs = 60 [system] barrier_interval_ms = 1000 From cc95f487e8399e64663b49afd9c72f96262a628f Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Tue, 16 Apr 2024 15:05:55 +0800 Subject: [PATCH 29/64] chore: a clearer error message for file source when the list fails (#16294) --- src/stream/src/executor/source/list_executor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index d0ff13f70ee4..6d5fde8ee86b 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -90,7 +90,7 @@ impl FsListExecutor { let rows = chunk .into_iter() .map(|item| { - let page_item = item.unwrap(); + let page_item = item.expect("list file failed, please check whether the source connector is configured correctly."); ( Op::Insert, OwnedRow::new(vec![ From b5ec2ae7f66dc4203db40da3a7ea15b89d1d2fdf Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 16 Apr 2024 15:11:49 +0800 Subject: [PATCH 30/64] fix(postgres-cdc): disable certificate verification for `prefer` and `require` (#16319) --- src/connector/src/source/cdc/external/postgres.rs | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 7660546af14c..9df4b32e3f5e 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -20,7 +20,7 @@ use futures::stream::BoxStream; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use openssl::ssl::{SslConnector, SslMethod}; +use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use postgres_openssl::MakeTlsConnector; use risingwave_common::catalog::Schema; use risingwave_common::row::{OwnedRow, Row}; @@ -146,14 +146,20 @@ impl PostgresExternalTableReader { let connector = match config.sslmode { SslMode::Disable => MaybeMakeTlsConnector::NoTls(NoTls), SslMode::Prefer => match SslConnector::builder(SslMethod::tls()) { - Ok(builder) => MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())), + Ok(mut builder) => { + // disable certificate verification for `prefer` + builder.set_verify(SslVerifyMode::NONE); + MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) + } Err(e) => { tracing::warn!(error = %e.as_report(), "SSL connector error"); MaybeMakeTlsConnector::NoTls(NoTls) } }, SslMode::Require => { - let builder = SslConnector::builder(SslMethod::tls())?; + let mut builder = SslConnector::builder(SslMethod::tls())?; + // disable certificate verification for `require` + builder.set_verify(SslVerifyMode::NONE); MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) } }; From 2f560ca5e6ef7393ead8e74fba6dc3d5ed56053f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 16 Apr 2024 15:22:46 +0800 Subject: [PATCH 31/64] chore(deps): Bump the aws group with 2 updates (#16312) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e3dc1c359ca3..aeee200fe918 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1004,9 +1004,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.1.2" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a7cb3510b95492bd9014b60e2e3bee3e48bc516e220316f8e6b60df18b47331" +checksum = "d67c6836a1009b23e3f4cd1457c83e0aa49a490d9c3033b53c3f7b8cf2facc0f" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -1349,9 +1349,9 @@ dependencies = [ [[package]] name = "aws-types" -version = "1.1.2" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8549aa62c5b7db5c57ab915200ee214b4f5d8f19b29a4a8fa0b3ad3bca1380e3" +checksum = "ee2739d97d47f47cdf0d27982019a405dcc736df25925d1a75049f1faa79df88" dependencies = [ "aws-credential-types", "aws-smithy-async", @@ -8448,7 +8448,7 @@ dependencies = [ "indoc", "libc", "memoffset", - "parking_lot 0.12.1", + "parking_lot 0.11.2", "portable-atomic", "pyo3-build-config", "pyo3-ffi", From 18de4d12292638e5ada267888b3d45524f142d8f Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 16 Apr 2024 15:54:11 +0800 Subject: [PATCH 32/64] feat(streaming): support alter rate limit for arrangement backfill (#16325) --- .../executor/backfill/arrangement_backfill.rs | 77 +++++++++++++------ .../executor/backfill/no_shuffle_backfill.rs | 4 +- 2 files changed, 56 insertions(+), 25 deletions(-) diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 2fea4b313c88..b4adf038e5b7 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -117,6 +117,7 @@ where let upstream_table_id = self.upstream_table.table_id(); let mut upstream_table = self.upstream_table; let vnodes = upstream_table.vnodes().clone(); + let mut rate_limit = self.rate_limit; // These builders will build data chunks. // We must supply them with the full datatypes which correspond to @@ -132,11 +133,8 @@ where .vnodes() .iter_vnodes() .map(|vnode| { - let builder = create_builder( - self.rate_limit, - self.chunk_size, - snapshot_data_types.clone(), - ); + let builder = + create_builder(rate_limit, self.chunk_size, snapshot_data_types.clone()); (vnode, builder) }) .collect(); @@ -215,8 +213,7 @@ where let mut upstream_chunk_buffer: Vec = vec![]; let mut pending_barrier: Option = None; - let rate_limiter = self.rate_limit.and_then(create_limiter); - let rate_limit = self.rate_limit; + let mut rate_limiter = rate_limit.and_then(create_limiter); let backfill_snapshot_read_row_count_metric = self .metrics @@ -401,25 +398,11 @@ where }; // Process barrier: - // - handle mutations // - consume snapshot rows left in builder. // - consume upstream buffer chunk + // - handle mutations // - switch snapshot - // handle mutations - if let Some(mutation) = barrier.mutation.as_deref() { - use crate::executor::Mutation; - match mutation { - Mutation::Pause => { - paused = true; - } - Mutation::Resume => { - paused = false; - } - _ => (), - } - } - // consume snapshot rows left in builder. // NOTE(kwannoel): `zip_eq_debug` does not work here, // we encounter "higher-ranked lifetime error". @@ -507,6 +490,54 @@ where "barrier persisted" ); + // handle mutations + if let Some(mutation) = barrier.mutation.as_deref() { + use crate::executor::Mutation; + match mutation { + Mutation::Pause => { + paused = true; + } + Mutation::Resume => { + paused = false; + } + Mutation::Throttle(actor_to_apply) => { + let new_rate_limit_entry = actor_to_apply.get(&self.actor_id); + if let Some(new_rate_limit) = new_rate_limit_entry { + let new_rate_limit = new_rate_limit.as_ref().map(|x| *x as _); + if new_rate_limit != rate_limit { + rate_limit = new_rate_limit; + tracing::info!( + id = self.actor_id, + new_rate_limit = ?rate_limit, + "actor rate limit changed", + ); + // The builder is emptied above via `DataChunkBuilder::consume_all`. + for (_, builder) in builders { + assert!( + builder.is_empty(), + "builder should already be emptied" + ); + } + builders = upstream_table + .vnodes() + .iter_vnodes() + .map(|vnode| { + let builder = create_builder( + rate_limit, + self.chunk_size, + snapshot_data_types.clone(), + ); + (vnode, builder) + }) + .collect(); + rate_limiter = new_rate_limit.and_then(create_limiter); + } + } + } + _ => {} + } + } + yield Message::Barrier(barrier); // We will switch snapshot at the start of the next iteration of the backfill loop. @@ -601,7 +632,7 @@ where if paused { #[for_await] for _ in tokio_stream::pending() { - yield None; + bail!("BUG: paused stream should not yield"); } } else { // Checked the rate limit is not zero. diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index b409e183f5a9..ca14f904e2dd 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -477,7 +477,7 @@ where rate_limit = new_rate_limit; tracing::info!( id = self.actor_id, - new_rate_limit = ?self.rate_limit, + new_rate_limit = ?rate_limit, "actor rate limit changed", ); // The builder is emptied above via `DataChunkBuilder::consume_all`. @@ -656,7 +656,7 @@ where if paused { #[for_await] for _ in tokio_stream::pending() { - yield None; + bail!("BUG: paused stream should not yield"); } } else { // Checked the rate limit is not zero. From dba4fab66d1403509c711e34cad530c7db63209d Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Tue, 16 Apr 2024 16:51:52 +0800 Subject: [PATCH 33/64] fix: stale metrics of meta leader election (#16334) --- src/meta/src/rpc/metrics.rs | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index cfe3c1f88077..2df847516c4a 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -47,17 +47,17 @@ use crate::rpc::ElectionClientRef; #[derive(Clone)] pub struct MetaMetrics { - /// ********************************** Meta ************************************ + // ********************************** Meta ************************************ /// The number of workers in the cluster. pub worker_num: IntGaugeVec, /// The roles of all meta nodes in the cluster. pub meta_type: IntGaugeVec, - /// ********************************** gRPC ************************************ + // ********************************** gRPC ************************************ /// gRPC latency of meta services pub grpc_latency: HistogramVec, - /// ********************************** Barrier ************************************ + // ********************************** Barrier ************************************ /// The duration from barrier injection to commit /// It is the sum of inflight-latency, sync-latency and wait-commit-latency pub barrier_latency: Histogram, @@ -73,11 +73,11 @@ pub struct MetaMetrics { /// The timestamp (UNIX epoch seconds) of the last committed barrier's epoch time. pub last_committed_barrier_time: IntGauge, - /// ********************************** Recovery ************************************ + // ********************************** Recovery ************************************ pub recovery_failure_cnt: IntCounter, pub recovery_latency: Histogram, - /// ********************************** Hummock ************************************ + // ********************************** Hummock ************************************ /// Max committed epoch pub max_committed_epoch: IntGauge, /// The smallest epoch that has not been `GCed`. @@ -160,16 +160,16 @@ pub struct MetaMetrics { pub compaction_event_consumed_latency: Histogram, pub compaction_event_loop_iteration_latency: Histogram, - /// ********************************** Object Store ************************************ + // ********************************** Object Store ************************************ // Object store related metrics (for backup/restore and version checkpoint) pub object_store_metric: Arc, - /// ********************************** Source ************************************ + // ********************************** Source ************************************ /// supervisor for which source is still up. pub source_is_up: LabelGuardedIntGaugeVec<2>, pub source_enumerator_metrics: Arc, - /// ********************************** Fragment ************************************ + // ********************************** Fragment ************************************ /// A dummpy gauge metrics with its label to be the mapping from actor id to fragment id pub actor_info: IntGaugeVec, /// A dummpy gauge metrics with its label to be the mapping from table id to actor id @@ -744,6 +744,10 @@ pub fn start_worker_info_monitor( } }; + // Reset metrics to clean the stale labels e.g. invalid lease ids + meta_metrics.worker_num.reset(); + meta_metrics.meta_type.reset(); + for (worker_type, worker_num) in node_map { meta_metrics .worker_num From 86ac793d95023614c9b5e441b66a4753f924082c Mon Sep 17 00:00:00 2001 From: xfz <73645462+xuefengze@users.noreply.github.com> Date: Tue, 16 Apr 2024 17:44:15 +0800 Subject: [PATCH 34/64] fix: use interval parse instead of from_iso_8601 (#16252) --- src/connector/src/parser/json_parser.rs | 13 +++++++++++-- src/connector/src/parser/unified/json.rs | 4 +--- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 454e8f1d0dd5..05e689f07be7 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -222,8 +222,8 @@ mod tests { fn get_payload() -> Vec> { vec![ - br#"{"i32":1,"bool":true,"i16":1,"i64":12345678,"f32":1.23,"f64":1.2345,"varchar":"varchar","date":"2021-01-01","timestamp":"2021-01-01 16:06:12.269","decimal":12345.67890}"#.to_vec(), - br#"{"i32":1,"f32":12345e+10,"f64":12345,"decimal":12345}"#.to_vec(), + br#"{"i32":1,"bool":true,"i16":1,"i64":12345678,"f32":1.23,"f64":1.2345,"varchar":"varchar","date":"2021-01-01","timestamp":"2021-01-01 16:06:12.269","decimal":12345.67890,"interval":"P1Y2M3DT0H5M0S"}"#.to_vec(), + br#"{"i32":1,"f32":12345e+10,"f64":12345,"decimal":12345,"interval":"1 day"}"#.to_vec(), ] } @@ -245,6 +245,7 @@ mod tests { SourceColumnDesc::simple("date", DataType::Date, 8.into()), SourceColumnDesc::simple("timestamp", DataType::Timestamp, 9.into()), SourceColumnDesc::simple("decimal", DataType::Decimal, 10.into()), + SourceColumnDesc::simple("interval", DataType::Interval, 11.into()), ]; let parser = JsonParser::new( @@ -307,6 +308,10 @@ mod tests { row.datum_at(9).to_owned_datum(), (Some(ScalarImpl::Decimal("12345.67890".parse().unwrap()))) ); + assert_eq!( + row.datum_at(10).to_owned_datum(), + (Some(ScalarImpl::Interval("P1Y2M3DT0H5M0S".parse().unwrap()))) + ); } { @@ -329,6 +334,10 @@ mod tests { row.datum_at(9).to_owned_datum(), (Some(ScalarImpl::Decimal(12345.into()))) ); + assert_eq!( + row.datum_at(10).to_owned_datum(), + (Some(ScalarImpl::Interval("1 day".parse().unwrap()))) + ); } } diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index ae3da8de4dc3..d9deb73d582d 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -513,9 +513,7 @@ impl JsonParseOptions { .into(), // ---- Interval ----- (Some(DataType::Interval), ValueType::String) => { - Interval::from_iso_8601(value.as_str().unwrap()) - .map_err(|_| create_error())? - .into() + value.as_str().unwrap().parse::().map_err(|_| create_error())?.into() } // ---- Struct ----- (Some(DataType::Struct(struct_type_info)), ValueType::Object) => StructValue::new( From 6a966e3b235f791d6f399e58dbe86d289cd06bc9 Mon Sep 17 00:00:00 2001 From: August Date: Tue, 16 Apr 2024 19:01:18 +0800 Subject: [PATCH 35/64] fix(sql-backend): reset worker auto increment id offset for worker table (#16342) --- src/ctl/src/cmd_impl/meta/migration.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index 66fde163eff5..8a77ca5650f9 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -145,6 +145,11 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an // workers. let workers = model::Worker::list(&meta_store).await?; + let next_worker_id = workers + .iter() + .map(|w| w.worker_node.id + 1) + .max() + .unwrap_or(1); for worker in workers { Worker::insert(worker::ActiveModel::from(&worker.worker_node)) .exec(&meta_store_sql.conn) @@ -795,6 +800,13 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an // Rest sequence for object and user. match meta_store_sql.conn.get_database_backend() { DbBackend::MySql => { + meta_store_sql + .conn + .execute(Statement::from_string( + DatabaseBackend::MySql, + format!("ALTER TABLE worker AUTO_INCREMENT = {next_worker_id};"), + )) + .await?; let next_object_id = next_available_id(); meta_store_sql .conn @@ -812,6 +824,13 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an .await?; } DbBackend::Postgres => { + meta_store_sql + .conn + .execute(Statement::from_string( + DatabaseBackend::Postgres, + "SELECT setval('worker_worker_id_seq', (SELECT MAX(worker_id) FROM worker));", + )) + .await?; meta_store_sql .conn .execute(Statement::from_string( From 2bcf892f5b03de291b92f75b2554996f03b298eb Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 16 Apr 2024 19:54:46 +0800 Subject: [PATCH 36/64] ci: fix s3 source test (#16315) Signed-off-by: xxchan --- ci/scripts/notify.py | 2 +- ci/scripts/s3-source-test.sh | 11 +++++++++-- ci/workflows/main-cron.yml | 8 ++++---- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py index 2c86c7697f6d..0ebd01f8621e 100755 --- a/ci/scripts/notify.py +++ b/ci/scripts/notify.py @@ -149,7 +149,7 @@ def format_cmd(messages): cmd=f""" cat <<- YAML | buildkite-agent pipeline upload steps: - - label: "Test" + - label: "trigger failed test notification" command: echo "running failed test notification" && exit 1 notify: - slack: diff --git a/ci/scripts/s3-source-test.sh b/ci/scripts/s3-source-test.sh index 62a58ab3242f..9bcb0d830cb4 100755 --- a/ci/scripts/s3-source-test.sh +++ b/ci/scripts/s3-source-test.sh @@ -4,7 +4,7 @@ set -euo pipefail source ci/scripts/common.sh -while getopts 'p:s:' opt; do +while getopts 'p:s:t:' opt; do case ${opt} in p ) profile=$OPTARG @@ -12,6 +12,9 @@ while getopts 'p:s:' opt; do s ) script=$OPTARG ;; + t ) + format_type=$OPTARG + ;; \? ) echo "Invalid Option: -$OPTARG" 1>&2 exit 1 @@ -30,7 +33,11 @@ risedev ci-start ci-1cn-1fe echo "--- Run test" python3 -m pip install minio psycopg2-binary opendal -python3 e2e_test/s3/"$script" +if [[ -v format_type ]]; then + python3 e2e_test/s3/"$script" "$format_type" +else + python3 e2e_test/s3/"$script" +fi echo "--- Kill cluster" risedev ci-kill diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 586d56e8fb64..7cb23f76ce25 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -540,7 +540,7 @@ steps: - label: "S3_v2 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 json'" + command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_v2.py -t json" if: | !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" @@ -562,7 +562,7 @@ steps: - label: "S3_v2 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 json'" + command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_batch.py -t json" if: | !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" @@ -584,7 +584,7 @@ steps: - label: "S3_v2 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 csv_without_header'" + command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_v2.py -t csv_without_header" if: | !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" @@ -606,7 +606,7 @@ steps: - label: "PosixFs source on OpenDAL fs engine (csv parser)" key: "s3-source-test-for-opendal-fs-engine-csv-parser" - command: "ci/scripts/s3-source-test.sh -p ci-release -s 'posix_fs_source.py csv_without_header'" + command: "ci/scripts/s3-source-test.sh -p ci-release -s posix_fs_source.py -t csv_without_header" if: | !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" From 546773a7fd80ce1bcbd48e769fa6150dcd9b3451 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Tue, 16 Apr 2024 19:58:21 +0800 Subject: [PATCH 37/64] feat(catalog): support column `indnkeyatts` in `pg_index` (#16339) --- e2e_test/batch/catalog/pg_attribute.slt.part | 4 ++- e2e_test/batch/catalog/pg_index.slt.part | 12 ++++----- .../tests/testdata/output/subquery.yaml | 2 +- .../system_catalog/pg_catalog/pg_index.rs | 26 ++++++++++++++++--- .../system_catalog/rw_catalog/rw_indexes.rs | 18 +++++++++++-- 5 files changed, 49 insertions(+), 13 deletions(-) diff --git a/e2e_test/batch/catalog/pg_attribute.slt.part b/e2e_test/batch/catalog/pg_attribute.slt.part index 7f96653af83a..4c8570ce2f47 100644 --- a/e2e_test/batch/catalog/pg_attribute.slt.part +++ b/e2e_test/batch/catalog/pg_attribute.slt.part @@ -38,7 +38,9 @@ select i.relname, a.attname, ix.indkey from pg_catalog.pg_class t join pg_catalog.pg_attribute a on t.oid = a.attrelid and a.attnum = ANY(ix.indkey) where t.relname = 'tmp' order by a.attnum; ---- -tmp_idx id2 {2} +tmp_idx id1 {2,1,3,5} +tmp_idx id2 {2,1,3,5} +tmp_idx id3 {2,1,3,5} statement ok drop table tmp; diff --git a/e2e_test/batch/catalog/pg_index.slt.part b/e2e_test/batch/catalog/pg_index.slt.part index c42e74d60ab4..bbcf1d7a48b6 100644 --- a/e2e_test/batch/catalog/pg_index.slt.part +++ b/e2e_test/batch/catalog/pg_index.slt.part @@ -5,34 +5,34 @@ statement ok create index tmp_id2_idx on tmp(id2) include(id2); query IT -select ix.indnatts, ix.indkey from pg_catalog.pg_class t +select ix.indnatts, ix.indnkeyatts, ix.indkey from pg_catalog.pg_class t join pg_catalog.pg_index ix on t.oid = ix.indrelid join pg_catalog.pg_class i on i.oid = ix.indexrelid where t.relname = 'tmp' and i.relname = 'tmp_id2_idx'; ---- -1 {2} +2 1 {2,3} statement ok create index tmp_id2_idx_include_id1 on tmp(id2) include(id1); query IT -select ix.indnatts, ix.indkey from pg_catalog.pg_class t +select ix.indnatts, ix.indnkeyatts, ix.indkey from pg_catalog.pg_class t join pg_catalog.pg_index ix on t.oid = ix.indrelid join pg_catalog.pg_class i on i.oid = ix.indexrelid where t.relname = 'tmp' and i.relname = 'tmp_id2_idx_include_id1'; ---- -1 {2} +3 1 {2,1,3} statement ok create index tmp_id1_id2_idx on tmp(id1, id2); query IT -select ix.indnatts, ix.indkey from pg_catalog.pg_class t +select ix.indnatts, ix.indnkeyatts, ix.indkey from pg_catalog.pg_class t join pg_catalog.pg_index ix on t.oid = ix.indrelid join pg_catalog.pg_class i on i.oid = ix.indexrelid where t.relname = 'tmp' and i.relname = 'tmp_id1_id2_idx'; ---- -2 {1,2} +3 2 {1,2,3} statement ok drop table tmp; diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 2d21de5743ab..451ce0b70a7b 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -243,7 +243,7 @@ │ │ │ │ │ │ └─LogicalProject { exprs: [rw_sources.id, rw_sources.name, 'source':Varchar, rw_sources.schema_id, rw_sources.owner, rw_sources.definition, rw_sources.acl] } │ │ │ │ │ │ └─LogicalSysScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at, rw_sources.initialized_at_cluster_version, rw_sources.created_at_cluster_version] } │ │ │ │ │ └─LogicalProject { exprs: [rw_indexes.id, rw_indexes.name, 'index':Varchar, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl] } - │ │ │ │ │ └─LogicalSysScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.indkey, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at, rw_indexes.initialized_at_cluster_version, rw_indexes.created_at_cluster_version] } + │ │ │ │ │ └─LogicalSysScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.key_columns, rw_indexes.include_columns, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at, rw_indexes.initialized_at_cluster_version, rw_indexes.created_at_cluster_version] } │ │ │ │ └─LogicalProject { exprs: [rw_sinks.id, rw_sinks.name, 'sink':Varchar, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.definition, rw_sinks.acl] } │ │ │ │ └─LogicalSysScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.connector, rw_sinks.sink_type, rw_sinks.connection_id, rw_sinks.definition, rw_sinks.acl, rw_sinks.initialized_at, rw_sinks.created_at, rw_sinks.initialized_at_cluster_version, rw_sinks.created_at_cluster_version] } │ │ │ └─LogicalProject { exprs: [rw_subscriptions.id, rw_subscriptions.name, 'subscription':Varchar, rw_subscriptions.schema_id, rw_subscriptions.owner, rw_subscriptions.definition, rw_subscriptions.acl] } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs index 2dfb15f9e527..b91bd9b698cb 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs @@ -22,14 +22,23 @@ use risingwave_frontend_macro::system_catalog; "pg_catalog.pg_index", "SELECT id AS indexrelid, primary_table_id AS indrelid, - ARRAY_LENGTH(indkey)::smallint AS indnatts, + ARRAY_LENGTH(key_columns || include_columns)::smallint AS indnatts, + ARRAY_LENGTH(key_columns)::smallint AS indnkeyatts, false AS indisunique, - indkey, + key_columns || include_columns AS indkey, ARRAY[]::smallint[] as indoption, NULL AS indexprs, NULL AS indpred, FALSE AS indisprimary, - ARRAY[]::int[] AS indclass + ARRAY[]::int[] AS indclass, + false AS indisexclusion, + true AS indimmediate, + false AS indisclustered, + true AS indisvalid, + false AS indcheckxmin, + true AS indisready, + true AS indislive, + false AS indisreplident FROM rw_catalog.rw_indexes" )] #[derive(Fields)] @@ -37,6 +46,7 @@ struct PgIndex { indexrelid: i32, indrelid: i32, indnatts: i16, + indnkeyatts: i16, // We return false as default to indicate that this is NOT a unique index indisunique: bool, indkey: Vec, @@ -49,4 +59,14 @@ struct PgIndex { indisprimary: bool, // Empty array. We only have a dummy implementation of `pg_opclass` yet. indclass: Vec, + + // Unused columns. Kept for compatibility with PG. + indisexclusion: bool, + indimmediate: bool, + indisclustered: bool, + indisvalid: bool, + indcheckxmin: bool, + indisready: bool, + indislive: bool, + indisreplident: bool, } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs index a38321942158..558e628a3fbf 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs @@ -24,7 +24,8 @@ struct RwIndex { id: i32, name: String, primary_table_id: i32, - indkey: Vec, + key_columns: Vec, + include_columns: Vec, schema_id: i32, owner: i32, definition: String, @@ -46,7 +47,7 @@ fn read_rw_indexes(reader: &SysCatalogReaderImpl) -> Result> { id: index.id.index_id as i32, name: index.name.clone(), primary_table_id: index.primary_table.id().table_id as i32, - indkey: index + key_columns: index .index_item .iter() .take(index.index_columns_len as usize) @@ -59,6 +60,19 @@ fn read_rw_indexes(reader: &SysCatalogReaderImpl) -> Result> { ind as i16 }) .collect(), + include_columns: index + .index_item + .iter() + .skip(index.index_columns_len as usize) + .map(|index| { + let ind = if let Some(input_ref) = index.as_input_ref() { + input_ref.index() + 1 + } else { + 0 + }; + ind as i16 + }) + .collect(), schema_id: schema.id() as i32, owner: index.index_table.owner as i32, definition: index.index_table.create_sql(), From 4ba80c90f568425341e518eea33fc7a374283045 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Tue, 16 Apr 2024 20:26:11 +0800 Subject: [PATCH 38/64] feat(expr): add `jsonb_populate_record(set)` function (#13421) Signed-off-by: Runji Wang --- proto/expr.proto | 4 + src/common/src/types/jsonb.rs | 108 ++++++++++++- src/expr/impl/src/scalar/cast.rs | 3 +- src/expr/impl/src/scalar/jsonb_record.rs | 145 ++++++++++++++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/expr/macro/src/gen.rs | 86 +++++++---- src/frontend/src/binder/expr/function.rs | 1 + src/frontend/src/expr/pure.rs | 2 + .../src/optimizer/plan_expr_visitor/strong.rs | 2 + 9 files changed, 319 insertions(+), 33 deletions(-) create mode 100644 src/expr/impl/src/scalar/jsonb_record.rs diff --git a/proto/expr.proto b/proto/expr.proto index 802397f0456f..4c55ee2b614b 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -274,6 +274,8 @@ message ExprNode { JSONB_PATH_MATCH = 621; JSONB_PATH_QUERY_ARRAY = 622; JSONB_PATH_QUERY_FIRST = 623; + JSONB_POPULATE_RECORD = 629; + JSONB_TO_RECORD = 630; // Non-pure functions below (> 1000) // ------------------------ @@ -328,6 +330,8 @@ message TableFunction { JSONB_EACH_TEXT = 13; JSONB_OBJECT_KEYS = 14; JSONB_PATH_QUERY = 15; + JSONB_POPULATE_RECORDSET = 16; + JSONB_TO_RECORDSET = 17; // User defined table function UDTF = 100; } diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 6b48893ac9ed..522ec788d864 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -19,7 +19,9 @@ use bytes::Buf; use jsonbb::{Value, ValueRef}; use risingwave_common_estimate_size::EstimateSize; -use crate::types::{Scalar, ScalarRef}; +use super::{Datum, IntoOrdered, ListValue, ScalarImpl, StructRef, ToOwnedDatum, F64}; +use crate::types::{DataType, Scalar, ScalarRef, StructType, StructValue}; +use crate::util::iter_util::ZipEqDebug; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct JsonbVal(pub(crate) Value); @@ -297,11 +299,12 @@ impl<'a> JsonbRef<'a> { /// /// According to RFC 8259, only number within IEEE 754 binary64 (double precision) has good /// interoperability. We do not support arbitrary precision like PostgreSQL `numeric` right now. - pub fn as_number(&self) -> Result { + pub fn as_number(&self) -> Result { self.0 .as_number() .ok_or_else(|| format!("cannot cast jsonb {} to type number", self.type_name()))? .as_f64() + .map(|f| f.into_ordered()) .ok_or_else(|| "jsonb number out of range".into()) } @@ -380,6 +383,107 @@ impl<'a> JsonbRef<'a> { self.0.serialize(&mut ser).map_err(|_| std::fmt::Error) } + /// Convert the jsonb value to a datum. + pub fn to_datum(self, ty: &DataType) -> Result { + if !matches!( + ty, + DataType::Jsonb + | DataType::Boolean + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Varchar + | DataType::List(_) + | DataType::Struct(_) + ) { + return Err(format!("cannot cast jsonb to {ty}")); + } + if self.0.as_null().is_some() { + return Ok(None); + } + Ok(Some(match ty { + DataType::Jsonb => ScalarImpl::Jsonb(self.into()), + DataType::Boolean => ScalarImpl::Bool(self.as_bool()?), + DataType::Int16 => ScalarImpl::Int16(self.as_number()?.try_into()?), + DataType::Int32 => ScalarImpl::Int32(self.as_number()?.try_into()?), + DataType::Int64 => ScalarImpl::Int64(self.as_number()?.try_into()?), + DataType::Float32 => ScalarImpl::Float32(self.as_number()?.try_into()?), + DataType::Float64 => ScalarImpl::Float64(self.as_number()?), + DataType::Varchar => ScalarImpl::Utf8(self.force_string().into()), + DataType::List(t) => ScalarImpl::List(self.to_list(t)?), + DataType::Struct(s) => ScalarImpl::Struct(self.to_struct(s)?), + _ => unreachable!(), + })) + } + + /// Convert the jsonb value to a list value. + pub fn to_list(self, elem_type: &DataType) -> Result { + let array = self + .0 + .as_array() + .ok_or_else(|| format!("expected JSON array, but found {self}"))?; + let mut builder = elem_type.create_array_builder(array.len()); + for v in array.iter() { + builder.append(Self(v).to_datum(elem_type)?); + } + Ok(ListValue::new(builder.finish())) + } + + /// Convert the jsonb value to a struct value. + pub fn to_struct(self, ty: &StructType) -> Result { + let object = self.0.as_object().ok_or_else(|| { + format!( + "cannot call populate_composite on a jsonb {}", + self.type_name() + ) + })?; + let mut fields = Vec::with_capacity(ty.len()); + for (name, ty) in ty.iter() { + let datum = match object.get(name) { + Some(v) => Self(v).to_datum(ty)?, + None => None, + }; + fields.push(datum); + } + Ok(StructValue::new(fields)) + } + + /// Expands the top-level JSON object to a row having the struct type of the `base` argument. + pub fn populate_struct( + self, + ty: &StructType, + base: Option>, + ) -> Result { + let Some(base) = base else { + return self.to_struct(ty); + }; + let object = self.0.as_object().ok_or_else(|| { + format!( + "cannot call populate_composite on a jsonb {}", + self.type_name() + ) + })?; + let mut fields = Vec::with_capacity(ty.len()); + for ((name, ty), base_field) in ty.iter().zip_eq_debug(base.iter_fields_ref()) { + let datum = match object.get(name) { + Some(v) => match ty { + // recursively populate the nested struct + DataType::Struct(s) => Some( + Self(v) + .populate_struct(s, base_field.map(|s| s.into_struct()))? + .into(), + ), + _ => Self(v).to_datum(ty)?, + }, + None => base_field.to_owned_datum(), + }; + fields.push(datum); + } + Ok(StructValue::new(fields)) + } + /// Returns the capacity of the underlying buffer. pub fn capacity(self) -> usize { self.0.capacity() diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index bff225ad687f..cb990bdcec6a 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -21,7 +21,7 @@ use itertools::Itertools; use risingwave_common::array::{ArrayImpl, DataChunk, ListRef, ListValue, StructRef, StructValue}; use risingwave_common::cast; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{Int256, IntoOrdered, JsonbRef, ToText, F64}; +use risingwave_common::types::{Int256, JsonbRef, ToText, F64}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::{build_func, Context, ExpressionBoxExt, InputRefExpression}; use risingwave_expr::{function, ExprError, Result}; @@ -79,7 +79,6 @@ pub fn jsonb_to_bool(v: JsonbRef<'_>) -> Result { pub fn jsonb_to_number>(v: JsonbRef<'_>) -> Result { v.as_number() .map_err(|e| ExprError::Parse(e.into()))? - .into_ordered() .try_into() .map_err(|_| ExprError::NumericOutOfRange) } diff --git a/src/expr/impl/src/scalar/jsonb_record.rs b/src/expr/impl/src/scalar/jsonb_record.rs new file mode 100644 index 000000000000..fcc9606897ab --- /dev/null +++ b/src/expr/impl/src/scalar/jsonb_record.rs @@ -0,0 +1,145 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_common::types::{JsonbRef, StructRef, StructValue}; +use risingwave_expr::expr::Context; +use risingwave_expr::{function, ExprError, Result}; + +/// Expands the top-level JSON object to a row having the composite type of the base argument. +/// The JSON object is scanned for fields whose names match column names of the output row type, +/// and their values are inserted into those columns of the output. (Fields that do not correspond +/// to any output column name are ignored.) In typical use, the value of base is just NULL, which +/// means that any output columns that do not match any object field will be filled with nulls. +/// However, if base isn't NULL then the values it contains will be used for unmatched columns. +/// +/// # Examples +/// +/// ```slt +/// query ITT +/// select (jsonb_populate_record( +/// null::struct>, +/// '{"a": 1, "b": ["2", "a b"], "c": {"d": 4, "e": "a b c"}, "x": "foo"}' +/// )).*; +/// ---- +/// 1 {2,"a b"} (4,"a b c") +/// +/// query ITT +/// select (jsonb_populate_record( +/// row(1, null, row(4, '5'))::struct>, +/// '{"b": ["2", "a b"], "c": {"e": "a b c"}, "x": "foo"}' +/// )).*; +/// ---- +/// 1 {2,"a b"} (4,"a b c") +/// ``` +#[function("jsonb_populate_record(struct, jsonb) -> struct")] +fn jsonb_populate_record( + base: Option>, + jsonb: JsonbRef<'_>, + ctx: &Context, +) -> Result { + let output_type = ctx.return_type.as_struct(); + jsonb.populate_struct(output_type, base).map_err(parse_err) +} + +/// Expands the top-level JSON array of objects to a set of rows having the composite type of the +/// base argument. Each element of the JSON array is processed as described above for +/// `jsonb_populate_record`. +/// +/// # Examples +/// +/// ```slt +/// query II +/// select * from jsonb_populate_recordset( +/// null::struct, +/// '[{"a":1,"b":2}, {"a":3,"b":4}]'::jsonb +/// ); +/// ---- +/// 1 2 +/// 3 4 +/// +/// query II +/// select * from jsonb_populate_recordset( +/// row(0, 0)::struct, +/// '[{}, {"a":1}, {"b":2}, {"a":1,"b":2}]'::jsonb +/// ); +/// ---- +/// 0 0 +/// 1 0 +/// 0 2 +/// 1 2 +/// ``` +#[function("jsonb_populate_recordset(struct, jsonb) -> setof struct")] +fn jsonb_populate_recordset<'a>( + base: Option>, + jsonb: JsonbRef<'a>, + ctx: &'a Context, +) -> Result> + 'a> { + let output_type = ctx.return_type.as_struct(); + Ok(jsonb + .array_elements() + .map_err(parse_err)? + .map(move |elem| elem.populate_struct(output_type, base).map_err(parse_err))) +} + +/// Expands the top-level JSON object to a row having the composite type defined by an AS clause. +/// The output record is filled from fields of the JSON object, in the same way as described above +/// for `jsonb_populate_record`. Since there is no input record value, unmatched columns are always +/// filled with nulls. +/// +/// # Examples +/// +/// // FIXME(runji): this query is blocked by parser and frontend support. +/// ```slt,ignore +/// query T +/// select * from jsonb_to_record('{"a":1,"b":[1,2,3],"c":[1,2,3],"e":"bar","r": {"a": 123, "b": "a b c"}}') +/// as x(a int, b text, c int[], d text, r struct); +/// ---- +/// 1 [1,2,3] {1,2,3} NULL (123,"a b c") +/// ``` +#[function("jsonb_to_record(jsonb) -> struct", type_infer = "panic")] +fn jsonb_to_record(jsonb: JsonbRef<'_>, ctx: &Context) -> Result { + let output_type = ctx.return_type.as_struct(); + jsonb.to_struct(output_type).map_err(parse_err) +} + +/// Expands the top-level JSON array of objects to a set of rows having the composite type defined +/// by an AS clause. Each element of the JSON array is processed as described above for +/// `jsonb_populate_record`. +/// +/// # Examples +/// +/// // FIXME(runji): this query is blocked by parser and frontend support. +/// ```slt,ignore +/// query IT +/// select * from jsonb_to_recordset('[{"a":1,"b":"foo"}, {"a":"2","c":"bar"}]') as x(a int, b text); +/// ---- +/// 1 foo +/// 2 NULL +/// ``` +#[function("jsonb_to_recordset(jsonb) -> setof struct", type_infer = "panic")] +fn jsonb_to_recordset<'a>( + jsonb: JsonbRef<'a>, + ctx: &'a Context, +) -> Result> + 'a> { + let output_type = ctx.return_type.as_struct(); + Ok(jsonb + .array_elements() + .map_err(parse_err)? + .map(|elem| elem.to_struct(output_type).map_err(parse_err))) +} + +/// Construct a parse error from String. +fn parse_err(s: String) -> ExprError { + ExprError::Parse(s.into()) +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index 27135a739763..d2f528ece0c6 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -57,6 +57,7 @@ mod jsonb_delete; mod jsonb_info; mod jsonb_object; mod jsonb_path; +mod jsonb_record; mod length; mod lower; mod make_time; diff --git a/src/expr/macro/src/gen.rs b/src/expr/macro/src/gen.rs index 684a0e215975..ca3203033a63 100644 --- a/src/expr/macro/src/gen.rs +++ b/src/expr/macro/src/gen.rs @@ -1037,10 +1037,10 @@ impl FunctionAttr { .map(|ty| format_ident!("{}Builder", types::array_type(ty))) .collect_vec(); let return_types = if return_types.len() == 1 { - vec![quote! { self.return_type.clone() }] + vec![quote! { self.context.return_type.clone() }] } else { (0..return_types.len()) - .map(|i| quote! { self.return_type.as_struct().types().nth(#i).unwrap().clone() }) + .map(|i| quote! { self.context.return_type.as_struct().types().nth(#i).unwrap().clone() }) .collect() }; #[allow(clippy::disallowed_methods)] @@ -1060,14 +1060,15 @@ impl FunctionAttr { } else { quote! { let value_array = StructArray::new( - self.return_type.as_struct().clone(), + self.context.return_type.as_struct().clone(), value_arrays.to_vec(), Bitmap::ones(len), ).into_ref(); } }; + let context = user_fn.context.then(|| quote! { &self.context, }); let prebuilt_arg = match &self.prebuild { - Some(_) => quote! { &self.prebuilt_arg }, + Some(_) => quote! { &self.prebuilt_arg, }, None => quote! {}, }; let prebuilt_arg_type = match &self.prebuild { @@ -1081,12 +1082,32 @@ impl FunctionAttr { .expect("invalid prebuild syntax"), None => quote! { () }, }; - let iter = match user_fn.return_type_kind { - ReturnTypeKind::T => quote! { iter }, - ReturnTypeKind::Option => quote! { if let Some(it) = iter { it } else { continue; } }, - ReturnTypeKind::Result => quote! { iter? }, + let iter = quote! { #fn_name(#(#inputs,)* #prebuilt_arg #context) }; + let mut iter = match user_fn.return_type_kind { + ReturnTypeKind::T => quote! { #iter }, + ReturnTypeKind::Result => quote! { #iter? }, + ReturnTypeKind::Option => quote! { if let Some(it) = #iter { it } else { continue; } }, ReturnTypeKind::ResultOption => { - quote! { if let Some(it) = iter? { it } else { continue; } } + quote! { if let Some(it) = #iter? { it } else { continue; } } + } + }; + // if user function accepts non-option arguments, we assume the function + // returns empty on null input, so we need to unwrap the inputs before calling. + #[allow(clippy::disallowed_methods)] // allow zip + let some_inputs = inputs + .iter() + .zip(user_fn.args_option.iter()) + .map(|(input, opt)| { + if *opt { + quote! { #input } + } else { + quote! { Some(#input) } + } + }); + iter = quote! { + match (#(#inputs,)*) { + (#(#some_inputs,)*) => #iter, + _ => continue, } }; let iterator_item_type = user_fn.iterator_item_kind.clone().ok_or_else(|| { @@ -1108,11 +1129,18 @@ impl FunctionAttr { use risingwave_common::types::*; use risingwave_common::buffer::Bitmap; use risingwave_common::util::iter_util::ZipEqFast; - use risingwave_expr::expr::BoxedExpression; + use risingwave_expr::expr::{BoxedExpression, Context}; use risingwave_expr::{Result, ExprError}; use risingwave_expr::codegen::*; risingwave_expr::ensure!(children.len() == #num_args); + + let context = Context { + return_type: return_type.clone(), + arg_types: children.iter().map(|c| c.return_type()).collect(), + variadic: false, + }; + let mut iter = children.into_iter(); #(let #all_child = iter.next().unwrap();)* #( @@ -1126,7 +1154,7 @@ impl FunctionAttr { #[derive(Debug)] struct #struct_name { - return_type: DataType, + context: Context, chunk_size: usize, #(#child: BoxedExpression,)* prebuilt_arg: #prebuilt_arg_type, @@ -1134,7 +1162,7 @@ impl FunctionAttr { #[async_trait] impl risingwave_expr::table_function::TableFunction for #struct_name { fn return_type(&self) -> DataType { - self.return_type.clone() + self.context.return_type.clone() } async fn eval<'a>(&'a self, input: &'a DataChunk) -> BoxStream<'a, Result> { self.eval_inner(input) @@ -1151,23 +1179,23 @@ impl FunctionAttr { let mut index_builder = I32ArrayBuilder::new(self.chunk_size); #(let mut #builders = #builder_types::with_type(self.chunk_size, #return_types);)* - for (i, (row, visible)) in multizip((#(#arrays.iter(),)*)).zip_eq_fast(input.visibility().iter()).enumerate() { - if let (#(Some(#inputs),)*) = row && visible { - let iter = #fn_name(#(#inputs,)* #prebuilt_arg); - for output in #iter { - index_builder.append(Some(i as i32)); - match #output { - Some((#(#outputs),*)) => { #(#builders.append(#optioned_outputs);)* } - None => { #(#builders.append_null();)* } - } + for (i, ((#(#inputs,)*), visible)) in multizip((#(#arrays.iter(),)*)).zip_eq_fast(input.visibility().iter()).enumerate() { + if !visible { + continue; + } + for output in #iter { + index_builder.append(Some(i as i32)); + match #output { + Some((#(#outputs),*)) => { #(#builders.append(#optioned_outputs);)* } + None => { #(#builders.append_null();)* } + } - if index_builder.len() == self.chunk_size { - let len = index_builder.len(); - let index_array = std::mem::replace(&mut index_builder, I32ArrayBuilder::new(self.chunk_size)).finish().into_ref(); - let value_arrays = [#(std::mem::replace(&mut #builders, #builder_types::with_type(self.chunk_size, #return_types)).finish().into_ref()),*]; - #build_value_array - yield DataChunk::new(vec![index_array, value_array], self.chunk_size); - } + if index_builder.len() == self.chunk_size { + let len = index_builder.len(); + let index_array = std::mem::replace(&mut index_builder, I32ArrayBuilder::new(self.chunk_size)).finish().into_ref(); + let value_arrays = [#(std::mem::replace(&mut #builders, #builder_types::with_type(self.chunk_size, #return_types)).finish().into_ref()),*]; + #build_value_array + yield DataChunk::new(vec![index_array, value_array], self.chunk_size); } } } @@ -1183,7 +1211,7 @@ impl FunctionAttr { } Ok(Box::new(#struct_name { - return_type, + context, chunk_size, #(#child,)* prebuilt_arg: #prebuilt_arg_value, diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 1d424f7be4f4..1b36e9ee2fd7 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -1121,6 +1121,7 @@ impl Binder { ("to_jsonb", raw_call(ExprType::ToJsonb)), ("jsonb_build_array", raw_call(ExprType::JsonbBuildArray)), ("jsonb_build_object", raw_call(ExprType::JsonbBuildObject)), + ("jsonb_populate_record", raw_call(ExprType::JsonbPopulateRecord)), ("jsonb_path_match", raw_call(ExprType::JsonbPathMatch)), ("jsonb_path_exists", raw_call(ExprType::JsonbPathExists)), ("jsonb_path_query_array", raw_call(ExprType::JsonbPathQueryArray)), diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index bc56733a4423..310e7a9187da 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -201,6 +201,8 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::JsonbBuildArray | expr_node::Type::JsonbBuildArrayVariadic | expr_node::Type::JsonbBuildObject + | expr_node::Type::JsonbPopulateRecord + | expr_node::Type::JsonbToRecord | expr_node::Type::JsonbBuildObjectVariadic | expr_node::Type::JsonbPathExists | expr_node::Type::JsonbPathMatch diff --git a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs index 79905a076e42..a3d2e0edfb15 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs @@ -285,6 +285,8 @@ impl Strong { | ExprType::JsonbPathMatch | ExprType::JsonbPathQueryArray | ExprType::JsonbPathQueryFirst + | ExprType::JsonbPopulateRecord + | ExprType::JsonbToRecord | ExprType::Vnode | ExprType::Proctime | ExprType::PgSleep From 32df9cb2b90cc68c0bfaaf888d7a2028522ae4eb Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Wed, 17 Apr 2024 10:23:27 +0800 Subject: [PATCH 39/64] feat(frontend): fearless recursion on deep plans (#16279) Signed-off-by: Bugen Zhao --- Cargo.lock | 14 + src/common/Cargo.toml | 1 + src/common/src/util/mod.rs | 1 + src/common/src/util/recursive.rs | 190 +++++++++++ src/frontend/src/optimizer/plan_node/mod.rs | 112 ++++--- src/frontend/src/stream_fragmenter/mod.rs | 346 ++++++++++---------- 6 files changed, 448 insertions(+), 216 deletions(-) create mode 100644 src/common/src/util/recursive.rs diff --git a/Cargo.lock b/Cargo.lock index aeee200fe918..d86418775220 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9325,6 +9325,7 @@ dependencies = [ "serde_with", "smallbitset", "speedate", + "stacker", "static_assertions", "strum 0.26.1", "strum_macros 0.26.1", @@ -12342,6 +12343,19 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +[[package]] +name = "stacker" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c886bd4480155fd3ef527d45e9ac8dd7118a898a46530b7b94c3e21866259fce" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "winapi", +] + [[package]] name = "static_assertions" version = "1.1.0" diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 3108e06e789f..d21e276089c2 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -93,6 +93,7 @@ serde_json = "1" serde_with = "3" smallbitset = "0.7.1" speedate = "0.14.0" +stacker = "0.1" static_assertions = "1" strum = "0.26" strum_macros = "0.26" diff --git a/src/common/src/util/mod.rs b/src/common/src/util/mod.rs index bb64f5a58c80..c8027ad46e38 100644 --- a/src/common/src/util/mod.rs +++ b/src/common/src/util/mod.rs @@ -30,6 +30,7 @@ pub mod pretty_bytes; pub mod prost; pub mod query_log; pub use rw_resource_util as resource_util; +pub mod recursive; pub mod row_id; pub mod row_serde; pub mod runtime; diff --git a/src/common/src/util/recursive.rs b/src/common/src/util/recursive.rs new file mode 100644 index 000000000000..2869b3c49633 --- /dev/null +++ b/src/common/src/util/recursive.rs @@ -0,0 +1,190 @@ +// 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. + +//! Track the recursion and grow the stack when necessary to enable fearless recursion. + +use std::cell::RefCell; + +// See documentation of `stacker` for the meaning of these constants. +// TODO: determine good values or make them configurable +const RED_ZONE: usize = 128 * 1024; // 128KiB +const STACK_SIZE: usize = 16 * RED_ZONE; // 2MiB + +/// Recursion depth. +struct Depth { + /// The current depth. + current: usize, + /// The max depth reached so far, not considering the current depth. + last_max: usize, +} + +impl Depth { + const fn new() -> Self { + Self { + current: 0, + last_max: 0, + } + } + + fn reset(&mut self) { + *self = Self::new(); + } +} + +/// The tracker for a recursive function. +pub struct Tracker { + depth: RefCell, +} + +impl Tracker { + /// Create a new tracker. + pub const fn new() -> Self { + Self { + depth: RefCell::new(Depth::new()), + } + } + + /// Retrieve the current depth of the recursion. Starts from 1 once the + /// recursive function is called. + pub fn depth(&self) -> usize { + self.depth.borrow().current + } + + /// Check if the current depth reaches the given depth **for the first time**. + /// + /// This is useful for logging without any duplication. + pub fn depth_reaches(&self, depth: usize) -> bool { + let d = self.depth.borrow(); + d.current == depth && d.current > d.last_max + } + + /// Run a recursive function. Grow the stack if necessary. + fn recurse(&self, f: impl FnOnce() -> T) -> T { + struct DepthGuard<'a> { + depth: &'a RefCell, + } + + impl<'a> DepthGuard<'a> { + fn new(depth: &'a RefCell) -> Self { + depth.borrow_mut().current += 1; + Self { depth } + } + } + + impl<'a> Drop for DepthGuard<'a> { + fn drop(&mut self) { + let mut d = self.depth.borrow_mut(); + d.last_max = d.last_max.max(d.current); // update the last max depth + d.current -= 1; // restore the current depth + if d.current == 0 { + d.reset(); // reset state if the recursion is finished + } + } + } + + let _guard = DepthGuard::new(&self.depth); + + if cfg!(madsim) { + f() // madsim does not support stack growth + } else { + stacker::maybe_grow(RED_ZONE, STACK_SIZE, f) + } + } +} + +/// The extension trait for a thread-local tracker to run a recursive function. +#[easy_ext::ext(Recurse)] +impl std::thread::LocalKey { + /// Run the given recursive function. Grow the stack if necessary. + /// + /// # Fearless Recursion + /// + /// This enables fearless recursion in most cases as long as a single frame + /// does not exceed the [`RED_ZONE`] size. That is, the caller can recurse + /// as much as it wants without worrying about stack overflow. + /// + /// # Tracker + /// + /// The caller can retrieve the [`Tracker`] of the current recursion from + /// the closure argument. This can be useful for checking the depth of the + /// recursion, logging or throwing an error gracefully if it's too deep. + /// + /// Note that different trackers defined in different functions are + /// independent of each other. If there's a cross-function recursion, the + /// tracker retrieved from the closure argument only represents the current + /// function's state. + /// + /// # Example + /// + /// Define the tracker with [`tracker!`] and call this method on it to run + /// a recursive function. + /// + /// ```ignore + /// #[inline(never)] + /// fn sum(x: u64) -> u64 { + /// tracker!().recurse(|t| { + /// if t.depth() % 100000 == 0 { + /// eprintln!("too deep!"); + /// } + /// if x == 0 { + /// return 0; + /// } + /// x + sum(x - 1) + /// }) + /// } + /// ``` + pub fn recurse(&'static self, f: impl FnOnce(&Tracker) -> T) -> T { + self.with(|t| t.recurse(|| f(t))) + } +} + +/// Define the tracker for recursion and return it. +/// +/// Call [`Recurse::recurse`] on it to run a recursive function. See +/// documentation there for usage. +#[macro_export] +macro_rules! __recursive_tracker { + () => {{ + use $crate::util::recursive::Tracker; + std::thread_local! { + static __TRACKER: Tracker = const { Tracker::new() }; + } + __TRACKER + }}; +} +pub use __recursive_tracker as tracker; + +#[cfg(all(test, not(madsim)))] +mod tests { + use super::*; + + #[test] + fn test_fearless_recursion() { + const X: u64 = 1919810; + const EXPECTED: u64 = 1842836177955; + + #[inline(never)] + fn sum(x: u64) -> u64 { + tracker!().recurse(|t| { + if x == 0 { + assert_eq!(t.depth(), X as usize + 1); + return 0; + } + x + sum(x - 1) + }) + } + + assert_eq!(sum(X), EXPECTED); + } +} diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 780e1d2b39cd..ff749781f926 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -39,6 +39,7 @@ use itertools::Itertools; use paste::paste; use pretty_xmlish::{Pretty, PrettyConfig}; use risingwave_common::catalog::Schema; +use risingwave_common::util::recursive::{self, Recurse}; use risingwave_pb::batch_plan::PlanNode as BatchPlanPb; use risingwave_pb::stream_plan::StreamNode as StreamPlanPb; use serde::Serialize; @@ -51,6 +52,7 @@ use self::utils::Distill; use super::property::{Distribution, FunctionalDependencySet, Order}; use crate::error::{ErrorCode, Result}; use crate::optimizer::ExpressionSimplifyRewriter; +use crate::session::current::notice_to_user; /// A marker trait for different conventions, used for enforcing type safety. /// @@ -694,6 +696,10 @@ impl dyn PlanNode { } } +const PLAN_DEPTH_THRESHOLD: usize = 30; +const PLAN_TOO_DEEP_NOTICE: &str = "The plan is too deep. \ +Consider simplifying or splitting the query if you encounter any issues."; + impl dyn PlanNode { /// Serialize the plan node and its children to a stream plan proto. /// @@ -703,41 +709,47 @@ impl dyn PlanNode { &self, state: &mut BuildFragmentGraphState, ) -> SchedulerResult { - use stream::prelude::*; + recursive::tracker!().recurse(|t| { + if t.depth_reaches(PLAN_DEPTH_THRESHOLD) { + notice_to_user(PLAN_TOO_DEEP_NOTICE); + } - if let Some(stream_table_scan) = self.as_stream_table_scan() { - return stream_table_scan.adhoc_to_stream_prost(state); - } - if let Some(stream_cdc_table_scan) = self.as_stream_cdc_table_scan() { - return stream_cdc_table_scan.adhoc_to_stream_prost(state); - } - if let Some(stream_source_scan) = self.as_stream_source_scan() { - return stream_source_scan.adhoc_to_stream_prost(state); - } - if let Some(stream_share) = self.as_stream_share() { - return stream_share.adhoc_to_stream_prost(state); - } + use stream::prelude::*; - let node = Some(self.try_to_stream_prost_body(state)?); - let input = self - .inputs() - .into_iter() - .map(|plan| plan.to_stream_prost(state)) - .try_collect()?; - // TODO: support pk_indices and operator_id - Ok(StreamPlanPb { - input, - identity: self.explain_myself_to_string(), - node_body: node, - operator_id: self.id().0 as _, - stream_key: self - .stream_key() - .unwrap_or_default() - .iter() - .map(|x| *x as u32) - .collect(), - fields: self.schema().to_prost(), - append_only: self.plan_base().append_only(), + if let Some(stream_table_scan) = self.as_stream_table_scan() { + return stream_table_scan.adhoc_to_stream_prost(state); + } + if let Some(stream_cdc_table_scan) = self.as_stream_cdc_table_scan() { + return stream_cdc_table_scan.adhoc_to_stream_prost(state); + } + if let Some(stream_source_scan) = self.as_stream_source_scan() { + return stream_source_scan.adhoc_to_stream_prost(state); + } + if let Some(stream_share) = self.as_stream_share() { + return stream_share.adhoc_to_stream_prost(state); + } + + let node = Some(self.try_to_stream_prost_body(state)?); + let input = self + .inputs() + .into_iter() + .map(|plan| plan.to_stream_prost(state)) + .try_collect()?; + // TODO: support pk_indices and operator_id + Ok(StreamPlanPb { + input, + identity: self.explain_myself_to_string(), + node_body: node, + operator_id: self.id().0 as _, + stream_key: self + .stream_key() + .unwrap_or_default() + .iter() + .map(|x| *x as u32) + .collect(), + fields: self.schema().to_prost(), + append_only: self.plan_base().append_only(), + }) }) } @@ -749,20 +761,26 @@ impl dyn PlanNode { /// Serialize the plan node and its children to a batch plan proto without the identity field /// (for testing). pub fn to_batch_prost_identity(&self, identity: bool) -> SchedulerResult { - let node_body = Some(self.try_to_batch_prost_body()?); - let children = self - .inputs() - .into_iter() - .map(|plan| plan.to_batch_prost_identity(identity)) - .try_collect()?; - Ok(BatchPlanPb { - children, - identity: if identity { - self.explain_myself_to_string() - } else { - "".into() - }, - node_body, + recursive::tracker!().recurse(|t| { + if t.depth_reaches(PLAN_DEPTH_THRESHOLD) { + notice_to_user(PLAN_TOO_DEEP_NOTICE); + } + + let node_body = Some(self.try_to_batch_prost_body()?); + let children = self + .inputs() + .into_iter() + .map(|plan| plan.to_batch_prost_identity(identity)) + .try_collect()?; + Ok(BatchPlanPb { + children, + identity: if identity { + self.explain_myself_to_string() + } else { + "".into() + }, + node_body, + }) }) } diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 009449ec9228..e7548ce5fa17 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -14,6 +14,7 @@ mod graph; use graph::*; +use risingwave_common::util::recursive::{self, Recurse as _}; use risingwave_pb::stream_plan::stream_node::NodeBody; mod rewrite; @@ -253,201 +254,208 @@ fn build_fragment( current_fragment: &mut StreamFragment, mut stream_node: StreamNode, ) -> Result { - // Update current fragment based on the node we're visiting. - match stream_node.get_node_body()? { - NodeBody::BarrierRecv(_) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::BarrierRecv as u32 - } + recursive::tracker!().recurse(|_t| { + // Update current fragment based on the node we're visiting. + match stream_node.get_node_body()? { + NodeBody::BarrierRecv(_) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::BarrierRecv as u32 + } - NodeBody::Source(node) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::Source as u32; + NodeBody::Source(node) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::Source as u32; - if let Some(source) = node.source_inner.as_ref() - && let Some(source_info) = source.info.as_ref() - && source_info.is_shared() - && !source_info.is_distributed - { - current_fragment.requires_singleton = true; + if let Some(source) = node.source_inner.as_ref() + && let Some(source_info) = source.info.as_ref() + && source_info.is_shared() + && !source_info.is_distributed + { + current_fragment.requires_singleton = true; + } } - } - NodeBody::Dml(_) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::Dml as u32; - } + NodeBody::Dml(_) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::Dml as u32; + } - NodeBody::Materialize(_) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::Mview as u32; - } + NodeBody::Materialize(_) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::Mview as u32; + } - NodeBody::Sink(_) => current_fragment.fragment_type_mask |= FragmentTypeFlag::Sink as u32, + NodeBody::Sink(_) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::Sink as u32 + } - NodeBody::Subscription(_) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::Subscription as u32 - } + NodeBody::Subscription(_) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::Subscription as u32 + } - NodeBody::TopN(_) => current_fragment.requires_singleton = true, + NodeBody::TopN(_) => current_fragment.requires_singleton = true, - NodeBody::StreamScan(node) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::StreamScan as u32; - // memorize table id for later use - // The table id could be a upstream CDC source - state - .dependent_table_ids - .insert(TableId::new(node.table_id)); - current_fragment.upstream_table_ids.push(node.table_id); - } + NodeBody::StreamScan(node) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::StreamScan as u32; + // memorize table id for later use + // The table id could be a upstream CDC source + state + .dependent_table_ids + .insert(TableId::new(node.table_id)); + current_fragment.upstream_table_ids.push(node.table_id); + } - NodeBody::StreamCdcScan(_) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::StreamScan as u32; - // the backfill algorithm is not parallel safe - current_fragment.requires_singleton = true; - } + NodeBody::StreamCdcScan(_) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::StreamScan as u32; + // the backfill algorithm is not parallel safe + current_fragment.requires_singleton = true; + } - NodeBody::CdcFilter(node) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::CdcFilter as u32; - // memorize upstream source id for later use - state - .dependent_table_ids - .insert(TableId::new(node.upstream_source_id)); - current_fragment - .upstream_table_ids - .push(node.upstream_source_id); - } - NodeBody::SourceBackfill(node) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::SourceScan as u32; - // memorize upstream source id for later use - let source_id = node.upstream_source_id; - state.dependent_table_ids.insert(source_id.into()); - current_fragment.upstream_table_ids.push(source_id); - } + NodeBody::CdcFilter(node) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::CdcFilter as u32; + // memorize upstream source id for later use + state + .dependent_table_ids + .insert(TableId::new(node.upstream_source_id)); + current_fragment + .upstream_table_ids + .push(node.upstream_source_id); + } + NodeBody::SourceBackfill(node) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::SourceScan as u32; + // memorize upstream source id for later use + let source_id = node.upstream_source_id; + state.dependent_table_ids.insert(source_id.into()); + current_fragment.upstream_table_ids.push(source_id); + } - NodeBody::Now(_) => { - // TODO: Remove this and insert a `BarrierRecv` instead. - current_fragment.fragment_type_mask |= FragmentTypeFlag::Now as u32; - current_fragment.requires_singleton = true; - } + NodeBody::Now(_) => { + // TODO: Remove this and insert a `BarrierRecv` instead. + current_fragment.fragment_type_mask |= FragmentTypeFlag::Now as u32; + current_fragment.requires_singleton = true; + } - NodeBody::Values(_) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::Values as u32; - current_fragment.requires_singleton = true; - } + NodeBody::Values(_) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::Values as u32; + current_fragment.requires_singleton = true; + } - _ => {} - }; + _ => {} + }; - // handle join logic - if let NodeBody::DeltaIndexJoin(delta_index_join) = stream_node.node_body.as_mut().unwrap() { - if delta_index_join.get_join_type()? == JoinType::Inner - && delta_index_join.condition.is_none() + // handle join logic + if let NodeBody::DeltaIndexJoin(delta_index_join) = stream_node.node_body.as_mut().unwrap() { - return build_delta_join_without_arrange(state, current_fragment, stream_node); - } else { - panic!("only inner join without non-equal condition is supported for delta joins"); + if delta_index_join.get_join_type()? == JoinType::Inner + && delta_index_join.condition.is_none() + { + return build_delta_join_without_arrange(state, current_fragment, stream_node); + } else { + panic!("only inner join without non-equal condition is supported for delta joins"); + } } - } - // Usually we do not expect exchange node to be visited here, which should be handled by the - // following logic of "visit children" instead. If it does happen (for example, `Share` will be - // transformed to an `Exchange`), it means we have an empty fragment and we need to add a no-op - // node to it, so that the meta service can handle it correctly. - if let NodeBody::Exchange(_) = stream_node.node_body.as_ref().unwrap() { - stream_node = state.gen_no_op_stream_node(stream_node); - } + // Usually we do not expect exchange node to be visited here, which should be handled by the + // following logic of "visit children" instead. If it does happen (for example, `Share` will be + // transformed to an `Exchange`), it means we have an empty fragment and we need to add a no-op + // node to it, so that the meta service can handle it correctly. + if let NodeBody::Exchange(_) = stream_node.node_body.as_ref().unwrap() { + stream_node = state.gen_no_op_stream_node(stream_node); + } - // Visit plan children. - stream_node.input = stream_node - .input - .into_iter() - .map(|mut child_node| { - match child_node.get_node_body()? { - // When exchange node is generated when doing rewrites, it could be having - // zero input. In this case, we won't recursively visit its children. - NodeBody::Exchange(_) if child_node.input.is_empty() => Ok(child_node), - // Exchange node indicates a new child fragment. - NodeBody::Exchange(exchange_node) => { - let exchange_node_strategy = exchange_node.get_strategy()?.clone(); - - // Exchange node should have only one input. - let [input]: [_; 1] = std::mem::take(&mut child_node.input).try_into().unwrap(); - let child_fragment = build_and_add_fragment(state, input)?; - - let result = state.fragment_graph.try_add_edge( - child_fragment.fragment_id, - current_fragment.fragment_id, - StreamFragmentEdge { - dispatch_strategy: exchange_node_strategy.clone(), - // Always use the exchange operator id as the link id. - link_id: child_node.operator_id, - }, - ); - - // It's possible that there're multiple edges between two fragments, while the - // meta service and the compute node does not expect this. In this case, we - // manually insert a fragment of `NoOp` between the two fragments. - if result.is_err() { - // Assign a new operator id for the `Exchange`, so we can distinguish it - // from duplicate edges and break the sharing. - child_node.operator_id = state.gen_operator_id() as u64; - - // Take the upstream plan node as the reference for properties of `NoOp`. - let ref_fragment_node = child_fragment.node.as_ref().unwrap(); - let no_shuffle_strategy = DispatchStrategy { - r#type: DispatcherType::NoShuffle as i32, - dist_key_indices: vec![], - output_indices: (0..ref_fragment_node.fields.len() as u32).collect(), - }; - - let no_shuffle_exchange_operator_id = state.gen_operator_id() as u64; - - let no_op_fragment = { - let node = state.gen_no_op_stream_node(StreamNode { - operator_id: no_shuffle_exchange_operator_id, - identity: "StreamNoShuffleExchange".into(), - node_body: Some(NodeBody::Exchange(ExchangeNode { - strategy: Some(no_shuffle_strategy.clone()), - })), - input: vec![], - - // Take reference's properties. - stream_key: ref_fragment_node.stream_key.clone(), - append_only: ref_fragment_node.append_only, - fields: ref_fragment_node.fields.clone(), - }); - - let mut fragment = state.new_stream_fragment(); - fragment.node = Some(node.into()); - Rc::new(fragment) - }; - - state.fragment_graph.add_fragment(no_op_fragment.clone()); - - state.fragment_graph.add_edge( + // Visit plan children. + stream_node.input = stream_node + .input + .into_iter() + .map(|mut child_node| { + match child_node.get_node_body()? { + // When exchange node is generated when doing rewrites, it could be having + // zero input. In this case, we won't recursively visit its children. + NodeBody::Exchange(_) if child_node.input.is_empty() => Ok(child_node), + // Exchange node indicates a new child fragment. + NodeBody::Exchange(exchange_node) => { + let exchange_node_strategy = exchange_node.get_strategy()?.clone(); + + // Exchange node should have only one input. + let [input]: [_; 1] = + std::mem::take(&mut child_node.input).try_into().unwrap(); + let child_fragment = build_and_add_fragment(state, input)?; + + let result = state.fragment_graph.try_add_edge( child_fragment.fragment_id, - no_op_fragment.fragment_id, - StreamFragmentEdge { - // Use `NoShuffle` exhcnage strategy for upstream edge. - dispatch_strategy: no_shuffle_strategy, - link_id: no_shuffle_exchange_operator_id, - }, - ); - state.fragment_graph.add_edge( - no_op_fragment.fragment_id, current_fragment.fragment_id, StreamFragmentEdge { - // Use the original exchange strategy for downstream edge. - dispatch_strategy: exchange_node_strategy, + dispatch_strategy: exchange_node_strategy.clone(), + // Always use the exchange operator id as the link id. link_id: child_node.operator_id, }, ); + + // It's possible that there're multiple edges between two fragments, while the + // meta service and the compute node does not expect this. In this case, we + // manually insert a fragment of `NoOp` between the two fragments. + if result.is_err() { + // Assign a new operator id for the `Exchange`, so we can distinguish it + // from duplicate edges and break the sharing. + child_node.operator_id = state.gen_operator_id() as u64; + + // Take the upstream plan node as the reference for properties of `NoOp`. + let ref_fragment_node = child_fragment.node.as_ref().unwrap(); + let no_shuffle_strategy = DispatchStrategy { + r#type: DispatcherType::NoShuffle as i32, + dist_key_indices: vec![], + output_indices: (0..ref_fragment_node.fields.len() as u32) + .collect(), + }; + + let no_shuffle_exchange_operator_id = state.gen_operator_id() as u64; + + let no_op_fragment = { + let node = state.gen_no_op_stream_node(StreamNode { + operator_id: no_shuffle_exchange_operator_id, + identity: "StreamNoShuffleExchange".into(), + node_body: Some(NodeBody::Exchange(ExchangeNode { + strategy: Some(no_shuffle_strategy.clone()), + })), + input: vec![], + + // Take reference's properties. + stream_key: ref_fragment_node.stream_key.clone(), + append_only: ref_fragment_node.append_only, + fields: ref_fragment_node.fields.clone(), + }); + + let mut fragment = state.new_stream_fragment(); + fragment.node = Some(node.into()); + Rc::new(fragment) + }; + + state.fragment_graph.add_fragment(no_op_fragment.clone()); + + state.fragment_graph.add_edge( + child_fragment.fragment_id, + no_op_fragment.fragment_id, + StreamFragmentEdge { + // Use `NoShuffle` exhcnage strategy for upstream edge. + dispatch_strategy: no_shuffle_strategy, + link_id: no_shuffle_exchange_operator_id, + }, + ); + state.fragment_graph.add_edge( + no_op_fragment.fragment_id, + current_fragment.fragment_id, + StreamFragmentEdge { + // Use the original exchange strategy for downstream edge. + dispatch_strategy: exchange_node_strategy, + link_id: child_node.operator_id, + }, + ); + } + + Ok(child_node) } - Ok(child_node) + // For other children, visit recursively. + _ => build_fragment(state, current_fragment, child_node), } - - // For other children, visit recursively. - _ => build_fragment(state, current_fragment, child_node), - } - }) - .collect::>()?; - Ok(stream_node) + }) + .collect::>()?; + Ok(stream_node) + }) } From a2694b975e80a4a4f7122c89fc2a0a1fcf23a7b1 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 17 Apr 2024 10:34:40 +0800 Subject: [PATCH 40/64] ci: bump ci image to ubuntu 24.04 (#16290) Signed-off-by: xxchan --- ci/Dockerfile | 27 ++++++++----------- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 10 +++---- ci/scripts/connector-node-integration-test.sh | 2 +- ci/scripts/e2e-cassandra-sink-test.sh | 9 ++++++- ci/scripts/e2e-iceberg-cdc.sh | 6 ++--- ci/scripts/e2e-iceberg-sink-v2-test.sh | 16 +++++------ ci/scripts/e2e-kafka-sink-test.sh | 2 +- ci/scripts/e2e-source-test.sh | 2 +- ci/scripts/pulsar-source-test.sh | 2 +- .../s3-source-test-for-opendal-fs-engine.sh | 2 +- ci/scripts/s3-source-test.sh | 2 +- e2e_test/iceberg/pyproject.toml | 2 +- integration_tests/iceberg-sink2/run.sh | 6 ++--- integration_tests/iceberg-source/run.sh | 6 ++--- scripts/source/prepare_ci_kafka.sh | 15 +++-------- scripts/source/prepare_data_after_alter.sh | 9 +------ 17 files changed, 54 insertions(+), 66 deletions(-) diff --git a/ci/Dockerfile b/ci/Dockerfile index 78c48f0c648d..5902b612c071 100644 --- a/ci/Dockerfile +++ b/ci/Dockerfile @@ -4,22 +4,18 @@ # and corresponding rw-build-env version in docker-compose.yml. # ############################################################################################# -FROM ubuntu:22.04 AS risingwave-build-env +FROM ubuntu:24.04 AS risingwave-build-env ENV LANG en_US.utf8 # Use AWS ubuntu mirror RUN sed -i 's|http://archive.ubuntu.com/ubuntu|http://us-east-2.ec2.archive.ubuntu.com/ubuntu/|g' /etc/apt/sources.list RUN apt-get update -yy && \ - DEBIAN_FRONTEND=noninteractive apt-get -y install make build-essential cmake protobuf-compiler curl parallel python3 python3-pip software-properties-common \ - openssl libssl-dev libsasl2-dev libcurl4-openssl-dev pkg-config bash openjdk-11-jdk wget unzip git tmux lld postgresql-client kafkacat netcat mysql-client \ - maven zstd libzstd-dev locales -yy \ + DEBIAN_FRONTEND=noninteractive apt-get -y install make build-essential cmake protobuf-compiler curl parallel python3 python3-pip python3-venv software-properties-common \ + openssl libssl-dev libsasl2-dev libcurl4-openssl-dev pkg-config bash openjdk-11-jdk wget unzip git tmux lld postgresql-client kcat netcat-openbsd mysql-client \ + maven zstd libzstd-dev locales \ + python3.12 python3.12-dev \ && rm -rf /var/lib/{apt,dpkg,cache,log}/ -# Install Python 3.12 -RUN add-apt-repository ppa:deadsnakes/ppa -y && \ - apt-get update -yy && \ - DEBIAN_FRONTEND=noninteractive apt-get install python3.12 python3.12-dev -yy && \ - rm -rf /var/lib/{apt,dpkg,cache,log}/ ENV PYO3_PYTHON=python3.12 SHELL ["/bin/bash", "-c"] @@ -34,21 +30,20 @@ RUN curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- --no-mo ENV PATH /root/.cargo/bin/:$PATH -RUN rustup show -RUN rustup default `rustup show active-toolchain | awk '{print $1}'` +RUN rustup show && \ + rustup default `rustup show active-toolchain | awk '{print $1}'` && \ + rustup component add rustfmt llvm-tools-preview clippy && \ + rustup target add wasm32-wasi RUN curl -sSL "https://github.com/bufbuild/buf/releases/download/v1.29.0/buf-$(uname -s)-$(uname -m).tar.gz" | \ tar -xvzf - -C /usr/local --strip-components 1 # install python dependencies -RUN pip3 install pyarrow pytest +RUN pip3 install --break-system-packages pyarrow pytest # Install poetry RUN curl -sSL https://install.python-poetry.org | python3 - - -# add required rustup components -RUN rustup component add rustfmt llvm-tools-preview clippy -RUN rustup target add wasm32-wasi +ENV PATH /root/.local/bin:$PATH ENV CARGO_REGISTRIES_CRATES_IO_PROTOCOL=sparse diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index 1a23144a8e8f..074f60aa97e0 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -10,7 +10,7 @@ cat ../rust-toolchain # shellcheck disable=SC2155 # REMEMBER TO ALSO UPDATE ci/docker-compose.yml -export BUILD_ENV_VERSION=v20240405_1 +export BUILD_ENV_VERSION=v20240413 export BUILD_TAG="public.ecr.aws/w1p7b4n3/rw-build-env:${BUILD_ENV_VERSION}" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 3b99cf1082df..1d67188b1c4f 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -71,7 +71,7 @@ services: retries: 5 source-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240405_1 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 depends_on: - mysql - db @@ -84,7 +84,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240405_1 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 depends_on: - mysql - db @@ -103,12 +103,12 @@ services: rw-build-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240405_1 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 volumes: - ..:/risingwave ci-flamegraph-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240405_1 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 # NOTE(kwannoel): This is used in order to permit # syscalls for `nperf` (perf_event_open), # so it can do CPU profiling. @@ -119,7 +119,7 @@ services: - ..:/risingwave regress-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240405_1 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 depends_on: db: condition: service_healthy diff --git a/ci/scripts/connector-node-integration-test.sh b/ci/scripts/connector-node-integration-test.sh index 0ebdd35ea682..519215584abb 100755 --- a/ci/scripts/connector-node-integration-test.sh +++ b/ci/scripts/connector-node-integration-test.sh @@ -86,7 +86,7 @@ tar xf java-binding-integration-test.tar.zst bin echo "--- prepare integration tests" cd "${RISINGWAVE_ROOT}"/java/connector-node -pip3 install grpcio grpcio-tools psycopg2 psycopg2-binary pyspark==3.3 black +pip3 install --break-system-packages grpcio grpcio-tools psycopg2 psycopg2-binary pyspark==3.3 black cd python-client && bash gen-stub.sh && bash format-python.sh --check export PYTHONPATH=proto diff --git a/ci/scripts/e2e-cassandra-sink-test.sh b/ci/scripts/e2e-cassandra-sink-test.sh index 6ca3d566d8a5..cae03843c470 100755 --- a/ci/scripts/e2e-cassandra-sink-test.sh +++ b/ci/scripts/e2e-cassandra-sink-test.sh @@ -38,6 +38,12 @@ sleep 1 echo "--- create cassandra table" curl https://downloads.apache.org/cassandra/4.1.3/apache-cassandra-4.1.3-bin.tar.gz --output apache-cassandra-4.1.3-bin.tar.gz tar xfvz apache-cassandra-4.1.3-bin.tar.gz +# remove bundled packages, and use installed packages, because Python 3.12 has removed asyncore, but I failed to install libev support for bundled Python driver. +rm apache-cassandra-4.1.3/lib/six-1.12.0-py2.py3-none-any.zip +rm apache-cassandra-4.1.3/lib/cassandra-driver-internal-only-3.25.0.zip +apt-get install -y libev4 libev-dev +pip3 install --break-system-packages cassandra-driver + cd apache-cassandra-4.1.3/bin export CQLSH_HOST=cassandra-server export CQLSH_PORT=9042 @@ -55,8 +61,9 @@ if cat ./query_result.csv | awk -F "," '{ exit !($1 == 1 && $2 == 1 && $3 == 1 && $4 == 1.1 && $5 == 1.2 && $6 == "test" && $7 == "2013-01-01" && $8 == "2013-01-01 01:01:01.000+0000" && $9 == "False\r"); }'; then echo "Cassandra sink check passed" else - cat ./query_result.csv echo "The output is not as expected." + echo "output:" + cat ./query_result.csv exit 1 fi diff --git a/ci/scripts/e2e-iceberg-cdc.sh b/ci/scripts/e2e-iceberg-cdc.sh index 58b18bd3e059..fc90f81d985a 100755 --- a/ci/scripts/e2e-iceberg-cdc.sh +++ b/ci/scripts/e2e-iceberg-cdc.sh @@ -47,13 +47,13 @@ bash ./start_spark_connect_server.sh # Don't remove the `--quiet` option since poetry has a bug when printing output, see # https://github.com/python-poetry/poetry/issues/3412 -"$HOME"/.local/bin/poetry update --quiet +poetry update --quiet # 1. import data to mysql mysql --host=mysql --port=3306 -u root -p123456 < ./test_case/cdc/mysql_cdc.sql # 2. create table and sink -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/cdc/no_partition_cdc_init.toml +poetry run python main.py -t ./test_case/cdc/no_partition_cdc_init.toml # 3. insert new data to mysql mysql --host=mysql --port=3306 -u root -p123456 < ./test_case/cdc/mysql_cdc_insert.sql @@ -61,4 +61,4 @@ mysql --host=mysql --port=3306 -u root -p123456 < ./test_case/cdc/mysql_cdc_inse sleep 20 # 4. check change -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/cdc/no_partition_cdc.toml \ No newline at end of file +poetry run python main.py -t ./test_case/cdc/no_partition_cdc.toml \ No newline at end of file diff --git a/ci/scripts/e2e-iceberg-sink-v2-test.sh b/ci/scripts/e2e-iceberg-sink-v2-test.sh index 847b6d32800a..dd2f78037a5f 100755 --- a/ci/scripts/e2e-iceberg-sink-v2-test.sh +++ b/ci/scripts/e2e-iceberg-sink-v2-test.sh @@ -37,14 +37,14 @@ bash ./start_spark_connect_server.sh # Don't remove the `--quiet` option since poetry has a bug when printing output, see # https://github.com/python-poetry/poetry/issues/3412 -"$HOME"/.local/bin/poetry update --quiet -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/no_partition_append_only.toml -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/no_partition_upsert.toml -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/partition_append_only.toml -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/partition_upsert.toml -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/range_partition_append_only.toml -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/range_partition_upsert.toml -"$HOME"/.local/bin/poetry run python main.py -t ./test_case/append_only_with_checkpoint_interval.toml +poetry update --quiet +poetry run python main.py -t ./test_case/no_partition_append_only.toml +poetry run python main.py -t ./test_case/no_partition_upsert.toml +poetry run python main.py -t ./test_case/partition_append_only.toml +poetry run python main.py -t ./test_case/partition_upsert.toml +poetry run python main.py -t ./test_case/range_partition_append_only.toml +poetry run python main.py -t ./test_case/range_partition_upsert.toml +poetry run python main.py -t ./test_case/append_only_with_checkpoint_interval.toml echo "--- Kill cluster" diff --git a/ci/scripts/e2e-kafka-sink-test.sh b/ci/scripts/e2e-kafka-sink-test.sh index b1d1f19c8f54..1dd7a27831d4 100755 --- a/ci/scripts/e2e-kafka-sink-test.sh +++ b/ci/scripts/e2e-kafka-sink-test.sh @@ -144,7 +144,7 @@ sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/drop_sink.slt' # test different encoding echo "preparing confluent schema registry" -python3 -m pip install requests confluent-kafka +python3 -m pip install --break-system-packages requests confluent-kafka echo "testing protobuf" cp src/connector/src/test_data/proto_recursive/recursive.pb ./proto-recursive diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 97e104096860..851fca303ab1 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -138,7 +138,7 @@ echo "--- e2e, ci-1cn-1fe, protobuf schema registry" export RISINGWAVE_CI=true RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-1cn-1fe -python3 -m pip install requests protobuf confluent-kafka +python3 -m pip install --break-system-packages requests protobuf confluent-kafka python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://message_queue:8081" "sr_pb_test" 20 user echo "make sure google/protobuf/source_context.proto is NOT in schema registry" curl --silent 'http://message_queue:8081/subjects'; echo diff --git a/ci/scripts/pulsar-source-test.sh b/ci/scripts/pulsar-source-test.sh index 6f0f7d176b00..86d99ed97f7d 100755 --- a/ci/scripts/pulsar-source-test.sh +++ b/ci/scripts/pulsar-source-test.sh @@ -45,7 +45,7 @@ echo "--- starting risingwave cluster with connector node" risedev ci-start ci-1cn-1fe echo "--- Run test" -python3 -m pip install psycopg2-binary +python3 -m pip install --break-system-packages psycopg2-binary python3 e2e_test/source/pulsar/astra-streaming.py # python3 e2e_test/source/pulsar/streamnative-cloud.py diff --git a/ci/scripts/s3-source-test-for-opendal-fs-engine.sh b/ci/scripts/s3-source-test-for-opendal-fs-engine.sh index 11bdb8ff2ce6..c0c7e91348ec 100755 --- a/ci/scripts/s3-source-test-for-opendal-fs-engine.sh +++ b/ci/scripts/s3-source-test-for-opendal-fs-engine.sh @@ -29,7 +29,7 @@ echo "--- starting risingwave cluster with connector node" risedev ci-start ci-3cn-3fe-opendal-fs-backend echo "--- Run test" -python3 -m pip install minio psycopg2-binary +python3 -m pip install --break-system-packages minio psycopg2-binary python3 e2e_test/s3/"$script" echo "--- Kill cluster" diff --git a/ci/scripts/s3-source-test.sh b/ci/scripts/s3-source-test.sh index 9bcb0d830cb4..532223693a21 100755 --- a/ci/scripts/s3-source-test.sh +++ b/ci/scripts/s3-source-test.sh @@ -32,7 +32,7 @@ echo "--- starting risingwave cluster with connector node" risedev ci-start ci-1cn-1fe echo "--- Run test" -python3 -m pip install minio psycopg2-binary opendal +python3 -m pip install --break-system-packages minio psycopg2-binary opendal if [[ -v format_type ]]; then python3 e2e_test/s3/"$script" "$format_type" else diff --git a/e2e_test/iceberg/pyproject.toml b/e2e_test/iceberg/pyproject.toml index d13be7227759..eba4bcd12f13 100644 --- a/e2e_test/iceberg/pyproject.toml +++ b/e2e_test/iceberg/pyproject.toml @@ -8,7 +8,7 @@ authors = ["risingwavelabs"] python = "^3.10" pyspark = { version = "3.4.1", extras = ["sql", "connect"] } tomli = "2.0" - +setuptools = "69" [build-system] requires = ["poetry-core"] diff --git a/integration_tests/iceberg-sink2/run.sh b/integration_tests/iceberg-sink2/run.sh index d58973f6c7c8..4fbf17d1116a 100755 --- a/integration_tests/iceberg-sink2/run.sh +++ b/integration_tests/iceberg-sink2/run.sh @@ -3,9 +3,9 @@ # Exits as soon as any line fails. set -euox pipefail -"$HOME"/.local/bin/poetry --version +poetry --version cd python # Don't remove the `--quiet` option since poetry has a bug when printing output, see # https://github.com/python-poetry/poetry/issues/3412 -"$HOME"/.local/bin/poetry update --quiet -"$HOME"/.local/bin/poetry run python main.py \ No newline at end of file +poetry update --quiet +poetry run python main.py \ No newline at end of file diff --git a/integration_tests/iceberg-source/run.sh b/integration_tests/iceberg-source/run.sh index d58973f6c7c8..4fbf17d1116a 100755 --- a/integration_tests/iceberg-source/run.sh +++ b/integration_tests/iceberg-source/run.sh @@ -3,9 +3,9 @@ # Exits as soon as any line fails. set -euox pipefail -"$HOME"/.local/bin/poetry --version +poetry --version cd python # Don't remove the `--quiet` option since poetry has a bug when printing output, see # https://github.com/python-poetry/poetry/issues/3412 -"$HOME"/.local/bin/poetry update --quiet -"$HOME"/.local/bin/poetry run python main.py \ No newline at end of file +poetry update --quiet +poetry run python main.py \ No newline at end of file diff --git a/scripts/source/prepare_ci_kafka.sh b/scripts/source/prepare_ci_kafka.sh index 68f69827bfc7..e79187282219 100755 --- a/scripts/source/prepare_ci_kafka.sh +++ b/scripts/source/prepare_ci_kafka.sh @@ -3,13 +3,6 @@ # Exits as soon as any line fails. set -e -KCAT_BIN="kcat" -# kcat bin name on linux is "kafkacat" -if [ "$(uname)" == "Linux" ] -then - KCAT_BIN="kafkacat" -fi - SCRIPT_PATH="$(cd "$(dirname "$0")" >/dev/null 2>&1 && pwd)" cd "$SCRIPT_PATH/.." || exit 1 @@ -54,7 +47,7 @@ done wait echo "Fulfill kafka topics" -python3 -m pip install requests fastavro confluent_kafka jsonschema +python3 -m pip install --break-system-packages requests fastavro confluent_kafka jsonschema for filename in $kafka_data_files; do ([ -e "$filename" ] base=$(basename "$filename") @@ -63,20 +56,20 @@ for filename in $kafka_data_files; do echo "Fulfill kafka topic $topic with data from $base" # binary data, one message a file, filename/topic ends with "bin" if [[ "$topic" = *bin ]]; then - ${KCAT_BIN} -P -b message_queue:29092 -t "$topic" "$filename" + kcat -P -b message_queue:29092 -t "$topic" "$filename" elif [[ "$topic" = *avro_json ]]; then python3 source/schema_registry_producer.py "message_queue:29092" "http://message_queue:8081" "$filename" "topic" "avro" elif [[ "$topic" = *json_schema ]]; then python3 source/schema_registry_producer.py "kafka:9093" "http://schemaregistry:8082" "$filename" "topic" "json" else - cat "$filename" | ${KCAT_BIN} -P -K ^ -b message_queue:29092 -t "$topic" + cat "$filename" | kcat -P -K ^ -b message_queue:29092 -t "$topic" fi ) & done # test additional columns: produce messages with headers ADDI_COLUMN_TOPIC="kafka_additional_columns" -for i in {0..100}; do echo "key$i:{\"a\": $i}" | ${KCAT_BIN} -P -b message_queue:29092 -t ${ADDI_COLUMN_TOPIC} -K : -H "header1=v1" -H "header2=v2"; done +for i in {0..100}; do echo "key$i:{\"a\": $i}" | kcat -P -b message_queue:29092 -t ${ADDI_COLUMN_TOPIC} -K : -H "header1=v1" -H "header2=v2"; done # write schema with name strategy diff --git a/scripts/source/prepare_data_after_alter.sh b/scripts/source/prepare_data_after_alter.sh index 3225ce5d128d..a82bd5932865 100644 --- a/scripts/source/prepare_data_after_alter.sh +++ b/scripts/source/prepare_data_after_alter.sh @@ -3,16 +3,9 @@ # Exits as soon as any line fails. set -e -KCAT_BIN="kcat" -# kcat bin name on linux is "kafkacat" -if [ "$(uname)" == "Linux" ] -then - KCAT_BIN="kafkacat" -fi - SCRIPT_PATH="$(cd "$(dirname "$0")" >/dev/null 2>&1 && pwd)" cd "$SCRIPT_PATH/.." || exit 1 FILE="./source/alter_data/kafka_alter.$1" echo "Send data from $FILE" -cat $FILE | ${KCAT_BIN} -P -b message_queue:29092 -t kafka_alter \ No newline at end of file +cat $FILE | kcat -P -b message_queue:29092 -t kafka_alter \ No newline at end of file From ffd4194351eb0aa43acb34d6e34c7eec84dab324 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 17 Apr 2024 10:40:38 +0800 Subject: [PATCH 41/64] feat(meta): make blocking sink default + support background sink (#16249) --- e2e_test/backfill/sink/create_sink.slt | 6 ++ .../rate_limit/snapshot_amplification.slt | 6 ++ .../rate_limit/upstream_amplification.slt | 6 ++ src/connector/src/sink/catalog/desc.rs | 2 +- src/meta/src/barrier/command.rs | 2 + src/meta/src/barrier/progress.rs | 91 ++++++++++--------- src/meta/src/manager/streaming_job.rs | 1 + src/meta/src/stream/stream_manager.rs | 1 + .../recovery/background_ddl.rs | 3 +- 9 files changed, 75 insertions(+), 43 deletions(-) diff --git a/e2e_test/backfill/sink/create_sink.slt b/e2e_test/backfill/sink/create_sink.slt index f98cd869bc9a..bc9fba04da5c 100644 --- a/e2e_test/backfill/sink/create_sink.slt +++ b/e2e_test/backfill/sink/create_sink.slt @@ -11,6 +11,9 @@ SET STREAMING_RATE_LIMIT = 1000; statement ok insert into t select * from generate_series(1, 10000); +statement ok +SET BACKGROUND_DDL=true; + statement ok create sink s as select x.v1 as v1 from t x join t y @@ -23,3 +26,6 @@ with ( allow.auto.create.topics=true, ) FORMAT DEBEZIUM ENCODE JSON; + +statement ok +SET BACKGROUND_DDL=false; diff --git a/e2e_test/streaming/rate_limit/snapshot_amplification.slt b/e2e_test/streaming/rate_limit/snapshot_amplification.slt index 231bc9b0eb94..f704bc637c02 100644 --- a/e2e_test/streaming/rate_limit/snapshot_amplification.slt +++ b/e2e_test/streaming/rate_limit/snapshot_amplification.slt @@ -17,6 +17,9 @@ INSERT INTO table select 1 from generate_series(1, 100000); statement ok flush; +statement ok +SET BACKGROUND_DDL=true; + statement ok CREATE SINK sink AS SELECT x.i1 as i1 FROM table x @@ -24,6 +27,9 @@ CREATE SINK sink AS JOIN table s2 ON x.i1 = s2.i1 WITH (connector = 'blackhole'); +statement ok +SET BACKGROUND_DDL=false; + # Let sink amplify... skipif in-memory sleep 1s diff --git a/e2e_test/streaming/rate_limit/upstream_amplification.slt b/e2e_test/streaming/rate_limit/upstream_amplification.slt index 71be801a78fc..63528472050a 100644 --- a/e2e_test/streaming/rate_limit/upstream_amplification.slt +++ b/e2e_test/streaming/rate_limit/upstream_amplification.slt @@ -18,6 +18,9 @@ WITH ( datagen.rows.per.second = '10000' ) FORMAT PLAIN ENCODE JSON; +statement ok +SET BACKGROUND_DDL=true; + statement ok CREATE SINK sink AS SELECT x.i1 as i1 FROM source_table x @@ -26,6 +29,9 @@ CREATE SINK sink AS JOIN source_table s3 ON x.i1 = s3.i1 WITH (connector = 'blackhole'); +statement ok +SET BACKGROUND_DDL=false; + # The following sequence of FLUSH should be fast, since barrier should be able to bypass sink. # Otherwise, these FLUSH will take a long time to complete, and trigger timeout. statement ok diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index ed17ebd1504e..0fb466e5a174 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -107,7 +107,7 @@ impl SinkDesc { target_table: self.target_table, created_at_cluster_version: None, initialized_at_cluster_version: None, - create_type: CreateType::Foreground, + create_type: self.create_type, } } diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 36245fcf7518..73157f91c2d0 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -23,6 +23,7 @@ use risingwave_common::catalog::TableId; use risingwave_common::hash::ActorMapping; use risingwave_connector::source::SplitImpl; use risingwave_hummock_sdk::HummockEpoch; +use risingwave_pb::catalog::CreateType; use risingwave_pb::meta::table_fragments::PbActorStatus; use risingwave_pb::meta::PausedReason; use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; @@ -164,6 +165,7 @@ pub enum Command { init_split_assignment: SplitAssignment, definition: String, ddl_type: DdlType, + create_type: CreateType, replace_table: Option, }, /// `CancelStreamingJob` command generates a `Stop` barrier including the actors of the given diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index a33380ea1cdd..746a263b0631 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::Epoch; +use risingwave_pb::catalog::CreateType; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; @@ -430,48 +431,56 @@ impl CreateMviewProgressTracker { return Some(TrackingJob::New(command)); } - let (creating_mv_id, upstream_mv_count, upstream_total_key_count, definition, ddl_type) = - if let Command::CreateStreamingJob { - table_fragments, - dispatchers, - upstream_root_actors, - definition, - ddl_type, - .. - } = &command.context.command - { - // Keep track of how many times each upstream MV appears. - let mut upstream_mv_count = HashMap::new(); - for (table_id, actors) in upstream_root_actors { - assert!(!actors.is_empty()); - let dispatch_count: usize = dispatchers - .iter() - .filter(|(upstream_actor_id, _)| actors.contains(upstream_actor_id)) - .map(|(_, v)| v.len()) - .sum(); - upstream_mv_count.insert(*table_id, dispatch_count / actors.len()); - } - - let upstream_total_key_count: u64 = upstream_mv_count + let ( + creating_mv_id, + upstream_mv_count, + upstream_total_key_count, + definition, + ddl_type, + create_type, + ) = if let Command::CreateStreamingJob { + table_fragments, + dispatchers, + upstream_root_actors, + definition, + ddl_type, + create_type, + .. + } = &command.context.command + { + // Keep track of how many times each upstream MV appears. + let mut upstream_mv_count = HashMap::new(); + for (table_id, actors) in upstream_root_actors { + assert!(!actors.is_empty()); + let dispatch_count: usize = dispatchers .iter() - .map(|(upstream_mv, count)| { - *count as u64 - * version_stats - .table_stats - .get(&upstream_mv.table_id) - .map_or(0, |stat| stat.total_key_count as u64) - }) + .filter(|(upstream_actor_id, _)| actors.contains(upstream_actor_id)) + .map(|(_, v)| v.len()) .sum(); - ( - table_fragments.table_id(), - upstream_mv_count, - upstream_total_key_count, - definition.to_string(), - ddl_type, - ) - } else { - unreachable!("Must be CreateStreamingJob."); - }; + upstream_mv_count.insert(*table_id, dispatch_count / actors.len()); + } + + let upstream_total_key_count: u64 = upstream_mv_count + .iter() + .map(|(upstream_mv, count)| { + *count as u64 + * version_stats + .table_stats + .get(&upstream_mv.table_id) + .map_or(0, |stat| stat.total_key_count as u64) + }) + .sum(); + ( + table_fragments.table_id(), + upstream_mv_count, + upstream_total_key_count, + definition.to_string(), + ddl_type, + create_type, + ) + } else { + unreachable!("Must be CreateStreamingJob."); + }; for &actor in &actors { self.actor_map.insert(actor, creating_mv_id); @@ -483,7 +492,7 @@ impl CreateMviewProgressTracker { upstream_total_key_count, definition, ); - if *ddl_type == DdlType::Sink { + if *ddl_type == DdlType::Sink && *create_type == CreateType::Background { // We return the original tracking job immediately. // This is because sink can be decoupled with backfill progress. // We don't need to wait for sink to finish backfill. diff --git a/src/meta/src/manager/streaming_job.rs b/src/meta/src/manager/streaming_job.rs index e042a222823f..a29e6e923de2 100644 --- a/src/meta/src/manager/streaming_job.rs +++ b/src/meta/src/manager/streaming_job.rs @@ -290,6 +290,7 @@ impl StreamingJob { Self::MaterializedView(table) => { table.get_create_type().unwrap_or(CreateType::Foreground) } + Self::Sink(s, _) => s.get_create_type().unwrap_or(CreateType::Foreground), _ => CreateType::Foreground, } } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index b2207e1b02fa..94b97abcdc00 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -461,6 +461,7 @@ impl GlobalStreamManager { definition: definition.to_string(), ddl_type, replace_table: replace_table_command, + create_type, }; tracing::debug!("sending Command::CreateStreamingJob"); if let Err(err) = self.barrier_scheduler.run_command(command).await { diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 38ce77c6ed79..93dc9ebb88a7 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -440,7 +440,7 @@ async fn test_foreground_index_cancel() -> Result<()> { } #[tokio::test] -async fn test_sink_create() -> Result<()> { +async fn test_background_sink_create() -> Result<()> { init_logger(); let mut cluster = Cluster::start(Configuration::for_background_ddl()).await?; let mut session = cluster.start_session(); @@ -450,6 +450,7 @@ async fn test_sink_create() -> Result<()> { let mut session2 = cluster.start_session(); tokio::spawn(async move { + session2.run(SET_BACKGROUND_DDL).await.unwrap(); session2.run(SET_RATE_LIMIT_2).await.unwrap(); session2 .run("CREATE SINK s FROM t WITH (connector='blackhole');") From 5c4a963c5ea295c6988492f4b1f03e6844c85177 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Wed, 17 Apr 2024 11:44:58 +0800 Subject: [PATCH 42/64] fix(object store): azure storage should check token expire (#16320) --- Cargo.lock | 2 +- Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index d86418775220..16350db6b46e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8823,7 +8823,7 @@ dependencies = [ [[package]] name = "reqsign" version = "0.14.9" -source = "git+https://github.com/wcy-fdu/reqsign.git?rev=e6cb304#e6cb3048581033275f4525431b757c8c612d37db" +source = "git+https://github.com/wcy-fdu/reqsign.git?rev=002ee2a#002ee2a41749b08bb5336f344e31f514d8fce718" dependencies = [ "anyhow", "async-trait", diff --git a/Cargo.toml b/Cargo.toml index 71f560623002..e6968b90a024 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -314,7 +314,7 @@ futures-timer = { git = "https://github.com/madsim-rs/futures-timer.git", rev = # patch: unlimit 4MB message size for grpc client etcd-client = { git = "https://github.com/risingwavelabs/etcd-client.git", rev = "4e84d40" } # todo(wcy-fdu): remove this patch fork after opendal release a new version to apply azure workload identity change. -reqsign = { git = "https://github.com/wcy-fdu/reqsign.git", rev = "e6cb304" } +reqsign = { git = "https://github.com/wcy-fdu/reqsign.git", rev = "002ee2a" } [workspace.metadata.dylint] libraries = [{ path = "./lints" }] From 9ab904f940e63d975e6ee6b65f23b2265dd71b9a Mon Sep 17 00:00:00 2001 From: August Date: Wed, 17 Apr 2024 12:40:09 +0800 Subject: [PATCH 43/64] chore: use v1.8.1 for sqlite docker compose file by default (#16350) --- docker/docker-compose-with-sqlite.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/docker-compose-with-sqlite.yml b/docker/docker-compose-with-sqlite.yml index b66f96e2d93c..c303a09b2b4f 100644 --- a/docker/docker-compose-with-sqlite.yml +++ b/docker/docker-compose-with-sqlite.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:nightly-20240328} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.1} services: risingwave-standalone: <<: *image From d2c2885783cc9ad1c35099ad83a6a458db90843b Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Wed, 17 Apr 2024 15:31:29 +0800 Subject: [PATCH 44/64] feat(storage): allow minio to use aws sdk or opendal (#15208) --- src/object_store/src/object/mod.rs | 21 ++++++-- .../opendal_engine/opendal_object_store.rs | 3 ++ .../src/object/opendal_engine/opendal_s3.rs | 50 +++++++++++++++++++ src/object_store/src/object/s3.rs | 1 + 4 files changed, 70 insertions(+), 5 deletions(-) diff --git a/src/object_store/src/object/mod.rs b/src/object_store/src/object/mod.rs index d122a5bb37fe..a623e4b116fd 100644 --- a/src/object_store/src/object/mod.rs +++ b/src/object_store/src/object/mod.rs @@ -888,11 +888,22 @@ pub async fn build_remote_object_store( set your endpoint to the environment variable RW_S3_ENDPOINT."); panic!("Passing s3-compatible is not supported, please modify the environment variable and pass in s3."); } - minio if minio.starts_with("minio://") => ObjectStoreImpl::S3( - S3ObjectStore::with_minio(minio, metrics.clone(), config.clone()) - .await - .monitored(metrics, config), - ), + minio if minio.starts_with("minio://") => { + if config.s3.developer.use_opendal { + tracing::info!("Using OpenDAL to access minio."); + ObjectStoreImpl::Opendal( + OpendalObjectStore::with_minio(minio, config.clone()) + .unwrap() + .monitored(metrics, config), + ) + } else { + ObjectStoreImpl::S3( + S3ObjectStore::with_minio(minio, metrics.clone(), config.clone()) + .await + .monitored(metrics, config), + ) + } + } "memory" => { if ident == "Meta Backup" { tracing::warn!("You're using in-memory remote object store for {}. This is not recommended for production environment.", ident); diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index d50422f015c7..2aa4bd458806 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -38,6 +38,7 @@ pub enum EngineType { Memory, Hdfs, Gcs, + Minio, S3, Obs, Oss, @@ -64,6 +65,7 @@ impl ObjectStore for OpendalObjectStore { fn get_object_prefix(&self, obj_id: u64) -> String { match self.engine_type { EngineType::S3 => prefix::s3::get_object_prefix(obj_id), + EngineType::Minio => prefix::s3::get_object_prefix(obj_id), EngineType::Memory => String::default(), EngineType::Hdfs => String::default(), EngineType::Gcs => String::default(), @@ -201,6 +203,7 @@ impl ObjectStore for OpendalObjectStore { match self.engine_type { EngineType::Memory => "Memory", EngineType::Hdfs => "Hdfs", + EngineType::Minio => "Minio", EngineType::S3 => "S3", EngineType::Gcs => "Gcs", EngineType::Obs => "Obs", diff --git a/src/object_store/src/object/opendal_engine/opendal_s3.rs b/src/object_store/src/object/opendal_engine/opendal_s3.rs index db2c7732d8fb..28f90a48e9ae 100644 --- a/src/object_store/src/object/opendal_engine/opendal_s3.rs +++ b/src/object_store/src/object/opendal_engine/opendal_s3.rs @@ -65,6 +65,56 @@ impl OpendalObjectStore { }) } + /// Creates a minio client. The server should be like `minio://key:secret@address:port/bucket`. + pub fn with_minio(server: &str, object_store_config: ObjectStoreConfig) -> ObjectResult { + let server = server.strip_prefix("minio://").unwrap(); + let (access_key_id, rest) = server.split_once(':').unwrap(); + let (secret_access_key, mut rest) = rest.split_once('@').unwrap(); + + let endpoint_prefix = if let Some(rest_stripped) = rest.strip_prefix("https://") { + rest = rest_stripped; + "https://" + } else if let Some(rest_stripped) = rest.strip_prefix("http://") { + rest = rest_stripped; + "http://" + } else { + "http://" + }; + let (address, bucket) = rest.split_once('/').unwrap(); + + let mut builder = S3::default(); + builder + .bucket(bucket) + .region("custom") + .access_key_id(access_key_id) + .secret_access_key(secret_access_key) + .endpoint(&format!("{}{}", endpoint_prefix, address)); + + builder.disable_config_load(); + let http_client = Self::new_http_client(&object_store_config)?; + builder.http_client(http_client); + let op: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .layer( + RetryLayer::new() + .with_min_delay(Duration::from_millis( + object_store_config.s3.object_store_req_retry_interval_ms, + )) + .with_max_delay(Duration::from_millis( + object_store_config.s3.object_store_req_retry_max_delay_ms, + )) + .with_max_times(object_store_config.s3.object_store_req_retry_max_attempts) + .with_factor(1.0) + .with_jitter(), + ) + .finish(); + + Ok(Self { + op, + engine_type: EngineType::Minio, + }) + } + pub fn new_http_client(config: &ObjectStoreConfig) -> ObjectResult { let mut client_builder = reqwest::ClientBuilder::new(); diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 3c270f5c1780..9d48f1175976 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -658,6 +658,7 @@ impl S3ObjectStore { let server = server.strip_prefix("minio://").unwrap(); let (access_key_id, rest) = server.split_once(':').unwrap(); let (secret_access_key, mut rest) = rest.split_once('@').unwrap(); + let endpoint_prefix = if let Some(rest_stripped) = rest.strip_prefix("https://") { rest = rest_stripped; "https://" From 142fdf68f07c9b5212c63500b33755d6f30852ad Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 17 Apr 2024 15:54:29 +0800 Subject: [PATCH 45/64] chore(scripts): update install script (#16355) --- scripts/install/install-risingwave.sh | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/scripts/install/install-risingwave.sh b/scripts/install/install-risingwave.sh index 4d72c2895379..e2095593d597 100755 --- a/scripts/install/install-risingwave.sh +++ b/scripts/install/install-risingwave.sh @@ -9,12 +9,9 @@ fi STATE_STORE_PATH="${HOME}/.risingwave/state_store" META_STORE_PATH="${HOME}/.risingwave/meta_store" -VERSION="v1.7.0-standalone" -# TODO(kwannoel): re-enable it once we have stable release in latest for single node mode. -#VERSION=$(curl -s https://api.github.com/repos/risingwavelabs/risingwave/releases/latest \ -# | grep '.tag_name' \ -# | sed -E -n 's/.*(v[0-9]+.[0-9]+.[0-9])\",/\1/p') -HOMEBREW_VERSION="1.7-standalone" +VERSION=$(curl -s https://api.github.com/repos/risingwavelabs/risingwave/releases/latest \ + | grep '.tag_name' \ + | sed -E -n 's/.*(v[0-9]+.[0-9]+.[0-9])\",/\1/p') BASE_URL="https://github.com/risingwavelabs/risingwave/releases/download" @@ -51,11 +48,11 @@ fi ############# BREW INSTALL if [ "${USE_BREW}" -eq 1 ]; then - echo "Installing RisingWave@${HOMEBREW_VERSION} using Homebrew." + echo "Installing RisingWave@${VERSION} using Homebrew." brew tap risingwavelabs/risingwave - brew install risingwave@${HOMEBREW_VERSION} + brew install risingwave echo - echo "Successfully installed RisingWave@${HOMEBREW_VERSION} using Homebrew." + echo "Successfully installed RisingWave@${VERSION} using Homebrew." echo echo "Run RisingWave:" echo @@ -107,5 +104,4 @@ if [ -z "${JAVA_HOME}" ]; then tput setaf 3 echo "WARNING: Java is required to use RisingWave's Java Connectors (e.g. MySQL)." echo "Please install Java, and set the \$JAVA_HOME environment variable." -fi -# TODO(kwannoel): Include link to our docs. \ No newline at end of file +fi \ No newline at end of file From adcdd8cc81f8343cdea5b3a13bfbb4eeecf04c19 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 17 Apr 2024 18:51:21 +0800 Subject: [PATCH 46/64] feat(meta): collect errors from all control streams (#16322) --- src/meta/src/barrier/mod.rs | 14 ++------ src/meta/src/barrier/recovery.rs | 17 +++------- src/meta/src/barrier/rpc.rs | 58 +++++++++++++++++++++++++------- 3 files changed, 53 insertions(+), 36 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index c99940fbc32f..0b570d9b2aaa 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -40,9 +40,7 @@ use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::PausedReason; use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; -use risingwave_pb::stream_service::{ - streaming_control_stream_response, BarrierCompleteResponse, StreamingControlStreamResponse, -}; +use risingwave_pb::stream_service::BarrierCompleteResponse; use thiserror_ext::AsReport; use tokio::sync::oneshot::{Receiver, Sender}; use tokio::sync::Mutex; @@ -670,16 +668,10 @@ impl GlobalBarrierManager { _ => {} } } - resp_result = self.control_stream_manager.next_response() => { + resp_result = self.control_stream_manager.next_complete_barrier_response() => { match resp_result { Ok((worker_id, prev_epoch, resp)) => { - let resp: StreamingControlStreamResponse = resp; - match resp.response { - Some(streaming_control_stream_response::Response::CompleteBarrier(resp)) => { - self.checkpoint_control.barrier_collected(worker_id, prev_epoch, resp); - }, - resp => unreachable!("invalid response: {:?}", resp), - } + self.checkpoint_control.barrier_collected(worker_id, prev_epoch, resp); } Err(e) => { diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 251ec401fa2f..f2ec59c3d6f4 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; use std::collections::{BTreeSet, HashMap, HashSet}; use std::sync::Arc; use std::time::{Duration, Instant}; @@ -29,9 +28,6 @@ use risingwave_pb::meta::{PausedReason, Recovery}; use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::AddMutation; -use risingwave_pb::stream_service::{ - streaming_control_stream_response, StreamingControlStreamResponse, -}; use thiserror_ext::AsReport; use tokio::sync::oneshot; use tokio_retry::strategy::{jitter, ExponentialBackoff}; @@ -502,15 +498,10 @@ impl GlobalBarrierManager { let mut node_to_collect = control_stream_manager.inject_barrier(command_ctx.clone())?; while !node_to_collect.is_empty() { - let (worker_id, _, resp) = control_stream_manager.next_response().await?; - assert_matches!( - resp, - StreamingControlStreamResponse { - response: Some( - streaming_control_stream_response::Response::CompleteBarrier(_) - ) - } - ); + let (worker_id, prev_epoch, _) = control_stream_manager + .next_complete_barrier_response() + .await?; + assert_eq!(prev_epoch, command_ctx.prev_epoch.value().0); assert!(node_to_collect.remove(&worker_id)); } diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index a098627afcd0..b7ea512ffbbf 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{HashMap, HashSet, VecDeque}; +use std::error::Error; use std::future::Future; use std::sync::Arc; use std::time::Duration; @@ -28,7 +29,7 @@ use risingwave_common::util::tracing::TracingContext; use risingwave_pb::common::{ActorInfo, WorkerNode}; use risingwave_pb::stream_plan::{Barrier, BarrierMutation, StreamActor}; use risingwave_pb::stream_service::{ - streaming_control_stream_request, streaming_control_stream_response, + streaming_control_stream_request, streaming_control_stream_response, BarrierCompleteResponse, BroadcastActorInfoTableRequest, BuildActorsRequest, DropActorsRequest, InjectBarrierRequest, StreamingControlStreamRequest, StreamingControlStreamResponse, UpdateActorsRequest, }; @@ -47,6 +48,8 @@ use super::GlobalBarrierManagerContext; use crate::manager::{MetaSrvEnv, WorkerId}; use crate::{MetaError, MetaResult}; +const COLLECT_ERROR_TIMEOUT: Duration = Duration::from_secs(3); + struct ControlStreamNode { worker: WorkerNode, sender: UnboundedSender, @@ -162,17 +165,25 @@ impl ControlStreamManager { Ok(()) } - pub(super) async fn next_response( + async fn next_response( + &mut self, + ) -> Option<(WorkerId, MetaResult)> { + let (worker_id, response_stream, result) = self.response_streams.next().await?; + if result.is_ok() { + self.response_streams + .push(into_future(worker_id, response_stream)); + } + Some((worker_id, result)) + } + + pub(super) async fn next_complete_barrier_response( &mut self, - ) -> MetaResult<(WorkerId, u64, StreamingControlStreamResponse)> { + ) -> MetaResult<(WorkerId, u64, BarrierCompleteResponse)> { loop { - let (worker_id, response_stream, result) = - pending_on_none(self.response_streams.next()).await; + let (worker_id, result) = pending_on_none(self.next_response()).await; match result { - Ok(resp) => match &resp.response { - Some(streaming_control_stream_response::Response::CompleteBarrier(_)) => { - self.response_streams - .push(into_future(worker_id, response_stream)); + Ok(resp) => match resp.response { + Some(streaming_control_stream_response::Response::CompleteBarrier(resp)) => { let node = self .nodes .get_mut(&worker_id) @@ -195,16 +206,39 @@ impl ControlStreamManager { // Note: No need to use `?` as the backtrace is from meta and not useful. warn!(node = ?node.worker, err = %err.as_report(), "get error from response stream"); if let Some(command) = node.inflight_barriers.pop_front() { + let errors = self.collect_errors(node.worker.id, err).await; + let err = merge_node_rpc_errors("get error from control stream", errors); self.context.report_collect_failure(&command, &err); break Err(err); } else { // for node with no inflight barrier, simply ignore the error + info!(node = ?node.worker, "no inflight barrier no node. Ignore error"); continue; } } } } } + + async fn collect_errors( + &mut self, + worker_id: WorkerId, + first_err: MetaError, + ) -> Vec<(WorkerId, MetaError)> { + let mut errors = vec![(worker_id, first_err)]; + #[cfg(not(madsim))] + { + let _ = timeout(COLLECT_ERROR_TIMEOUT, async { + while let Some((worker_id, result)) = self.next_response().await { + if let Err(e) = result { + errors.push((worker_id, e)); + } + } + }) + .await; + } + errors + } } impl ControlStreamManager { @@ -356,7 +390,7 @@ impl StreamRpcManager { let client = pool.get(node).await.map_err(|e| (node.id, e))?; f(client, input).await.map_err(|e| (node.id, e)) }); - let result = try_join_all_with_error_timeout(iters, Duration::from_secs(3)).await; + let result = try_join_all_with_error_timeout(iters, COLLECT_ERROR_TIMEOUT).await; result.map_err(|results_err| merge_node_rpc_errors("merged RPC Error", results_err)) } @@ -491,9 +525,9 @@ where Err(results_err) } -fn merge_node_rpc_errors( +fn merge_node_rpc_errors( message: &str, - errors: impl IntoIterator, + errors: impl IntoIterator, ) -> MetaError { use std::fmt::Write; From 02a3985548bfa41cf8d7aa93235c2be76efe0233 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Wed, 17 Apr 2024 21:24:50 +0800 Subject: [PATCH 47/64] feat: support `INET_ATON` and `INET_NTOA` (#16358) --- clippy.toml | 3 +- proto/expr.proto | 2 + src/expr/impl/src/scalar/inet.rs | 134 ++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/frontend/src/binder/expr/function.rs | 2 + src/frontend/src/expr/pure.rs | 451 +++++++++--------- .../src/optimizer/plan_expr_visitor/strong.rs | 4 +- 7 files changed, 371 insertions(+), 226 deletions(-) create mode 100644 src/expr/impl/src/scalar/inet.rs diff --git a/clippy.toml b/clippy.toml index bbda73f32259..551de0eb6c47 100644 --- a/clippy.toml +++ b/clippy.toml @@ -33,7 +33,8 @@ doc-valid-idents = [ "PostgreSQL", "MySQL", "TopN", - "VNode" + "VNode", + "IPv4", ] avoid-breaking-exported-api = false upper-case-acronyms-aggressive = true diff --git a/proto/expr.proto b/proto/expr.proto index 4c55ee2b614b..48fe88d1ff49 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -192,6 +192,8 @@ message ExprNode { CONVERT_TO = 323; DECRYPT = 324; ENCRYPT = 325; + INET_ATON = 328; + INET_NTOA = 329; // Unary operators NEG = 401; diff --git a/src/expr/impl/src/scalar/inet.rs b/src/expr/impl/src/scalar/inet.rs new file mode 100644 index 000000000000..71be67c1f3b3 --- /dev/null +++ b/src/expr/impl/src/scalar/inet.rs @@ -0,0 +1,134 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_expr::{function, ExprError, Result}; + +/// Given the dotted-quad representation of an IPv4 network address as a string, +/// returns an integer that represents the numeric value of the address +/// in network byte order (big endian). The returning value is a BIGINT (8-byte integer) +/// because PG doesn't support unsigned 32-bit integer. +/// +/// Short-form IP addresses (such as '127.1' as a representation of '127.0.0.1') +/// are NOT supported. +/// +/// This function is ported from MySQL. +/// Ref: . +/// +/// # Example +/// +/// ```slt +/// query I +/// select inet_aton('10.0.5.9'); +/// ---- +/// 167773449 +/// ``` +#[function("inet_aton(varchar) -> int8")] +pub fn inet_aton(str: &str) -> Result { + let mut parts = str.split('.'); + let mut result = 0; + for _ in 0..4 { + let part = parts.next().ok_or(ExprError::InvalidParam { + name: "str", + reason: format!("Invalid IP address: {}", &str).into(), + })?; + let part = part.parse::().map_err(|_| ExprError::InvalidParam { + name: "str", + reason: format!("Invalid IP address: {}", &str).into(), + })?; + result = (result << 8) | part as i64; + } + Ok(result) +} + +/// Given a numeric IPv4 network address in network byte order (big endian), +/// returns the dotted-quad string representation of the address as a string. +/// +/// This function is ported from MySQL. +/// Ref: . +/// +/// # Example +/// +/// ```slt +/// query T +/// select inet_ntoa(167773449); +/// ---- +/// 10.0.5.9 +/// ``` +#[function("inet_ntoa(int8) -> varchar")] +pub fn inet_ntoa(mut num: i64) -> Result> { + if (num > u32::MAX as i64) || (num < 0) { + return Err(ExprError::InvalidParam { + name: "num", + reason: format!("Invalid IP number: {}", num).into(), + }); + } + let mut parts = [0u8, 0, 0, 0]; + for i in (0..4).rev() { + parts[i] = (num & 0xFF) as u8; + num >>= 8; + } + let str = parts + .iter() + .map(|&x| x.to_string()) + .collect::>() + .join("."); + Ok(str.into_boxed_str()) +} + +#[cfg(test)] +mod tests { + use std::assert_matches::assert_matches; + + use super::*; + + #[test] + fn test_inet_aton() { + assert_eq!(inet_aton("10.0.5.9").unwrap(), 167773449); + assert_eq!(inet_aton("203.117.31.34").unwrap(), 3413450530); + + if let ExprError::InvalidParam { name, reason } = inet_aton("127.1").unwrap_err() { + assert_eq!(name, "str"); + assert_eq!(reason, "Invalid IP address: 127.1".into()); + } else { + panic!("Expected InvalidParam error"); + } + + assert_matches!(inet_aton("127.0.1"), Err(ExprError::InvalidParam { .. })); + assert_matches!(inet_aton("1.0.0.256"), Err(ExprError::InvalidParam { .. })); + assert_matches!(inet_aton("1.0.0.-1"), Err(ExprError::InvalidParam { .. })); + } + + #[test] + fn test_inet_ntoa() { + assert_eq!(inet_ntoa(167773449).unwrap(), "10.0.5.9".into()); + assert_eq!(inet_ntoa(3413450530).unwrap(), "203.117.31.34".into()); + assert_eq!(inet_ntoa(0).unwrap(), "0.0.0.0".into()); + assert_eq!( + inet_ntoa(u32::MAX as i64).unwrap(), + "255.255.255.255".into() + ); + + if let ExprError::InvalidParam { name, reason } = inet_ntoa(-1).unwrap_err() { + assert_eq!(name, "num"); + assert_eq!(reason, "Invalid IP number: -1".into()); + } else { + panic!("Expected InvalidParam error"); + } + + assert_matches!( + inet_ntoa(u32::MAX as i64 + 1), + Err(ExprError::InvalidParam { .. }) + ); + } +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index d2f528ece0c6..edbaaf4de01a 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -82,6 +82,7 @@ mod vnode; pub use to_jsonb::*; mod encrypt; mod external; +mod inet; mod to_timestamp; mod translate; mod trigonometric; diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 1b36e9ee2fd7..2134e08fe8c6 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -1026,6 +1026,8 @@ impl Binder { ("decrypt", raw_call(ExprType::Decrypt)), ("left", raw_call(ExprType::Left)), ("right", raw_call(ExprType::Right)), + ("inet_aton", raw_call(ExprType::InetAton)), + ("inet_ntoa", raw_call(ExprType::InetNtoa)), ("int8send", raw_call(ExprType::PgwireSend)), ("int8recv", guard_by_len(1, raw(|_binder, mut inputs| { // Similar to `cast` from string, return type is set explicitly rather than inferred. diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 310e7a9187da..bc18959e5be5 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use expr_node::Type; use risingwave_pb::expr::expr_node; use super::{ExprImpl, ExprVisitor}; @@ -33,216 +34,218 @@ impl ExprVisitor for ImpureAnalyzer { fn visit_function_call(&mut self, func_call: &super::FunctionCall) { match func_call.func_type() { - expr_node::Type::Unspecified => unreachable!(), - expr_node::Type::Add - | expr_node::Type::Subtract - | expr_node::Type::Multiply - | expr_node::Type::Divide - | expr_node::Type::Modulus - | expr_node::Type::Equal - | expr_node::Type::NotEqual - | expr_node::Type::LessThan - | expr_node::Type::LessThanOrEqual - | expr_node::Type::GreaterThan - | expr_node::Type::GreaterThanOrEqual - | expr_node::Type::And - | expr_node::Type::Or - | expr_node::Type::Not - | expr_node::Type::In - | expr_node::Type::Some - | expr_node::Type::All - | expr_node::Type::BitwiseAnd - | expr_node::Type::BitwiseOr - | expr_node::Type::BitwiseXor - | expr_node::Type::BitwiseNot - | expr_node::Type::BitwiseShiftLeft - | expr_node::Type::BitwiseShiftRight - | expr_node::Type::Extract - | expr_node::Type::DatePart - | expr_node::Type::TumbleStart - | expr_node::Type::SecToTimestamptz - | expr_node::Type::AtTimeZone - | expr_node::Type::DateTrunc - | expr_node::Type::MakeDate - | expr_node::Type::MakeTime - | expr_node::Type::MakeTimestamp - | expr_node::Type::CharToTimestamptz - | expr_node::Type::CharToDate - | expr_node::Type::CastWithTimeZone - | expr_node::Type::AddWithTimeZone - | expr_node::Type::SubtractWithTimeZone - | expr_node::Type::Cast - | expr_node::Type::Substr - | expr_node::Type::Length - | expr_node::Type::Like - | expr_node::Type::ILike - | expr_node::Type::SimilarToEscape - | expr_node::Type::Upper - | expr_node::Type::Lower - | expr_node::Type::Trim - | expr_node::Type::Replace - | expr_node::Type::Position - | expr_node::Type::Ltrim - | expr_node::Type::Rtrim - | expr_node::Type::Case - | expr_node::Type::ConstantLookup - | expr_node::Type::RoundDigit - | expr_node::Type::Round - | expr_node::Type::Ascii - | expr_node::Type::Translate - | expr_node::Type::Coalesce - | expr_node::Type::ConcatWs - | expr_node::Type::ConcatWsVariadic - | expr_node::Type::Abs - | expr_node::Type::SplitPart - | expr_node::Type::Ceil - | expr_node::Type::Floor - | expr_node::Type::Trunc - | expr_node::Type::ToChar - | expr_node::Type::Md5 - | expr_node::Type::CharLength - | expr_node::Type::Repeat - | expr_node::Type::ConcatOp - | expr_node::Type::Concat - | expr_node::Type::ConcatVariadic - | expr_node::Type::BoolOut - | expr_node::Type::OctetLength - | expr_node::Type::BitLength - | expr_node::Type::Overlay - | expr_node::Type::RegexpMatch - | expr_node::Type::RegexpReplace - | expr_node::Type::RegexpCount - | expr_node::Type::RegexpSplitToArray - | expr_node::Type::RegexpEq - | expr_node::Type::Pow - | expr_node::Type::Exp - | expr_node::Type::Ln - | expr_node::Type::Log10 - | expr_node::Type::Chr - | expr_node::Type::StartsWith - | expr_node::Type::Initcap - | expr_node::Type::Lpad - | expr_node::Type::Rpad - | expr_node::Type::Reverse - | expr_node::Type::Strpos - | expr_node::Type::ToAscii - | expr_node::Type::ToHex - | expr_node::Type::QuoteIdent - | expr_node::Type::Sin - | expr_node::Type::Cos - | expr_node::Type::Tan - | expr_node::Type::Cot - | expr_node::Type::Asin - | expr_node::Type::Acos - | expr_node::Type::Atan - | expr_node::Type::Atan2 - | expr_node::Type::Sqrt - | expr_node::Type::Cbrt - | expr_node::Type::Sign - | expr_node::Type::Scale - | expr_node::Type::MinScale - | expr_node::Type::TrimScale - | expr_node::Type::Left - | expr_node::Type::Right - | expr_node::Type::Degrees - | expr_node::Type::Radians - | expr_node::Type::IsTrue - | expr_node::Type::IsNotTrue - | expr_node::Type::IsFalse - | expr_node::Type::IsNotFalse - | expr_node::Type::IsNull - | expr_node::Type::IsNotNull - | expr_node::Type::IsDistinctFrom - | expr_node::Type::IsNotDistinctFrom - | expr_node::Type::Neg - | expr_node::Type::Field - | expr_node::Type::Array - | expr_node::Type::ArrayAccess - | expr_node::Type::ArrayRangeAccess - | expr_node::Type::Row - | expr_node::Type::ArrayToString - | expr_node::Type::ArrayCat - | expr_node::Type::ArrayMax - | expr_node::Type::ArraySum - | expr_node::Type::ArraySort - | expr_node::Type::ArrayAppend - | expr_node::Type::ArrayPrepend - | expr_node::Type::FormatType - | expr_node::Type::ArrayDistinct - | expr_node::Type::ArrayMin - | expr_node::Type::ArrayDims - | expr_node::Type::ArrayLength - | expr_node::Type::Cardinality - | expr_node::Type::TrimArray - | expr_node::Type::ArrayRemove - | expr_node::Type::ArrayReplace - | expr_node::Type::ArrayPosition - | expr_node::Type::ArrayContains - | expr_node::Type::ArrayContained - | expr_node::Type::HexToInt256 - | expr_node::Type::JsonbConcat - | expr_node::Type::JsonbAccess - | expr_node::Type::JsonbAccessStr - | expr_node::Type::JsonbExtractPath - | expr_node::Type::JsonbExtractPathVariadic - | expr_node::Type::JsonbExtractPathText - | expr_node::Type::JsonbExtractPathTextVariadic - | expr_node::Type::JsonbTypeof - | expr_node::Type::JsonbArrayLength - | expr_node::Type::JsonbObject - | expr_node::Type::JsonbPretty - | expr_node::Type::JsonbDeletePath - | expr_node::Type::JsonbContains - | expr_node::Type::JsonbContained - | expr_node::Type::JsonbExists - | expr_node::Type::JsonbExistsAny - | expr_node::Type::JsonbExistsAll - | expr_node::Type::JsonbStripNulls - | expr_node::Type::JsonbBuildArray - | expr_node::Type::JsonbBuildArrayVariadic - | expr_node::Type::JsonbBuildObject - | expr_node::Type::JsonbPopulateRecord - | expr_node::Type::JsonbToRecord - | expr_node::Type::JsonbBuildObjectVariadic - | expr_node::Type::JsonbPathExists - | expr_node::Type::JsonbPathMatch - | expr_node::Type::JsonbPathQueryArray - | expr_node::Type::JsonbPathQueryFirst - | expr_node::Type::IsJson - | expr_node::Type::ToJsonb - | expr_node::Type::Sind - | expr_node::Type::Cosd - | expr_node::Type::Cotd - | expr_node::Type::Asind - | expr_node::Type::Sinh - | expr_node::Type::Cosh - | expr_node::Type::Coth - | expr_node::Type::Tanh - | expr_node::Type::Atanh - | expr_node::Type::Asinh - | expr_node::Type::Acosh - | expr_node::Type::Decode - | expr_node::Type::Encode - | expr_node::Type::Sha1 - | expr_node::Type::Sha224 - | expr_node::Type::Sha256 - | expr_node::Type::Sha384 - | expr_node::Type::Sha512 - | expr_node::Type::Decrypt - | expr_node::Type::Encrypt - | expr_node::Type::Tand - | expr_node::Type::ArrayPositions - | expr_node::Type::StringToArray - | expr_node::Type::Format - | expr_node::Type::FormatVariadic - | expr_node::Type::PgwireSend - | expr_node::Type::PgwireRecv - | expr_node::Type::ArrayTransform - | expr_node::Type::Greatest - | expr_node::Type::Least - | expr_node::Type::ConvertFrom - | expr_node::Type::ConvertTo - | expr_node::Type::IcebergTransform => + Type::Unspecified => unreachable!(), + Type::Add + | Type::Subtract + | Type::Multiply + | Type::Divide + | Type::Modulus + | Type::Equal + | Type::NotEqual + | Type::LessThan + | Type::LessThanOrEqual + | Type::GreaterThan + | Type::GreaterThanOrEqual + | Type::And + | Type::Or + | Type::Not + | Type::In + | Type::Some + | Type::All + | Type::BitwiseAnd + | Type::BitwiseOr + | Type::BitwiseXor + | Type::BitwiseNot + | Type::BitwiseShiftLeft + | Type::BitwiseShiftRight + | Type::Extract + | Type::DatePart + | Type::TumbleStart + | Type::SecToTimestamptz + | Type::AtTimeZone + | Type::DateTrunc + | Type::MakeDate + | Type::MakeTime + | Type::MakeTimestamp + | Type::CharToTimestamptz + | Type::CharToDate + | Type::CastWithTimeZone + | Type::AddWithTimeZone + | Type::SubtractWithTimeZone + | Type::Cast + | Type::Substr + | Type::Length + | Type::Like + | Type::ILike + | Type::SimilarToEscape + | Type::Upper + | Type::Lower + | Type::Trim + | Type::Replace + | Type::Position + | Type::Ltrim + | Type::Rtrim + | Type::Case + | Type::ConstantLookup + | Type::RoundDigit + | Type::Round + | Type::Ascii + | Type::Translate + | Type::Coalesce + | Type::ConcatWs + | Type::ConcatWsVariadic + | Type::Abs + | Type::SplitPart + | Type::Ceil + | Type::Floor + | Type::Trunc + | Type::ToChar + | Type::Md5 + | Type::CharLength + | Type::Repeat + | Type::ConcatOp + | Type::Concat + | Type::ConcatVariadic + | Type::BoolOut + | Type::OctetLength + | Type::BitLength + | Type::Overlay + | Type::RegexpMatch + | Type::RegexpReplace + | Type::RegexpCount + | Type::RegexpSplitToArray + | Type::RegexpEq + | Type::Pow + | Type::Exp + | Type::Ln + | Type::Log10 + | Type::Chr + | Type::StartsWith + | Type::Initcap + | Type::Lpad + | Type::Rpad + | Type::Reverse + | Type::Strpos + | Type::ToAscii + | Type::ToHex + | Type::QuoteIdent + | Type::Sin + | Type::Cos + | Type::Tan + | Type::Cot + | Type::Asin + | Type::Acos + | Type::Atan + | Type::Atan2 + | Type::Sqrt + | Type::Cbrt + | Type::Sign + | Type::Scale + | Type::MinScale + | Type::TrimScale + | Type::Left + | Type::Right + | Type::Degrees + | Type::Radians + | Type::IsTrue + | Type::IsNotTrue + | Type::IsFalse + | Type::IsNotFalse + | Type::IsNull + | Type::IsNotNull + | Type::IsDistinctFrom + | Type::IsNotDistinctFrom + | Type::Neg + | Type::Field + | Type::Array + | Type::ArrayAccess + | Type::ArrayRangeAccess + | Type::Row + | Type::ArrayToString + | Type::ArrayCat + | Type::ArrayMax + | Type::ArraySum + | Type::ArraySort + | Type::ArrayAppend + | Type::ArrayPrepend + | Type::FormatType + | Type::ArrayDistinct + | Type::ArrayMin + | Type::ArrayDims + | Type::ArrayLength + | Type::Cardinality + | Type::TrimArray + | Type::ArrayRemove + | Type::ArrayReplace + | Type::ArrayPosition + | Type::ArrayContains + | Type::ArrayContained + | Type::HexToInt256 + | Type::JsonbConcat + | Type::JsonbAccess + | Type::JsonbAccessStr + | Type::JsonbExtractPath + | Type::JsonbExtractPathVariadic + | Type::JsonbExtractPathText + | Type::JsonbExtractPathTextVariadic + | Type::JsonbTypeof + | Type::JsonbArrayLength + | Type::JsonbObject + | Type::JsonbPretty + | Type::JsonbDeletePath + | Type::JsonbContains + | Type::JsonbContained + | Type::JsonbExists + | Type::JsonbExistsAny + | Type::JsonbExistsAll + | Type::JsonbStripNulls + | Type::JsonbBuildArray + | Type::JsonbBuildArrayVariadic + | Type::JsonbBuildObject + | Type::JsonbPopulateRecord + | Type::JsonbToRecord + | Type::JsonbBuildObjectVariadic + | Type::JsonbPathExists + | Type::JsonbPathMatch + | Type::JsonbPathQueryArray + | Type::JsonbPathQueryFirst + | Type::IsJson + | Type::ToJsonb + | Type::Sind + | Type::Cosd + | Type::Cotd + | Type::Asind + | Type::Sinh + | Type::Cosh + | Type::Coth + | Type::Tanh + | Type::Atanh + | Type::Asinh + | Type::Acosh + | Type::Decode + | Type::Encode + | Type::Sha1 + | Type::Sha224 + | Type::Sha256 + | Type::Sha384 + | Type::Sha512 + | Type::Decrypt + | Type::Encrypt + | Type::Tand + | Type::ArrayPositions + | Type::StringToArray + | Type::Format + | Type::FormatVariadic + | Type::PgwireSend + | Type::PgwireRecv + | Type::ArrayTransform + | Type::Greatest + | Type::Least + | Type::ConvertFrom + | Type::ConvertTo + | Type::IcebergTransform + | Type::InetNtoa + | Type::InetAton => // expression output is deterministic(same result for the same input) { func_call @@ -251,20 +254,20 @@ impl ExprVisitor for ImpureAnalyzer { .for_each(|expr| self.visit_expr(expr)); } // expression output is not deterministic - expr_node::Type::Vnode - | expr_node::Type::Proctime - | expr_node::Type::PgSleep - | expr_node::Type::PgSleepFor - | expr_node::Type::PgSleepUntil - | expr_node::Type::CastRegclass - | expr_node::Type::PgGetIndexdef - | expr_node::Type::ColDescription - | expr_node::Type::PgGetViewdef - | expr_node::Type::PgGetUserbyid - | expr_node::Type::PgIndexesSize - | expr_node::Type::PgRelationSize - | expr_node::Type::PgGetSerialSequence - | expr_node::Type::MakeTimestamptz => self.impure = true, + Type::Vnode + | Type::Proctime + | Type::PgSleep + | Type::PgSleepFor + | Type::PgSleepUntil + | Type::CastRegclass + | Type::PgGetIndexdef + | Type::ColDescription + | Type::PgGetViewdef + | Type::PgGetUserbyid + | Type::PgIndexesSize + | Type::PgRelationSize + | Type::PgGetSerialSequence + | Type::MakeTimestamptz => self.impure = true, } } } diff --git a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs index a3d2e0edfb15..e30cdb0b6e31 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs @@ -300,7 +300,9 @@ impl Strong { | ExprType::PgIndexesSize | ExprType::PgRelationSize | ExprType::PgGetSerialSequence - | ExprType::IcebergTransform => false, + | ExprType::IcebergTransform + | ExprType::InetAton + | ExprType::InetNtoa => false, ExprType::Unspecified => unreachable!(), } } From 144986ed9d7ee522793eac654900289d120ce0d5 Mon Sep 17 00:00:00 2001 From: lmatz Date: Thu, 18 Apr 2024 09:38:49 +0800 Subject: [PATCH 48/64] chore: update docker image version to 1.8.1 (#16340) --- docker/docker-compose-distributed.yml | 2 +- docker/docker-compose-with-azblob.yml | 2 +- docker/docker-compose-with-gcs.yml | 2 +- docker/docker-compose-with-obs.yml | 2 +- docker/docker-compose-with-oss.yml | 2 +- docker/docker-compose-with-s3.yml | 2 +- docker/docker-compose.yml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/docker-compose-distributed.yml b/docker/docker-compose-distributed.yml index 75c09b0428b8..55cb1cbcffe3 100644 --- a/docker/docker-compose-distributed.yml +++ b/docker/docker-compose-distributed.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.1} services: compactor-0: <<: *image diff --git a/docker/docker-compose-with-azblob.yml b/docker/docker-compose-with-azblob.yml index 2fe5fa61a586..e0b44c576801 100644 --- a/docker/docker-compose-with-azblob.yml +++ b/docker/docker-compose-with-azblob.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.1} services: risingwave-standalone: <<: *image diff --git a/docker/docker-compose-with-gcs.yml b/docker/docker-compose-with-gcs.yml index 41000ffa00d3..847172c2d09c 100644 --- a/docker/docker-compose-with-gcs.yml +++ b/docker/docker-compose-with-gcs.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.1} services: risingwave-standalone: <<: *image diff --git a/docker/docker-compose-with-obs.yml b/docker/docker-compose-with-obs.yml index 487229a1adfb..5d0df0ca4f72 100644 --- a/docker/docker-compose-with-obs.yml +++ b/docker/docker-compose-with-obs.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.1} services: risingwave-standalone: <<: *image diff --git a/docker/docker-compose-with-oss.yml b/docker/docker-compose-with-oss.yml index 14d4ee79473e..7296a7074d5a 100644 --- a/docker/docker-compose-with-oss.yml +++ b/docker/docker-compose-with-oss.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.1} services: risingwave-standalone: <<: *image diff --git a/docker/docker-compose-with-s3.yml b/docker/docker-compose-with-s3.yml index 13b4b7807a1e..815489f82493 100644 --- a/docker/docker-compose-with-s3.yml +++ b/docker/docker-compose-with-s3.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.1} services: risingwave-standalone: <<: *image diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index 1d301bdba1d5..6259a5757b14 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -1,7 +1,7 @@ --- version: "3" x-image: &image - image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.0} + image: ${RW_IMAGE:-risingwavelabs/risingwave:v1.8.1} services: risingwave-standalone: <<: *image From ebf01043d9396765f81cfe7f32c6e430435bda1a Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 18 Apr 2024 11:09:11 +0800 Subject: [PATCH 49/64] feat(frontend): change `streaming_rate_limit=0` to pause stream instead of disable rate limit (#16333) --- e2e_test/udf/always_retry_python.slt | 2 +- src/common/src/session_config/mod.rs | 13 +++++++++---- src/frontend/src/utils/overwrite_options.rs | 11 +++++++---- .../integration_tests/recovery/background_ddl.rs | 2 +- 4 files changed, 18 insertions(+), 10 deletions(-) diff --git a/e2e_test/udf/always_retry_python.slt b/e2e_test/udf/always_retry_python.slt index 64477a7df6be..19e66dd60b07 100644 --- a/e2e_test/udf/always_retry_python.slt +++ b/e2e_test/udf/always_retry_python.slt @@ -56,7 +56,7 @@ SELECT count(*) > 0 FROM mv_no_retry where s1 is NULL; t statement ok -SET STREAMING_RATE_LIMIT=0; +SET STREAMING_RATE_LIMIT TO DEFAULT; statement ok SET BACKGROUND_DDL=false; diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index c8e5d4b0fcf9..4ee7617ee751 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -52,6 +52,10 @@ pub enum SessionConfigError { type SessionConfigResult = std::result::Result; +// NOTE(kwannoel): We declare it separately as a constant, +// otherwise seems like it can't infer the type of -1 when written inline. +const DISABLE_STREAMING_RATE_LIMIT: i32 = -1; + #[serde_as] /// This is the Session Config of RisingWave. #[derive(Clone, Debug, Deserialize, Serialize, SessionConfig, ConfigDoc, PartialEq)] @@ -241,10 +245,11 @@ pub struct SessionConfig { #[parameter(default = STANDARD_CONFORMING_STRINGS)] standard_conforming_strings: String, - /// Set streaming rate limit (rows per second) for each parallelism for mv backfilling - #[serde_as(as = "DisplayFromStr")] - #[parameter(default = ConfigNonZeroU64::default())] - streaming_rate_limit: ConfigNonZeroU64, + /// Set streaming rate limit (rows per second) for each parallelism for mv / source backfilling, source reads. + /// If set to -1, disable rate limit. + /// If set to 0, this pauses the snapshot read / source read. + #[parameter(default = DISABLE_STREAMING_RATE_LIMIT)] + streaming_rate_limit: i32, /// Cache policy for partition cache in streaming over window. /// Can be "full", "recent", "`recent_first_n`" or "`recent_last_n`". diff --git a/src/frontend/src/utils/overwrite_options.rs b/src/frontend/src/utils/overwrite_options.rs index 8d4ff2febfd2..a88a4c12d7fb 100644 --- a/src/frontend/src/utils/overwrite_options.rs +++ b/src/frontend/src/utils/overwrite_options.rs @@ -24,6 +24,7 @@ impl OverwriteOptions { pub fn new(args: &mut HandlerArgs) -> Self { let streaming_rate_limit = { + // CREATE MATERIALIZED VIEW m1 WITH (rate_limit = N) ... if let Some(x) = args .with_options .inner_mut() @@ -32,10 +33,12 @@ impl OverwriteOptions { // FIXME(tabVersion): validate the value Some(x.parse::().unwrap()) } else { - args.session - .config() - .streaming_rate_limit() - .map(|limit| limit.get() as u32) + let rate_limit = args.session.config().streaming_rate_limit(); + if rate_limit < 0 { + None + } else { + Some(rate_limit as u32) + } } }; Self { diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 93dc9ebb88a7..80ff76a3f102 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -30,7 +30,7 @@ const SEED_TABLE_100: &str = "INSERT INTO t SELECT generate_series FROM generate const SET_BACKGROUND_DDL: &str = "SET BACKGROUND_DDL=true;"; const SET_RATE_LIMIT_2: &str = "SET STREAMING_RATE_LIMIT=2;"; const SET_RATE_LIMIT_1: &str = "SET STREAMING_RATE_LIMIT=1;"; -const RESET_RATE_LIMIT: &str = "SET STREAMING_RATE_LIMIT=0;"; +const RESET_RATE_LIMIT: &str = "SET STREAMING_RATE_LIMIT=DEFAULT;"; const CREATE_MV1: &str = "CREATE MATERIALIZED VIEW mv1 as SELECT * FROM t;"; const DROP_MV1: &str = "DROP MATERIALIZED VIEW mv1;"; const WAIT: &str = "WAIT;"; From 4aed67f43eab44aa594f4a37fe2aa5461c4807ca Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 18 Apr 2024 11:41:35 +0800 Subject: [PATCH 50/64] fix(meta): only return for background sink once initial barrier collected (#16367) --- src/meta/src/rpc/ddl_controller.rs | 9 ++++++--- src/meta/src/rpc/ddl_controller_v2.rs | 9 ++++++--- src/tests/simulation/src/slt.rs | 11 +++++++++-- 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index c4a6e8c6ed62..78fb65089eb4 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -774,8 +774,11 @@ impl DdlController { } }; - match create_type { - CreateType::Foreground | CreateType::Unspecified => { + match (create_type, &stream_job) { + (CreateType::Foreground, _) + | (CreateType::Unspecified, _) + // FIXME(kwannoel): Unify background stream's creation path with MV below. + | (CreateType::Background, &StreamingJob::Sink(_, _)) => { self.create_streaming_job_inner( mgr, stream_job, @@ -785,7 +788,7 @@ impl DdlController { ) .await } - CreateType::Background => { + (CreateType::Background, _) => { let ctrl = self.clone(); let mgr = mgr.clone(); let stream_job_id = stream_job.id(); diff --git a/src/meta/src/rpc/ddl_controller_v2.rs b/src/meta/src/rpc/ddl_controller_v2.rs index c0ab2f13f2ff..3e948e88e282 100644 --- a/src/meta/src/rpc/ddl_controller_v2.rs +++ b/src/meta/src/rpc/ddl_controller_v2.rs @@ -204,8 +204,11 @@ impl DdlController { // create streaming jobs. let stream_job_id = streaming_job.id(); - match streaming_job.create_type() { - CreateType::Unspecified | CreateType::Foreground => { + match (streaming_job.create_type(), streaming_job) { + (CreateType::Unspecified, _) + | (CreateType::Foreground, _) + // FIXME(kwannoel): Unify background stream's creation path with MV below. + | (CreateType::Background, StreamingJob::Sink(_, _)) => { let replace_table_job_info = ctx.replace_table_job_info.as_ref().map( |(streaming_job, ctx, table_fragments)| { ( @@ -241,7 +244,7 @@ impl DdlController { Ok(version) } - CreateType::Background => { + (CreateType::Background, _) => { let ctrl = self.clone(); let mgr = mgr.clone(); let fut = async move { diff --git a/src/tests/simulation/src/slt.rs b/src/tests/simulation/src/slt.rs index 9e341c715190..ec1aca82d36c 100644 --- a/src/tests/simulation/src/slt.rs +++ b/src/tests/simulation/src/slt.rs @@ -223,6 +223,10 @@ pub async fn run_slt_task( // NOTE(kwannoel): For background ddl let mut background_ddl_enabled = false; + // If background ddl is set to true within the test case, prevent random setting of background_ddl to true. + // We can revert it back to false only if we encounter a record that sets background_ddl to false. + let mut manual_background_ddl_enabled = false; + for record in sqllogictest::parse_file(path).expect("failed to parse file") { // uncomment to print metrics for task counts // let metrics = madsim::runtime::Handle::current().metrics(); @@ -254,8 +258,10 @@ pub async fn run_slt_task( }; tracing::debug!(?cmd, "Running"); - if matches!(cmd, SqlCmd::SetBackgroundDdl { .. }) && background_ddl_rate > 0.0 { - panic!("We cannot run background_ddl statement with background_ddl_rate > 0.0, since it could be reset"); + if background_ddl_rate > 0.0 + && let SqlCmd::SetBackgroundDdl { enable } = cmd + { + manual_background_ddl_enabled = enable; } // For each background ddl compatible statement, provide a chance for background_ddl=true. @@ -266,6 +272,7 @@ pub async fn run_slt_task( .. } = &record && matches!(cmd, SqlCmd::CreateMaterializedView { .. }) + && !manual_background_ddl_enabled { let background_ddl_setting = rng.gen_bool(background_ddl_rate); let set_background_ddl = Record::Statement { From 57c5727610811f2a8d4c8fe93e3955687bb35f57 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Thu, 18 Apr 2024 13:02:59 +0800 Subject: [PATCH 51/64] feat(meta): use version object ids view to clean stale objects and deprecate gc_object_ids (#16309) --- proto/hummock.proto | 3 +- .../rw_catalog/rw_hummock_version_deltas.rs | 2 - src/meta/src/hummock/manager/checkpoint.rs | 58 +++++++++++++------ .../manager/compaction_group_manager.rs | 4 +- src/meta/src/hummock/manager/gc.rs | 8 +-- src/meta/src/hummock/manager/mod.rs | 12 ++-- src/meta/src/hummock/manager/tests.rs | 43 +++++++++++++- src/meta/src/hummock/vacuum.rs | 34 ++++------- .../compaction_group/hummock_version_ext.rs | 1 - src/storage/hummock_sdk/src/version.rs | 39 +++++++++++-- 10 files changed, 136 insertions(+), 68 deletions(-) diff --git a/proto/hummock.proto b/proto/hummock.proto index 50509160e2ef..4a3dac7ff6f4 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -168,7 +168,8 @@ message HummockVersionDelta { // Reads against such an epoch will fail. uint64 safe_epoch = 5; bool trivial_move = 6; - repeated uint64 gc_object_ids = 7; + reserved 7; + reserved "gc_object_ids"; map new_table_watermarks = 8; repeated uint32 removed_table_ids = 9; } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs index 052cb5fae170..a46312b817c8 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs @@ -27,7 +27,6 @@ struct RwHummockVersionDelta { max_committed_epoch: i64, safe_epoch: i64, trivial_move: bool, - gc_object_ids: JsonbVal, group_deltas: JsonbVal, } @@ -42,7 +41,6 @@ async fn read(reader: &SysCatalogReaderImpl) -> Result, } @@ -119,18 +126,29 @@ impl HummockManager { let timer = self.metrics.version_checkpoint_latency.start_timer(); // 1. hold read lock and create new checkpoint let versioning_guard = read_lock!(self, versioning).await; - let versioning = versioning_guard.deref(); - let current_version = &versioning.current_version; - let old_checkpoint = &versioning.checkpoint; + let versioning: &Versioning = versioning_guard.deref(); + let current_version: &HummockVersion = &versioning.current_version; + let old_checkpoint: &HummockVersionCheckpoint = &versioning.checkpoint; let new_checkpoint_id = current_version.id; let old_checkpoint_id = old_checkpoint.version.id; if new_checkpoint_id < old_checkpoint_id + min_delta_log_num { return Ok(0); } + if cfg!(test) && new_checkpoint_id == old_checkpoint_id { + drop(versioning_guard); + let mut versioning = write_lock!(self, versioning).await; + versioning.mark_objects_for_deletion(); + let min_pinned_version_id = versioning.min_pinned_version_id(); + trigger_gc_stat(&self.metrics, &versioning.checkpoint, min_pinned_version_id); + return Ok(0); + } + assert!(new_checkpoint_id > old_checkpoint_id); let mut archive: Option = None; let mut stale_objects = old_checkpoint.stale_objects.clone(); // `object_sizes` is used to calculate size of stale objects. let mut object_sizes = object_size_map(&old_checkpoint.version); + // The set of object ids that once exist in any hummock version + let mut versions_object_ids = old_checkpoint.version.get_object_ids(); for (_, version_delta) in versioning .hummock_version_deltas .range((Excluded(old_checkpoint_id), Included(new_checkpoint_id))) @@ -144,22 +162,24 @@ impl HummockManager { .map(|t| (t.object_id, t.file_size)), ); } - let removed_object_ids = version_delta.gc_object_ids.clone(); - if removed_object_ids.is_empty() { - continue; - } - let total_file_size = removed_object_ids - .iter() - .map(|t| object_sizes.get(t).copied().unwrap()) - .sum::(); - stale_objects.insert( - version_delta.id, - StaleObjects { - id: removed_object_ids, - total_file_size, - }, - ); + + versions_object_ids.extend(version_delta.newly_added_object_ids()); } + + // Object ids that once exist in any hummock version but not exist in the latest hummock version + let removed_object_ids = &versions_object_ids - ¤t_version.get_object_ids(); + + let total_file_size = removed_object_ids + .iter() + .map(|t| object_sizes.get(t).copied().unwrap()) + .sum::(); + stale_objects.insert( + current_version.id, + StaleObjects { + id: removed_object_ids.into_iter().collect(), + total_file_size, + }, + ); if self.env.opts.enable_hummock_data_archive { archive = Some(PbHummockVersionArchive { version: Some(old_checkpoint.version.to_protobuf()), @@ -189,7 +209,7 @@ impl HummockManager { // 3. hold write lock and update in memory state let mut versioning_guard = write_lock!(self, versioning).await; let versioning = versioning_guard.deref_mut(); - assert!(new_checkpoint.version.id >= versioning.checkpoint.version.id); + assert!(new_checkpoint.version.id > versioning.checkpoint.version.id); versioning.checkpoint = new_checkpoint; // Not delete stale objects when archive is enabled if !self.env.opts.enable_hummock_data_archive { diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index c99c1b8d492e..18bf9b263e73 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -325,9 +325,7 @@ impl HummockManager { // group is to be removed. // However, we need to take care of SST GC for the removed group. for (object_id, sst_id) in get_compaction_group_ssts(current_version, *group_id) { - if drop_sst(&mut branched_ssts, *group_id, object_id, sst_id) { - new_version_delta.gc_object_ids.push(object_id); - } + drop_sst(&mut branched_ssts, *group_id, object_id, sst_id); } let group_deltas = &mut new_version_delta .group_deltas diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index ac10c58d78cf..dafd3231afcc 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -121,14 +121,14 @@ impl HummockManager { let versioning_guard = read_lock!(self, versioning).await; let versioning: &Versioning = &versioning_guard; - // object ids in current version - let mut tracked_object_ids = versioning.current_version.get_object_ids(); - // add object ids removed between checkpoint version and current version + // object ids in checkpoint version + let mut tracked_object_ids = versioning.checkpoint.version.get_object_ids(); + // add object ids added between checkpoint version and current version for (_, delta) in versioning.hummock_version_deltas.range(( Excluded(versioning.checkpoint.version.id), Included(versioning.current_version.id), )) { - tracked_object_ids.extend(delta.gc_object_ids.iter().cloned()); + tracked_object_ids.extend(delta.newly_added_object_ids()); } // add stale object ids before the checkpoint version let min_pinned_version_id = versioning.min_pinned_version_id(); diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index b3031e24a7ce..0dc940e00658 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -1985,6 +1985,7 @@ impl HummockManager { pub async fn check_state_consistency(&self) { let mut compaction_guard = write_lock!(self, compaction).await; let mut versioning_guard = write_lock!(self, versioning).await; + let objects_to_delete = versioning_guard.objects_to_delete.clone(); // We don't check `checkpoint` because it's allowed to update its in memory state without // persisting to object store. let get_state = @@ -2022,6 +2023,7 @@ impl HummockManager { mem_state, loaded_state, "hummock in-mem state is inconsistent with meta store state", ); + versioning_guard.objects_to_delete = objects_to_delete; } /// Gets current version without pinning it. @@ -3353,7 +3355,6 @@ fn gen_version_delta<'a>( .entry(compact_task.compaction_group_id) .or_default() .group_deltas; - let mut gc_object_ids = vec![]; let mut removed_table_ids_map: BTreeMap> = BTreeMap::default(); for level in &compact_task.input_ssts { @@ -3364,15 +3365,13 @@ fn gen_version_delta<'a>( .map(|sst| { let object_id = sst.get_object_id(); let sst_id = sst.get_sst_id(); - if !trivial_move - && drop_sst( + if !trivial_move { + drop_sst( branched_ssts, compact_task.compaction_group_id, object_id, sst_id, - ) - { - gc_object_ids.push(object_id); + ); } sst_id }) @@ -3405,7 +3404,6 @@ fn gen_version_delta<'a>( })), }; group_deltas.push(group_delta); - version_delta.gc_object_ids.append(&mut gc_object_ids); version_delta.safe_epoch = std::cmp::max(old_version.safe_epoch, compact_task.watermark); // Don't persist version delta generated by compaction to meta store in deterministic mode. diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index d3417f76f537..65ddef44af8a 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -1157,12 +1157,51 @@ async fn test_extend_objects_to_delete() { ); let objects_to_delete = hummock_manager.get_objects_to_delete().await; assert_eq!(objects_to_delete.len(), orphan_sst_num as usize); - let pinned_version2 = hummock_manager.pin_version(context_id).await.unwrap(); + let pinned_version2: HummockVersion = hummock_manager.pin_version(context_id).await.unwrap(); + let objects_to_delete = hummock_manager.get_objects_to_delete().await; + assert_eq!( + objects_to_delete.len(), + orphan_sst_num as usize, + "{:?}", + objects_to_delete + ); hummock_manager .unpin_version_before(context_id, pinned_version2.id) .await .unwrap(); - // version1 is unpin, and then the sst removed in compaction can be reclaimed + let objects_to_delete = hummock_manager.get_objects_to_delete().await; + assert_eq!( + objects_to_delete.len(), + orphan_sst_num as usize, + "{:?}", + objects_to_delete + ); + // version1 is unpin, but version2 is pinned, and version2 is the checkpoint version. + // stale objects are combined in the checkpoint of version2, so no sst to reclaim + assert_eq!( + hummock_manager + .extend_objects_to_delete_from_scan(&all_object_ids) + .await, + orphan_sst_num as usize + ); + let objects_to_delete = hummock_manager.get_objects_to_delete().await; + assert_eq!(objects_to_delete.len(), orphan_sst_num as usize); + let new_epoch = pinned_version2.max_committed_epoch.next_epoch(); + hummock_manager + .commit_epoch( + new_epoch, + CommitEpochInfo::for_test(Vec::::new(), Default::default()), + ) + .await + .unwrap(); + let pinned_version3: HummockVersion = hummock_manager.pin_version(context_id).await.unwrap(); + assert_eq!(new_epoch, pinned_version3.max_committed_epoch); + hummock_manager + .unpin_version_before(context_id, pinned_version3.id) + .await + .unwrap(); + // version3 is the min pinned, and sst removed in compaction can be reclaimed, because they were tracked + // in the stale objects of version2 checkpoint assert_eq!( hummock_manager .extend_objects_to_delete_from_scan(&all_object_ids) diff --git a/src/meta/src/hummock/vacuum.rs b/src/meta/src/hummock/vacuum.rs index 0ff1d29aaa2a..9e45fa5a2bcc 100644 --- a/src/meta/src/hummock/vacuum.rs +++ b/src/meta/src/hummock/vacuum.rs @@ -228,40 +228,29 @@ mod tests { backup_manager, compactor_manager.clone(), )); - assert_eq!(VacuumManager::vacuum_metadata(&vacuum).await.unwrap(), 0); - assert_eq!( - VacuumManager::vacuum_object(&vacuum).await.unwrap().len(), - 0 - ); + assert_eq!(vacuum.vacuum_metadata().await.unwrap(), 0); + assert_eq!(vacuum.vacuum_object().await.unwrap().len(), 0); hummock_manager.pin_version(context_id).await.unwrap(); let sst_infos = add_test_tables(hummock_manager.as_ref(), context_id).await; - assert_eq!(VacuumManager::vacuum_metadata(&vacuum).await.unwrap(), 0); + assert_eq!(vacuum.vacuum_metadata().await.unwrap(), 0); hummock_manager.create_version_checkpoint(1).await.unwrap(); - assert_eq!(VacuumManager::vacuum_metadata(&vacuum).await.unwrap(), 6); - assert_eq!(VacuumManager::vacuum_metadata(&vacuum).await.unwrap(), 0); + assert_eq!(vacuum.vacuum_metadata().await.unwrap(), 6); + assert_eq!(vacuum.vacuum_metadata().await.unwrap(), 0); assert!(hummock_manager.get_objects_to_delete().await.is_empty()); hummock_manager .unpin_version_before(context_id, HummockVersionId::MAX) .await .unwrap(); + hummock_manager.create_version_checkpoint(0).await.unwrap(); assert!(!hummock_manager.get_objects_to_delete().await.is_empty()); // No SST deletion is scheduled because no available worker. - assert_eq!( - VacuumManager::vacuum_object(&vacuum).await.unwrap().len(), - 0 - ); + assert_eq!(vacuum.vacuum_object().await.unwrap().len(), 0); let _receiver = compactor_manager.add_compactor(context_id); // SST deletion is scheduled. - assert_eq!( - VacuumManager::vacuum_object(&vacuum).await.unwrap().len(), - 3 - ); + assert_eq!(vacuum.vacuum_object().await.unwrap().len(), 3); // The deletion is not acked yet. - assert_eq!( - VacuumManager::vacuum_object(&vacuum).await.unwrap().len(), - 3 - ); + assert_eq!(vacuum.vacuum_object().await.unwrap().len(), 3); // The vacuum task is reported. vacuum .report_vacuum_task(VacuumTask { @@ -275,9 +264,6 @@ mod tests { .await .unwrap(); // No objects_to_delete. - assert_eq!( - VacuumManager::vacuum_object(&vacuum).await.unwrap().len(), - 0 - ); + assert_eq!(vacuum.vacuum_object().await.unwrap().len(), 0); } } 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 560205a128b8..19295a5cf124 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 @@ -1065,7 +1065,6 @@ pub fn build_version_delta_after_version(version: &HummockVersion) -> HummockVer trivial_move: false, max_committed_epoch: version.max_committed_epoch, group_deltas: Default::default(), - gc_object_ids: vec![], new_table_watermarks: HashMap::new(), removed_table_ids: vec![], } diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 6f227b988936..4fa87296667a 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -12,14 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::mem::size_of; use std::sync::Arc; use prost::Message; use risingwave_common::catalog::TableId; +use risingwave_pb::hummock::group_delta::DeltaType; use risingwave_pb::hummock::hummock_version::PbLevels; -use risingwave_pb::hummock::hummock_version_delta::PbGroupDeltas; +use risingwave_pb::hummock::hummock_version_delta::GroupDeltas as PbGroupDeltas; use risingwave_pb::hummock::{PbHummockVersion, PbHummockVersionDelta}; use crate::table_watermark::TableWatermarks; @@ -118,7 +119,6 @@ pub struct HummockVersionDelta { pub max_committed_epoch: u64, pub safe_epoch: u64, pub trivial_move: bool, - pub gc_object_ids: Vec, pub new_table_watermarks: HashMap, pub removed_table_ids: Vec, } @@ -150,7 +150,6 @@ impl HummockVersionDelta { max_committed_epoch: delta.max_committed_epoch, safe_epoch: delta.safe_epoch, trivial_move: delta.trivial_move, - gc_object_ids: delta.gc_object_ids.clone(), new_table_watermarks: delta .new_table_watermarks .iter() @@ -177,7 +176,6 @@ impl HummockVersionDelta { max_committed_epoch: self.max_committed_epoch, safe_epoch: self.safe_epoch, trivial_move: self.trivial_move, - gc_object_ids: self.gc_object_ids.clone(), new_table_watermarks: self .new_table_watermarks .iter() @@ -191,3 +189,34 @@ impl HummockVersionDelta { } } } + +impl HummockVersionDelta { + /// Get the newly added object ids from the version delta. + /// + /// Note: the result can be false positive because we only collect the set of sst object ids in the `inserted_table_infos`, + /// but it is possible that the object is moved or split from other compaction groups or levels. + pub fn newly_added_object_ids(&self) -> HashSet { + let mut ret = HashSet::new(); + for group_deltas in self.group_deltas.values() { + for group_delta in &group_deltas.group_deltas { + if let Some(delta_type) = &group_delta.delta_type { + match delta_type { + DeltaType::IntraLevel(level_delta) => { + ret.extend( + level_delta + .inserted_table_infos + .iter() + .map(|sst| sst.object_id), + ); + } + DeltaType::GroupConstruct(_) + | DeltaType::GroupDestroy(_) + | DeltaType::GroupMetaChange(_) + | DeltaType::GroupTableChange(_) => {} + } + } + } + } + ret + } +} From 4b5efa2785d0b430669b2e14a53963ae9deee589 Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 18 Apr 2024 13:40:05 +0800 Subject: [PATCH 52/64] test: add Kafka consumer-group, add-partition tests (inline style) (#16244) Signed-off-by: xxchan --- Cargo.lock | 4 +- Makefile.toml | 3 +- ci/Dockerfile | 21 +++- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 10 +- ci/scripts/e2e-source-test.sh | 5 +- e2e_test/source/README.md | 4 + e2e_test/source_inline/README.md | 30 +++++ e2e_test/source_inline/commands.toml | 107 ++++++++++++++++++ .../source_inline/kafka/add_partition.slt | 74 ++++++++++++ .../source_inline/kafka/consumer_group.mjs | 88 ++++++++++++++ .../source_inline/kafka/consumer_group.slt | 101 +++++++++++++++++ risedev.yml | 8 +- scripts/source/prepare_ci_kafka.sh | 6 +- src/risedevtool/src/bin/risedev-dev.rs | 24 +--- src/risedevtool/src/risedev_env.rs | 6 + src/risedevtool/src/service_config.rs | 46 ++++++++ .../src/task/ensure_stop_service.rs | 13 ++- src/risedevtool/src/task/kafka_service.rs | 11 +- .../src/task/task_kafka_ready_check.rs | 8 +- 20 files changed, 526 insertions(+), 45 deletions(-) create mode 100644 e2e_test/source_inline/README.md create mode 100644 e2e_test/source_inline/commands.toml create mode 100644 e2e_test/source_inline/kafka/add_partition.slt create mode 100755 e2e_test/source_inline/kafka/consumer_group.mjs create mode 100644 e2e_test/source_inline/kafka/consumer_group.slt diff --git a/Cargo.lock b/Cargo.lock index 16350db6b46e..25b9f0324f7e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12079,9 +12079,9 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.20.0" +version = "0.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8518892e5e36bfa90163e53c4e4f36a388e0afa1cd6a3de0614253b3c9029c7" +checksum = "7e7c6a33098cd55e4fead1bd1f85c1d2064f02bafdb9fe004ca39fd94aee36e6" dependencies = [ "async-trait", "educe 0.4.23", diff --git a/Makefile.toml b/Makefile.toml index 604b7b2b4e44..f95ed99e5ec4 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -15,6 +15,7 @@ extend = [ { path = "src/storage/backup/integration_tests/Makefile.toml" }, { path = "src/java_binding/make-java-binding.toml" }, { path = "src/stream/tests/integration_tests/integration_test.toml" }, + { path = "e2e_test/source_inline/commands.toml" }, ] env_files = ["./risedev-components.user.env"] @@ -1292,7 +1293,7 @@ echo "All processes has exited." [tasks.slt] env = { SLT_HOST = "${RISEDEV_RW_FRONTEND_LISTEN_ADDRESS}", SLT_PORT = "${RISEDEV_RW_FRONTEND_PORT}", SLT_DB = "dev" } category = "RiseDev - Test - SQLLogicTest" -install_crate = { version = "0.20.0", crate_name = "sqllogictest-bin", binary = "sqllogictest", test_arg = [ +install_crate = { version = "0.20.1", crate_name = "sqllogictest-bin", binary = "sqllogictest", test_arg = [ "--help", ], install_command = "binstall" } dependencies = ["check-risedev-env-file"] diff --git a/ci/Dockerfile b/ci/Dockerfile index 5902b612c071..cffa1a026be3 100644 --- a/ci/Dockerfile +++ b/ci/Dockerfile @@ -18,6 +18,15 @@ RUN apt-get update -yy && \ && rm -rf /var/lib/{apt,dpkg,cache,log}/ ENV PYO3_PYTHON=python3.12 +# Install nvm and zx +ENV NVM_DIR /root/.nvm +ENV NODE_VERSION 20.11.1 +RUN curl -o- https://raw.githubusercontent.com/nvm-sh/nvm/v0.39.7/install.sh | bash \ + && . $NVM_DIR/nvm.sh \ + && nvm install $NODE_VERSION +ENV PATH $NVM_DIR/versions/node/v$NODE_VERSION/bin:$PATH +RUN npm install -g zx + SHELL ["/bin/bash", "-c"] RUN mkdir -p /risingwave @@ -43,6 +52,16 @@ RUN pip3 install --break-system-packages pyarrow pytest # Install poetry RUN curl -sSL https://install.python-poetry.org | python3 - +# Install rpk +RUN if [ "$(uname -m)" = "amd64" ] || [ "$(uname -m)" = "x86_64" ]; then \ + curl -LO https://github.com/redpanda-data/redpanda/releases/latest/download/rpk-linux-amd64.zip && \ + unzip rpk-linux-amd64.zip -d ~/.local/bin/ && \ + rm rpk-linux-amd64.zip; \ + else \ + curl -LO https://github.com/redpanda-data/redpanda/releases/latest/download/rpk-linux-arm64.zip && \ + unzip rpk-linux-arm64.zip -d ~/.local/bin/ && \ + rm rpk-linux-arm64.zip; \ + fi ENV PATH /root/.local/bin:$PATH ENV CARGO_REGISTRIES_CRATES_IO_PROTOCOL=sparse @@ -51,7 +70,7 @@ ENV CARGO_REGISTRIES_CRATES_IO_PROTOCOL=sparse RUN curl -L --proto '=https' --tlsv1.2 -sSf https://raw.githubusercontent.com/cargo-bins/cargo-binstall/main/install-from-binstall-release.sh | bash RUN cargo binstall -y --no-symlinks cargo-llvm-cov cargo-nextest cargo-hakari cargo-sort cargo-cache cargo-audit \ cargo-make@0.37.9 \ - sqllogictest-bin@0.19.1 \ + sqllogictest-bin@0.20.1 \ sccache@0.7.4 \ && cargo cache -a \ && rm -rf "/root/.cargo/registry/index" \ diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index 074f60aa97e0..1ec12359d896 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -10,7 +10,7 @@ cat ../rust-toolchain # shellcheck disable=SC2155 # REMEMBER TO ALSO UPDATE ci/docker-compose.yml -export BUILD_ENV_VERSION=v20240413 +export BUILD_ENV_VERSION=v20240414_x export BUILD_TAG="public.ecr.aws/w1p7b4n3/rw-build-env:${BUILD_ENV_VERSION}" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 1d67188b1c4f..c754dcc174ed 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -71,7 +71,7 @@ services: retries: 5 source-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240414_x depends_on: - mysql - db @@ -84,7 +84,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240414_x depends_on: - mysql - db @@ -103,12 +103,12 @@ services: rw-build-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240414_x volumes: - ..:/risingwave ci-flamegraph-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240414_x # NOTE(kwannoel): This is used in order to permit # syscalls for `nperf` (perf_event_open), # so it can do CPU profiling. @@ -119,7 +119,7 @@ services: - ..:/risingwave regress-test-env: - image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240413 + image: public.ecr.aws/w1p7b4n3/rw-build-env:v20240414_x depends_on: db: condition: service_healthy diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 851fca303ab1..8a683f56b855 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -152,7 +152,7 @@ risedev ci-kill echo "--- e2e, ci-kafka-plus-pubsub, kafka and pubsub source" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ -risedev ci-start ci-pubsub +risedev ci-start ci-pubsub-kafka ./scripts/source/prepare_ci_kafka.sh cargo run --bin prepare_ci_pubsub risedev slt './e2e_test/source/basic/*.slt' @@ -168,6 +168,9 @@ echo "--- e2e, kafka alter source again" ./scripts/source/prepare_data_after_alter.sh 3 risedev slt './e2e_test/source/basic/alter/kafka_after_new_data_2.slt' +echo "--- e2e, inline test" +risedev slt './e2e_test/source_inline/**/*.slt' + echo "--- Run CH-benCHmark" risedev slt './e2e_test/ch_benchmark/batch/ch_benchmark.slt' risedev slt './e2e_test/ch_benchmark/streaming/*.slt' diff --git a/e2e_test/source/README.md b/e2e_test/source/README.md index b6e9dfa30816..4152ab3dc973 100644 --- a/e2e_test/source/README.md +++ b/e2e_test/source/README.md @@ -1,3 +1,7 @@ +> [!NOTE] +> +> Please write new tests according to the style in `e2e_test/source_inline`. + Test in this directory needs some prior setup. See also `ci/scripts/e2e-source-test.sh`, and `scripts/source` diff --git a/e2e_test/source_inline/README.md b/e2e_test/source_inline/README.md new file mode 100644 index 000000000000..3a9070639b8c --- /dev/null +++ b/e2e_test/source_inline/README.md @@ -0,0 +1,30 @@ +# "Inline" style source e2e tests + +Compared with prior source tests ( `e2e_test/source` ), tests in this directory are expected to be easy to run locally and easy to write. + +Refer to https://github.com/risingwavelabs/risingwave/issues/12451#issuecomment-2051861048 for more details. + +## Install Dependencies + +Some additional tools are needed to run the `system` commands in tests. + +- `rpk`: Redpanda (Kafka) CLI toolbox. https://docs.redpanda.com/current/get-started/rpk-install/ +- `zx`: A tool for writing better scripts. `npm install -g zx` + +## Run tests + +To run locally, use `risedev d` to start services (including external systems like Kafka and Postgres, or specify `user-managed` to use your own service). +Then use `risedev slt` to run the tests, which will load the environment variables (ports, etc.) +according to the services started by `risedev d` . + +```sh +risedev slt 'e2e_test/source_inline/**/*.slt' +``` + +## Write tests + +To write tests, please ensure each file is self-contained and does not depend on running external scripts to setup the environment. + +Use `system` command to setup instead. +For simple cases, you can directly write a bash command; +For more complex cases, you can write a test script (with any language like bash, python, zx), and invoke it in the `system` command. diff --git a/e2e_test/source_inline/commands.toml b/e2e_test/source_inline/commands.toml new file mode 100644 index 000000000000..8af865099ac7 --- /dev/null +++ b/e2e_test/source_inline/commands.toml @@ -0,0 +1,107 @@ +# This file contains commands used by the tests. + +[tasks.source-test-hook] +private = true +dependencies = ["check-risedev-env-file"] +env_files = ["${PREFIX_CONFIG}/risedev-env"] + +# Note about the Kafka CLI tooling: +# - Built-in Kafka console tools: +# Java based. +# Style example: kafka-topics.sh --bootstrap-server localhost:9092 --topic t --create +# Some limitations: cannot disable logging easily, cannot consume to end and then exit. +# - kcat: +# C based (rdkafka) +# Some limitations: cannot do admin operations, only consume/produce. +# - rpk: +# Golang based. +# Style example: RPK_BROKERS=localhost:9092 rpk topic create t +[tasks.kafka-hook] +private = true +description = "Check if Kafka is started by RiseDev" +dependencies = ["source-test-hook"] +script = ''' +#!/usr/bin/env sh +set -e + +if [ ! -d "${PREFIX_BIN}/kafka" ]; then + echo "Kafka is not installed in ${PREFIX_BIN}/kafka. Did you enable Kafka using $(tput setaf 4)\`./risedev configure\`$(tput sgr0)?" + exit 1 +fi + +if [ -z "${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}" ]; then + echo "RISEDEV_KAFKA_BOOTSTRAP_SERVERS is not set in risedev-env file. Did you start Kafka using $(tput setaf 4)\`./risedev d\`$(tput sgr0)?" + exit 1 +fi +''' + +[tasks.clean-kafka] +category = "RiseDev - Test - Source Test - Kafka" +description = "Delete all kafka topics." +dependencies = ["kafka-hook"] +command = "rpk" +args = ["topic", "delete", "-r", "*"] + +[tasks.kafka-topics] +category = "RiseDev - Test - Source Test - Kafka" +dependencies = ["kafka-hook"] +script = """ +#!/usr/bin/env sh +set -e +${PREFIX_BIN}/kafka/bin/kafka-topics.sh --bootstrap-server ${RISEDEV_KAFKA_BOOTSTRAP_SERVERS} "$@" +""" + +[tasks.kafka-produce] +category = "RiseDev - Test - Source Test - Kafka" +dependencies = ["kafka-hook"] +script = """ +#!/usr/bin/env sh +set -e +${PREFIX_BIN}/kafka/bin/kafka-console-producer.sh --bootstrap-server ${RISEDEV_KAFKA_BOOTSTRAP_SERVERS} "$@" +""" + +[tasks.kafka-consume] +category = "RiseDev - Test - Source Test - Kafka" +dependencies = ["kafka-hook"] +script = """ +#!/usr/bin/env sh +set -e +${PREFIX_BIN}/kafka/bin/kafka-console-consumer.sh --bootstrap-server ${RISEDEV_KAFKA_BOOTSTRAP_SERVERS} "$@" +""" + +[tasks.kafka-consumer-groups] +category = "RiseDev - Test - Source Test - Kafka" +dependencies = ["kafka-hook"] +script = """ +#!/usr/bin/env sh +set -e +${PREFIX_BIN}/kafka/bin/kafka-consumer-groups.sh --bootstrap-server ${RISEDEV_KAFKA_BOOTSTRAP_SERVERS} "$@" +""" + +# rpk tools +[tasks.rpk] +category = "RiseDev - Test - Source Test - Kafka" +dependencies = ["kafka-hook"] +# check https://docs.redpanda.com/current/reference/rpk/rpk-x-options/ or rpk -X help/list for options +script = """ +#!/usr/bin/env sh +set -e + +if [ -z "$(which rpk)" ]; then + echo "rpk is not installed. Install it via https://docs.redpanda.com/current/get-started/rpk-install/" + exit 1 +fi + +rpk "$@" +""" + +[tasks.redpanda-console] +category = "RiseDev - Test - Source Test - Kafka" +description = "Start Redpanda console (Kafka GUI) at localhost:8080." +dependencies = ["kafka-hook"] +script = ''' +#!/usr/bin/env sh +set -e +echo "$(tput setaf 2)Start Redpanda console at http://localhost:8080$(tput sgr0)" +docker run --network host -e KAFKA_BROKERS=$RPK_BROKERS docker.redpanda.com/redpandadata/console:latest +''' diff --git a/e2e_test/source_inline/kafka/add_partition.slt b/e2e_test/source_inline/kafka/add_partition.slt new file mode 100644 index 000000000000..9399cf732b97 --- /dev/null +++ b/e2e_test/source_inline/kafka/add_partition.slt @@ -0,0 +1,74 @@ +# Note: control substitution on will force us to use "\\n" instead of "\n" in commands +control substitution on + +system ok +rpk topic create test_add_partition -p 3 + +system ok +cat < { + const [_broker_id, group_name] = line.split(/\s+/); + return group_name; + }) + .filter((group_name) => { + return group_name.startsWith(`rw-consumer-${fragment_id}`); + }); +} + +async function describe_consumer_group(group_name) { + const res = await $`rpk group describe -s ${group_name}`; + // GROUP rw-consumer-1-1 + // COORDINATOR 0 + // STATE Empty + // BALANCER + // MEMBERS 0 + // TOTAL-LAG 2 + const obj = {}; + for (const line of res.toString().trim().split("\n")) { + const [key, value] = line.split(/\s+/); + obj[key] = value; + } + return obj; +} + +async function list_consumer_group_members(fragment_id) { + const groups = await list_consumer_groups(fragment_id); + return Promise.all( + groups.map(async (group_name) => { + return (await describe_consumer_group(group_name))["MEMBERS"] + }) + ); +} + +async function list_consumer_group_lags(fragment_id) { + const groups = await list_consumer_groups(fragment_id); + return Promise.all( + groups.map(async (group_name) => { + return (await describe_consumer_group(group_name))["TOTAL-LAG"] + }) + ); +} + +const fragment_id = await get_fragment_id_of_mv(mv); +if (command == "list-groups") { + echo`${(await list_consumer_groups(fragment_id))}`; +} else if (command == "list-members") { + echo`${await list_consumer_group_members(fragment_id)}`; +} else if (command == "list-lags") { + echo`${await list_consumer_group_lags(fragment_id)}`; +} else { + throw new Error(`Invalid command: ${command}`); +} diff --git a/e2e_test/source_inline/kafka/consumer_group.slt b/e2e_test/source_inline/kafka/consumer_group.slt new file mode 100644 index 000000000000..ed97dec558f3 --- /dev/null +++ b/e2e_test/source_inline/kafka/consumer_group.slt @@ -0,0 +1,101 @@ +# Note: control substitution on will force us to use "\\n" instead of "\n" in commands +control substitution on + +# Note either `./risedev rpk` or `rpk` is ok here. +# risedev-env contains env var RPK_BROKERS, which is read by rpk +system ok +rpk topic create test_consumer_group -p 3 + +system ok +cat </dev/null 2>&1 && pwd)" cd "$SCRIPT_PATH/.." || exit 1 -KAFKA_BIN="$SCRIPT_PATH/../../.risingwave/bin/kafka/bin" - echo "$SCRIPT_PATH" if [ "$1" == "compress" ]; then @@ -39,10 +37,10 @@ for filename in $kafka_data_files; do # always ok echo "Drop topic $topic" - "$KAFKA_BIN"/kafka-topics.sh --bootstrap-server message_queue:29092 --topic "$topic" --delete || true + risedev kafka-topics --topic "$topic" --delete || true echo "Recreate topic $topic with partition $partition" - "$KAFKA_BIN"/kafka-topics.sh --bootstrap-server message_queue:29092 --topic "$topic" --create --partitions "$partition") & + risedev kafka-topics --topic "$topic" --create --partitions "$partition") & done wait diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 1e4df4b43a80..d5a14cbf8c1b 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -98,28 +98,8 @@ fn task_main( let mut ports = vec![]; for service in services { - let listen_info = match service { - ServiceConfig::Minio(c) => Some((c.port, c.id.clone())), - ServiceConfig::Etcd(c) => Some((c.port, c.id.clone())), - ServiceConfig::Sqlite(_) => None, - ServiceConfig::Prometheus(c) => Some((c.port, c.id.clone())), - ServiceConfig::ComputeNode(c) => Some((c.port, c.id.clone())), - ServiceConfig::MetaNode(c) => Some((c.port, c.id.clone())), - ServiceConfig::Frontend(c) => Some((c.port, c.id.clone())), - ServiceConfig::Compactor(c) => Some((c.port, c.id.clone())), - ServiceConfig::Grafana(c) => Some((c.port, c.id.clone())), - ServiceConfig::Tempo(c) => Some((c.port, c.id.clone())), - ServiceConfig::Kafka(c) => Some((c.port, c.id.clone())), - ServiceConfig::Pubsub(c) => Some((c.port, c.id.clone())), - ServiceConfig::Redis(c) => Some((c.port, c.id.clone())), - ServiceConfig::ZooKeeper(c) => Some((c.port, c.id.clone())), - ServiceConfig::AwsS3(_) => None, - ServiceConfig::Opendal(_) => None, - ServiceConfig::RedPanda(_) => None, - }; - - if let Some(x) = listen_info { - ports.push(x); + if let Some(port) = service.port() { + ports.push((port, service.id().to_string(), service.user_managed())); } } diff --git a/src/risedevtool/src/risedev_env.rs b/src/risedevtool/src/risedev_env.rs index 077f1ce51f82..1efdf1470998 100644 --- a/src/risedevtool/src/risedev_env.rs +++ b/src/risedevtool/src/risedev_env.rs @@ -71,6 +71,12 @@ pub fn generate_risedev_env(services: &Vec) -> String { let port = &c.port; writeln!(env, "RISEDEV_RW_FRONTEND_PORT=\"{port}\"",).unwrap(); } + ServiceConfig::Kafka(c) => { + let brokers = format!("{}:{}", c.address, c.port); + writeln!(env, r#"RISEDEV_KAFKA_BOOTSTRAP_SERVERS="{brokers}""#,).unwrap(); + writeln!(env, r#"RISEDEV_KAFKA_WITH_OPTIONS_COMMON="connector='kafka',properties.bootstrap.server='{brokers}'""#).unwrap(); + writeln!(env, r#"RPK_BROKERS="{brokers}""#).unwrap(); + } _ => {} } } diff --git a/src/risedevtool/src/service_config.rs b/src/risedevtool/src/service_config.rs index 97996bbeb510..e5f149b8d10c 100644 --- a/src/risedevtool/src/service_config.rs +++ b/src/risedevtool/src/service_config.rs @@ -277,6 +277,8 @@ pub struct KafkaConfig { pub provide_zookeeper: Option>, pub persist_data: bool, pub broker_id: u32, + + pub user_managed: bool, } #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] #[serde(rename_all = "kebab-case")] @@ -378,6 +380,50 @@ impl ServiceConfig { Self::Opendal(c) => &c.id, } } + + pub fn port(&self) -> Option { + match self { + Self::ComputeNode(c) => Some(c.port), + Self::MetaNode(c) => Some(c.port), + Self::Frontend(c) => Some(c.port), + Self::Compactor(c) => Some(c.port), + Self::Minio(c) => Some(c.port), + Self::Etcd(c) => Some(c.port), + Self::Sqlite(_) => None, + Self::Prometheus(c) => Some(c.port), + Self::Grafana(c) => Some(c.port), + Self::Tempo(c) => Some(c.port), + Self::AwsS3(_) => None, + Self::ZooKeeper(c) => Some(c.port), + Self::Kafka(c) => Some(c.port), + Self::Pubsub(c) => Some(c.port), + Self::Redis(c) => Some(c.port), + Self::RedPanda(_c) => None, + Self::Opendal(_) => None, + } + } + + pub fn user_managed(&self) -> bool { + match self { + Self::ComputeNode(c) => c.user_managed, + Self::MetaNode(c) => c.user_managed, + Self::Frontend(c) => c.user_managed, + Self::Compactor(c) => c.user_managed, + Self::Minio(_c) => false, + Self::Etcd(_c) => false, + Self::Sqlite(_c) => false, + Self::Prometheus(_c) => false, + Self::Grafana(_c) => false, + Self::Tempo(_c) => false, + Self::AwsS3(_c) => false, + Self::ZooKeeper(_c) => false, + Self::Kafka(c) => c.user_managed, + Self::Pubsub(_c) => false, + Self::Redis(_c) => false, + Self::RedPanda(_c) => false, + Self::Opendal(_c) => false, + } + } } mod string { diff --git a/src/risedevtool/src/task/ensure_stop_service.rs b/src/risedevtool/src/task/ensure_stop_service.rs index 24de2d32e0bb..519804d11688 100644 --- a/src/risedevtool/src/task/ensure_stop_service.rs +++ b/src/risedevtool/src/task/ensure_stop_service.rs @@ -17,11 +17,12 @@ use anyhow::Result; use super::{ExecuteContext, Task}; pub struct EnsureStopService { - ports: Vec<(u16, String)>, + /// `(port, id, user_managed)` + ports: Vec<(u16, String, bool)>, } impl EnsureStopService { - pub fn new(ports: Vec<(u16, String)>) -> Result { + pub fn new(ports: Vec<(u16, String, bool)>) -> Result { Ok(Self { ports }) } } @@ -30,12 +31,16 @@ impl Task for EnsureStopService { fn execute(&mut self, ctx: &mut ExecuteContext) -> anyhow::Result<()> { ctx.service(self); - for (port, service) in &self.ports { + for (port, service_id, user_managed) in &self.ports { + // Do not require stopping user-managed services + if *user_managed { + continue; + } let address = format!("127.0.0.1:{}", port); ctx.pb.set_message(format!( "waiting for port close - {} (will be used by {})", - address, service + address, service_id )); ctx.wait_tcp_close(&address)?; } diff --git a/src/risedevtool/src/task/kafka_service.rs b/src/risedevtool/src/task/kafka_service.rs index 9bbdd3ac5efa..df0eb4a0fa31 100644 --- a/src/risedevtool/src/task/kafka_service.rs +++ b/src/risedevtool/src/task/kafka_service.rs @@ -74,7 +74,16 @@ impl Task for KafkaService { cmd.arg(config_path); - ctx.run_command(ctx.tmux_run(cmd)?)?; + if !self.config.user_managed { + ctx.run_command(ctx.tmux_run(cmd)?)?; + } else { + ctx.pb.set_message("user managed"); + writeln!( + &mut ctx.log, + "Please start your Kafka at {}:{}\n\n", + self.config.listen_address, self.config.port + )?; + } ctx.pb.set_message("started"); diff --git a/src/risedevtool/src/task/task_kafka_ready_check.rs b/src/risedevtool/src/task/task_kafka_ready_check.rs index ef8823956206..79838bf8eca6 100644 --- a/src/risedevtool/src/task/task_kafka_ready_check.rs +++ b/src/risedevtool/src/task/task_kafka_ready_check.rs @@ -33,8 +33,12 @@ impl KafkaReadyCheckTask { impl Task for KafkaReadyCheckTask { fn execute(&mut self, ctx: &mut ExecuteContext) -> anyhow::Result<()> { - ctx.pb.set_message("waiting for online..."); - + if self.config.user_managed { + ctx.pb + .set_message("waiting for user-managed service online..."); + } else { + ctx.pb.set_message("waiting for online..."); + } let mut config = ClientConfig::new(); config.set( "bootstrap.servers", From a365c0370c6ed091b678c7795b0b0b9fd7baed9a Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 18 Apr 2024 15:17:46 +0800 Subject: [PATCH 53/64] chore: bump `thiserror-ext` to v0.1 (#16373) Signed-off-by: Bugen Zhao --- Cargo.lock | 10 +++++----- Cargo.toml | 2 +- src/frontend/src/error.rs | 5 ++--- src/meta/src/error.rs | 6 ++++-- src/object_store/src/object/error.rs | 4 ++-- src/storage/src/error.rs | 4 ++-- src/storage/src/hummock/error.rs | 4 ++-- src/stream/src/error.rs | 4 ++-- src/stream/src/executor/error.rs | 6 ++++-- 9 files changed, 24 insertions(+), 21 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 25b9f0324f7e..1f6f31de2cf7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8448,7 +8448,7 @@ dependencies = [ "indoc", "libc", "memoffset", - "parking_lot 0.11.2", + "parking_lot 0.12.1", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -12675,9 +12675,9 @@ dependencies = [ [[package]] name = "thiserror-ext" -version = "0.0.11" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54a6a1db013d9f7175962ec64a0fb90760832062e29f6664f1aefee91d072a50" +checksum = "a7c19760dc47062bca5c1b3699b032111c93802d51ac47660db11b08afc6bad2" dependencies = [ "thiserror", "thiserror-ext-derive", @@ -12685,9 +12685,9 @@ dependencies = [ [[package]] name = "thiserror-ext-derive" -version = "0.0.11" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98c358cfea7a4a0f4554d4df6533f9b652d44c0108e99db3685f28f30b0954aa" +checksum = "667c8c48f68021098038115926c64d9950b0582062ae63f7d30638b1168daf03" dependencies = [ "either", "proc-macro2", diff --git a/Cargo.toml b/Cargo.toml index e6968b90a024..2bd253f9e4d2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -156,7 +156,7 @@ deltalake = { git = "https://github.com/risingwavelabs/delta-rs", rev = "5c2dccd itertools = "0.12.0" lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "2682b85" } parquet = "50" -thiserror-ext = "0.0.11" +thiserror-ext = "0.1.2" tikv-jemalloc-ctl = { git = "https://github.com/risingwavelabs/jemallocator.git", rev = "64a2d9" } tikv-jemallocator = { git = "https://github.com/risingwavelabs/jemallocator.git", features = [ "profiling", diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index 9898e26a79ba..27a6418a7b0c 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -23,7 +23,6 @@ use risingwave_expr::ExprError; use risingwave_pb::PbFieldNotFound; use risingwave_rpc_client::error::{RpcError, TonicStatusWrapper}; use thiserror::Error; -use thiserror_ext::Box; use tokio::task::JoinError; /// The error type for the frontend crate, acting as the top-level error type for the @@ -33,8 +32,8 @@ use tokio::task::JoinError; // - Some variants are never constructed. // - Some variants store a type-erased `BoxedError` to resolve the reverse dependency. // It's not necessary anymore as the error type is now defined at the top-level. -#[derive(Error, Debug, Box)] -#[thiserror_ext(newtype(name = RwError, backtrace, report_debug))] +#[derive(Error, thiserror_ext::ReportDebug, thiserror_ext::Box)] +#[thiserror_ext(newtype(name = RwError, backtrace))] pub enum ErrorCode { #[error("internal error: {0}")] InternalError(String), diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 72eb9de2f67d..8aeaed2f9c5a 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -26,8 +26,10 @@ use crate::storage::MetaStoreError; pub type MetaResult = std::result::Result; -#[derive(thiserror::Error, Debug, thiserror_ext::Arc, thiserror_ext::Construct)] -#[thiserror_ext(newtype(name = MetaError, backtrace, report_debug))] +#[derive( + thiserror::Error, thiserror_ext::ReportDebug, thiserror_ext::Arc, thiserror_ext::Construct, +)] +#[thiserror_ext(newtype(name = MetaError, backtrace))] pub enum MetaErrorInner { #[error("MetaStore transaction error: {0}")] TransactionError( diff --git a/src/object_store/src/object/error.rs b/src/object_store/src/object/error.rs index f13ae2b4d8f5..d4e8e0bebe15 100644 --- a/src/object_store/src/object/error.rs +++ b/src/object_store/src/object/error.rs @@ -24,8 +24,8 @@ use thiserror::Error; use thiserror_ext::AsReport; use tokio::sync::oneshot::error::RecvError; -#[derive(Error, Debug, thiserror_ext::Box, thiserror_ext::Construct)] -#[thiserror_ext(newtype(name = ObjectError, backtrace, report_debug))] +#[derive(Error, thiserror_ext::ReportDebug, thiserror_ext::Box, thiserror_ext::Construct)] +#[thiserror_ext(newtype(name = ObjectError, backtrace))] pub enum ObjectErrorInner { #[error("s3 error: {0}")] S3(#[source] BoxedError), diff --git a/src/storage/src/error.rs b/src/storage/src/error.rs index a0795c9e7b55..49cadc6e82a3 100644 --- a/src/storage/src/error.rs +++ b/src/storage/src/error.rs @@ -18,8 +18,8 @@ use thiserror::Error; use crate::hummock::HummockError; use crate::mem_table::MemTableError; -#[derive(Error, Debug, thiserror_ext::Box)] -#[thiserror_ext(newtype(name = StorageError, backtrace, report_debug))] +#[derive(Error, thiserror_ext::ReportDebug, thiserror_ext::Box)] +#[thiserror_ext(newtype(name = StorageError, backtrace))] pub enum ErrorKind { #[error("Hummock error: {0}")] Hummock( diff --git a/src/storage/src/hummock/error.rs b/src/storage/src/hummock/error.rs index 6c2f0b912991..10e7ffbf325a 100644 --- a/src/storage/src/hummock/error.rs +++ b/src/storage/src/hummock/error.rs @@ -18,8 +18,8 @@ use thiserror_ext::AsReport; use tokio::sync::oneshot::error::RecvError; // TODO(error-handling): should prefer use error types than strings. -#[derive(Error, Debug, thiserror_ext::Box)] -#[thiserror_ext(newtype(name = HummockError, backtrace, report_debug))] +#[derive(Error, thiserror_ext::ReportDebug, thiserror_ext::Box)] +#[thiserror_ext(newtype(name = HummockError, backtrace))] pub enum HummockErrorInner { #[error("Magic number mismatch: expected {expected}, found: {found}")] MagicMismatch { expected: u32, found: u32 }, diff --git a/src/stream/src/error.rs b/src/stream/src/error.rs index bad9307cc17a..ea957086f038 100644 --- a/src/stream/src/error.rs +++ b/src/stream/src/error.rs @@ -29,12 +29,12 @@ pub type StreamResult = std::result::Result; /// The error type for streaming tasks. #[derive( thiserror::Error, - Debug, + thiserror_ext::ReportDebug, thiserror_ext::Arc, thiserror_ext::ContextInto, thiserror_ext::Construct, )] -#[thiserror_ext(newtype(name = StreamError, backtrace, report_debug))] +#[thiserror_ext(newtype(name = StreamError, backtrace))] pub enum ErrorKind { #[error("Storage error: {0}")] Storage( diff --git a/src/stream/src/executor/error.rs b/src/stream/src/executor/error.rs index 558c0c7088d9..48d093f9883e 100644 --- a/src/stream/src/executor/error.rs +++ b/src/stream/src/executor/error.rs @@ -32,8 +32,10 @@ use super::Barrier; pub type StreamExecutorResult = std::result::Result; /// The error type for streaming executors. -#[derive(thiserror::Error, Debug, thiserror_ext::Box, thiserror_ext::Construct)] -#[thiserror_ext(newtype(name = StreamExecutorError, backtrace, report_debug))] +#[derive( + thiserror::Error, thiserror_ext::ReportDebug, thiserror_ext::Box, thiserror_ext::Construct, +)] +#[thiserror_ext(newtype(name = StreamExecutorError, backtrace))] #[derive(AsRefStr)] pub enum ErrorKind { #[error("Storage error: {0}")] From c2827c721c3bca35510f5935298c244fbcbb3cf4 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Thu, 18 Apr 2024 15:25:35 +0800 Subject: [PATCH 54/64] feat(frontend): support cursor. (#15180) Co-authored-by: William Wen Co-authored-by: Patrick Huang --- ci/scripts/run-e2e-test.sh | 5 + e2e_test/batch/transaction/cursor.slt | 8 +- e2e_test/subscription/check_sql_statement.slt | 50 ++ .../create_table_and_subscription.slt | 8 + .../drop_table_and_subscription.slt | 5 + e2e_test/subscription/main.py | 238 ++++++++ proto/catalog.proto | 1 + src/common/src/catalog/internal_table.rs | 7 + src/common/src/util/stream_graph_visitor.rs | 2 +- src/frontend/planner_test/src/lib.rs | 14 +- .../src/catalog/subscription_catalog.rs | 28 + src/frontend/src/handler/close_cursor.rs | 17 +- src/frontend/src/handler/create_sink.rs | 32 +- .../src/handler/create_subscription.rs | 6 +- src/frontend/src/handler/declare_cursor.rs | 123 +++- src/frontend/src/handler/fetch_cursor.rs | 33 +- src/frontend/src/handler/mod.rs | 18 +- src/frontend/src/handler/transaction.rs | 4 +- src/frontend/src/handler/util.rs | 88 ++- .../plan_node/stream_subscription.rs | 1 + src/frontend/src/optimizer/plan_node/utils.rs | 12 +- src/frontend/src/scheduler/snapshot.rs | 9 + src/frontend/src/session.rs | 41 +- src/frontend/src/session/cursor.rs | 136 ----- src/frontend/src/session/cursor_manager.rs | 544 ++++++++++++++++++ src/frontend/src/session/transaction.rs | 4 + src/meta/model_v2/migration/src/lib.rs | 2 + ...062305_subscription_internal_table_name.rs | 41 ++ src/meta/model_v2/src/subscription.rs | 2 + src/meta/src/controller/catalog.rs | 20 +- src/meta/src/controller/mod.rs | 1 + src/meta/src/manager/catalog/mod.rs | 13 +- src/sqlparser/src/ast/mod.rs | 51 +- src/sqlparser/src/ast/statement.rs | 160 ++++++ src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 97 ++-- .../log_store_impl/subscription_log_store.rs | 9 +- src/stream/src/executor/subscription.rs | 6 +- src/utils/pgwire/src/pg_response.rs | 4 +- src/utils/pgwire/src/types.rs | 4 + 40 files changed, 1506 insertions(+), 339 deletions(-) create mode 100644 e2e_test/subscription/check_sql_statement.slt create mode 100644 e2e_test/subscription/create_table_and_subscription.slt create mode 100644 e2e_test/subscription/drop_table_and_subscription.slt create mode 100644 e2e_test/subscription/main.py delete mode 100644 src/frontend/src/session/cursor.rs create mode 100644 src/frontend/src/session/cursor_manager.rs create mode 100644 src/meta/model_v2/migration/src/m20240417_062305_subscription_internal_table_name.rs diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index 84b336362613..9c205e418222 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -96,6 +96,11 @@ sqllogictest -p 4566 -d dev './e2e_test/ttl/ttl.slt' sqllogictest -p 4566 -d dev './e2e_test/database/prepare.slt' sqllogictest -p 4566 -d test './e2e_test/database/test.slt' +echo "--- e2e, $mode, subscription" +python3 -m pip install --break-system-packages psycopg2-binary +sqllogictest -p 4566 -d dev './e2e_test/subscription/check_sql_statement.slt' +python3 ./e2e_test/subscription/main.py + echo "--- e2e, $mode, Apache Superset" sqllogictest -p 4566 -d dev './e2e_test/superset/*.slt' --junit "batch-${profile}" diff --git a/e2e_test/batch/transaction/cursor.slt b/e2e_test/batch/transaction/cursor.slt index ae0ac3706e3d..124224c40136 100644 --- a/e2e_test/batch/transaction/cursor.slt +++ b/e2e_test/batch/transaction/cursor.slt @@ -30,7 +30,7 @@ DECLARE test_cursor CURSOR FOR SELECT * FROM test where a > 2 ORDER BY a; -statement error cursor "test_cursor" already exists +statement error DECLARE test_cursor CURSOR FOR SELECT * FROM test where a > 2; @@ -40,7 +40,7 @@ DECLARE test_cursor CURSOR FOR SELECT * FROM test_non_existent where a > 2; -statement error cursor "test_cursor_non_existent" does not exist +statement error FETCH NEXT from test_cursor_non_existent; query II @@ -57,13 +57,13 @@ query II FETCH NEXT from test_cursor; ---- -statement error cursor "test_cursor_non_existent" does not exist +statement error CLOSE test_cursor_non_existent; statement ok CLOSE test_cursor; -statement error cursor "test_cursor" does not exist +statement error FETCH NEXT from test_cursor; statement ok diff --git a/e2e_test/subscription/check_sql_statement.slt b/e2e_test/subscription/check_sql_statement.slt new file mode 100644 index 000000000000..7b7d9488495c --- /dev/null +++ b/e2e_test/subscription/check_sql_statement.slt @@ -0,0 +1,50 @@ +statement ok +create table t1 (v1 int, v2 int, v3 int); + +statement ok +insert into t1 values (1,2), (2,3); + +statement ok +create subscription sub from t1 with(retention = '1D'); + +statement ok +declare cur subscription cursor for sub; + +statement ok +declare cur1 subscription cursor for sub since now(); + +statement ok +declare cur2 subscription cursor for sub since proctime(); + +statement ok +declare cur3 subscription cursor for sub since begin(); + +statement error +declare cur4 subscription cursor for sub since 1; + +statement error +declare cur5 subscription cursor for sub since asd; + +statement error +declare cur6 subscription cursor for sub since 18446744073709551615; + +statement error +declare cur subscription cursor for sub; + +statement ok +close cur; + +statement ok +close cur1; + +statement ok +close cur2; + +statement ok +close cur3; + +statement ok +drop subscription sub; + +statement ok +drop table t1; \ No newline at end of file diff --git a/e2e_test/subscription/create_table_and_subscription.slt b/e2e_test/subscription/create_table_and_subscription.slt new file mode 100644 index 000000000000..94039f98b11c --- /dev/null +++ b/e2e_test/subscription/create_table_and_subscription.slt @@ -0,0 +1,8 @@ +statement ok +create table t1 (v1 int, v2 int); + +statement ok +insert into t1 values (1,2); + +statement ok +create subscription sub from t1 with(retention = '1D'); \ No newline at end of file diff --git a/e2e_test/subscription/drop_table_and_subscription.slt b/e2e_test/subscription/drop_table_and_subscription.slt new file mode 100644 index 000000000000..0df183a5b779 --- /dev/null +++ b/e2e_test/subscription/drop_table_and_subscription.slt @@ -0,0 +1,5 @@ +statement ok +drop subscription sub; + +statement ok +drop table t1; \ No newline at end of file diff --git a/e2e_test/subscription/main.py b/e2e_test/subscription/main.py new file mode 100644 index 000000000000..f8e78813801f --- /dev/null +++ b/e2e_test/subscription/main.py @@ -0,0 +1,238 @@ +import subprocess +import psycopg2 +import time + + +def execute_slt(slt): + if slt is None or slt == "": + return + cmd = f"sqllogictest -p 4566 -d dev {slt}" + print(f"Command line is [{cmd}]") + subprocess.run(cmd, + shell=True, + check=True) + time.sleep(3) + +def create_table_subscription(): + execute_slt("./e2e_test/subscription/create_table_and_subscription.slt") + +def drop_table_subscription(): + execute_slt("./e2e_test/subscription/drop_table_and_subscription.slt") + +def execute_query(sql,conn): + cur = conn.cursor() + cur.execute(sql) + conn.commit() + rows = cur.fetchall() + cur.close() + return rows + +def execute_insert(sql,conn): + cur = conn.cursor() + cur.execute(sql) + conn.commit() + cur.close() + +def check_rows_data(expect_vec,rows,status): + row = rows[0] + for index, value in enumerate(row): + if index == 0: + continue + if index == 1: + assert value == status,f"expect {value} but got {status}" + continue + assert value == expect_vec[index-2],f"expect {expect_vec[index-2]} but got {value}" + +def test_cursor_snapshot(): + print(f"test_cursor_snapshot") + create_table_subscription() + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + execute_insert("declare cur subscription cursor for sub",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([1,2],row,1) + row = execute_query("fetch next from cur",conn) + assert row == [] + execute_insert("close cur",conn) + drop_table_subscription() + + +def test_cursor_snapshot_log_store(): + print(f"test_cursor_snapshot_log_store") + create_table_subscription() + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + execute_insert("declare cur subscription cursor for sub",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([1,2],row,1) + row = execute_query("fetch next from cur",conn) + assert row == [] + execute_insert("insert into t1 values(4,4)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(5,5)",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([4,4],row,1) + row = execute_query("fetch next from cur",conn) + check_rows_data([5,5],row,1) + row = execute_query("fetch next from cur",conn) + assert row == [] + execute_insert("close cur",conn) + drop_table_subscription() + +def test_cursor_since_begin(): + print(f"test_cursor_since_begin") + create_table_subscription() + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + execute_insert("insert into t1 values(4,4)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(5,5)",conn) + execute_insert("flush",conn) + execute_insert("declare cur subscription cursor for sub since begin()",conn) + execute_insert("insert into t1 values(6,6)",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([4,4],row,1) + row = execute_query("fetch next from cur",conn) + check_rows_data([5,5],row,1) + row = execute_query("fetch next from cur",conn) + check_rows_data([6,6],row,1) + row = execute_query("fetch next from cur",conn) + assert row == [] + execute_insert("close cur",conn) + drop_table_subscription() + +def test_cursor_since_now(): + print(f"test_cursor_since_now") + create_table_subscription() + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + execute_insert("insert into t1 values(4,4)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(5,5)",conn) + execute_insert("flush",conn) + execute_insert("declare cur subscription cursor for sub since now()",conn) + time.sleep(2) + execute_insert("insert into t1 values(6,6)",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([6,6],row,1) + row = execute_query("fetch next from cur",conn) + assert row == [] + execute_insert("close cur",conn) + drop_table_subscription() + +def test_cursor_since_rw_timestamp(): + print(f"test_cursor_since_rw_timestamp") + create_table_subscription() + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + execute_insert("insert into t1 values(4,4)",conn) + execute_insert("flush",conn) + execute_insert("insert into t1 values(5,5)",conn) + execute_insert("flush",conn) + execute_insert("declare cur subscription cursor for sub since begin()",conn) + execute_insert("insert into t1 values(6,6)",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + rw_timestamp_1 = row[0][0] + check_rows_data([4,4],row,1) + row = execute_query("fetch next from cur",conn) + rw_timestamp_2 = row[0][0] - 1 + check_rows_data([5,5],row,1) + row = execute_query("fetch next from cur",conn) + rw_timestamp_3 = row[0][0] + 1 + check_rows_data([6,6],row,1) + row = execute_query("fetch next from cur",conn) + assert row == [] + execute_insert("close cur",conn) + + execute_insert(f"declare cur subscription cursor for sub since {rw_timestamp_1}",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([4,4],row,1) + execute_insert("close cur",conn) + + execute_insert(f"declare cur subscription cursor for sub since {rw_timestamp_2}",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([5,5],row,1) + execute_insert("close cur",conn) + + execute_insert(f"declare cur subscription cursor for sub since {rw_timestamp_3}",conn) + row = execute_query("fetch next from cur",conn) + assert row == [] + execute_insert("close cur",conn) + + drop_table_subscription() + +def test_cursor_op(): + print(f"test_cursor_op") + create_table_subscription() + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + execute_insert("declare cur subscription cursor for sub",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([1,2],row,1) + row = execute_query("fetch next from cur",conn) + assert row == [] + + execute_insert("insert into t1 values(4,4)",conn) + execute_insert("flush",conn) + execute_insert("update t1 set v2 = 10 where v1 = 4",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([4,4],row,1) + row = execute_query("fetch next from cur",conn) + check_rows_data([4,4],row,4) + row = execute_query("fetch next from cur",conn) + check_rows_data([4,10],row,3) + row = execute_query("fetch next from cur",conn) + assert row == [] + + execute_insert("delete from t1 where v1 = 4",conn) + execute_insert("flush",conn) + row = execute_query("fetch next from cur",conn) + check_rows_data([4,10],row,2) + row = execute_query("fetch next from cur",conn) + assert row == [] + + execute_insert("close cur",conn) + drop_table_subscription() + +if __name__ == "__main__": + test_cursor_snapshot() + test_cursor_op() + test_cursor_snapshot_log_store() + test_cursor_since_rw_timestamp() + test_cursor_since_now() + test_cursor_since_begin() diff --git a/proto/catalog.proto b/proto/catalog.proto index 4c7380079e66..a3fcd1ab2a63 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -195,6 +195,7 @@ message Subscription { optional string created_at_cluster_version = 16; string subscription_from_name = 17; + optional string subscription_internal_table_name = 18; } message Connection { diff --git a/src/common/src/catalog/internal_table.rs b/src/common/src/catalog/internal_table.rs index 1e991db6975f..72e377b04d41 100644 --- a/src/common/src/catalog/internal_table.rs +++ b/src/common/src/catalog/internal_table.rs @@ -43,6 +43,13 @@ pub fn valid_table_name(table_name: &str) -> bool { !INTERNAL_TABLE_NAME.is_match(table_name) } +pub fn is_subscription_internal_table(subscription_name: &str, table_name: &str) -> bool { + let regex = + Regex::new(format!(r"__internal_{}_(\d+)_subscription_(\d+)", subscription_name).as_str()) + .unwrap(); + regex.is_match(table_name) +} + pub fn get_dist_key_in_pk_indices>( dist_key_indices: &[I], pk_indices: &[I], diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index faab5ddab2ee..a3bb869b5527 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -199,7 +199,7 @@ pub fn visit_stream_node_tables_inner( // Subscription NodeBody::Subscription(node) => { - // A Subscription should have a state table. + // A Subscription should have a log store optional!(node.log_store_table, "Subscription") } diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index 05518670b7d5..ca44fb2b76e8 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -29,8 +29,8 @@ use anyhow::{anyhow, bail, Result}; pub use resolve_id::*; use risingwave_frontend::handler::util::SourceSchemaCompatExt; use risingwave_frontend::handler::{ - close_cursor, create_index, create_mv, create_schema, create_source, create_table, create_view, - declare_cursor, drop_table, explain, fetch_cursor, variable, HandlerArgs, + create_index, create_mv, create_schema, create_source, create_table, create_view, drop_table, + explain, variable, HandlerArgs, }; use risingwave_frontend::session::SessionImpl; use risingwave_frontend::test_utils::{create_proto_file, get_explain_output, LocalFrontend}; @@ -572,16 +572,6 @@ impl TestCase { create_schema::handle_create_schema(handler_args, schema_name, if_not_exists) .await?; } - Statement::DeclareCursor { cursor_name, query } => { - declare_cursor::handle_declare_cursor(handler_args, cursor_name, *query) - .await?; - } - Statement::FetchCursor { cursor_name, count } => { - fetch_cursor::handle_fetch_cursor(handler_args, cursor_name, count).await?; - } - Statement::CloseCursor { cursor_name } => { - close_cursor::handle_close_cursor(handler_args, cursor_name).await?; - } _ => return Err(anyhow!("Unsupported statement type")), } } diff --git a/src/frontend/src/catalog/subscription_catalog.rs b/src/frontend/src/catalog/subscription_catalog.rs index 3ded154cf055..1409948e07bd 100644 --- a/src/frontend/src/catalog/subscription_catalog.rs +++ b/src/frontend/src/catalog/subscription_catalog.rs @@ -12,15 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. +use core::str::FromStr; use std::collections::{BTreeMap, HashSet}; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, TableId, UserId, OBJECT_ID_PLACEHOLDER}; +use risingwave_common::types::Interval; use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::{PbStreamJobStatus, PbSubscription}; +use thiserror_ext::AsReport; use super::OwnedByUserCatalog; +use crate::error::{ErrorCode, Result}; #[derive(Clone, Debug, PartialEq, Eq, Hash)] #[cfg_attr(test, derive(Default))] @@ -67,6 +71,7 @@ pub struct SubscriptionCatalog { pub created_at_cluster_version: Option, pub initialized_at_cluster_version: Option, + pub subscription_internal_table_name: Option, } #[derive(Clone, Copy, Debug, Default, Hash, PartialOrd, PartialEq, Eq, Ord)] @@ -98,10 +103,31 @@ impl SubscriptionCatalog { self } + pub fn get_retention_seconds(&self) -> Result { + let retention_seconds_str = self.properties.get("retention").ok_or_else(|| { + ErrorCode::InternalError("Subscription retention time not set.".to_string()) + })?; + let retention_seconds = (Interval::from_str(retention_seconds_str) + .map_err(|err| { + ErrorCode::InternalError(format!( + "Retention needs to be set in Interval format: {:?}", + err.to_report_string() + )) + })? + .epoch_in_micros() + / 1000000) as u64; + + Ok(retention_seconds) + } + pub fn create_sql(&self) -> String { self.definition.clone() } + pub fn get_log_store_name(&self) -> String { + self.subscription_internal_table_name.clone().unwrap() + } + pub fn to_proto(&self) -> PbSubscription { assert!(!self.dependent_relations.is_empty()); PbSubscription { @@ -130,6 +156,7 @@ impl SubscriptionCatalog { stream_job_status: PbStreamJobStatus::Creating.into(), initialized_at_cluster_version: self.initialized_at_cluster_version.clone(), created_at_cluster_version: self.created_at_cluster_version.clone(), + subscription_internal_table_name: self.subscription_internal_table_name.clone(), } } } @@ -165,6 +192,7 @@ impl From<&PbSubscription> for SubscriptionCatalog { initialized_at_epoch: prost.initialized_at_epoch.map(Epoch::from), created_at_cluster_version: prost.created_at_cluster_version.clone(), initialized_at_cluster_version: prost.initialized_at_cluster_version.clone(), + subscription_internal_table_name: prost.subscription_internal_table_name.clone(), } } } diff --git a/src/frontend/src/handler/close_cursor.rs b/src/frontend/src/handler/close_cursor.rs index 14a5537aea0e..1678b85f8535 100644 --- a/src/frontend/src/handler/close_cursor.rs +++ b/src/frontend/src/handler/close_cursor.rs @@ -13,20 +13,25 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_sqlparser::ast::ObjectName; +use risingwave_sqlparser::ast::CloseCursorStatement; use super::RwPgResponse; use crate::error::Result; use crate::handler::HandlerArgs; +use crate::Binder; pub async fn handle_close_cursor( - handler_args: HandlerArgs, - cursor_name: Option, + handle_args: HandlerArgs, + stmt: CloseCursorStatement, ) -> Result { - if let Some(name) = cursor_name { - handler_args.session.drop_cursor(name).await?; + let session = handle_args.session.clone(); + let cursor_manager = session.get_cursor_manager(); + let db_name = session.database(); + if let Some(cursor_name) = stmt.cursor_name { + let (_, cursor_name) = Binder::resolve_schema_qualified_name(db_name, cursor_name.clone())?; + cursor_manager.remove_cursor(cursor_name).await?; } else { - handler_args.session.drop_all_cursors().await; + cursor_manager.remove_all_cursor().await; } Ok(PgResponse::empty_result(StatementType::CLOSE_CURSOR)) } diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 1cba0e21fc24..bed409de178f 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -38,13 +38,13 @@ use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DispatcherType, MergeNode, StreamFragmentGraph, StreamNode}; use risingwave_sqlparser::ast::{ - ConnectorSchema, CreateSink, CreateSinkStatement, EmitMode, Encode, Format, ObjectName, Query, - Select, SelectItem, SetExpr, Statement, TableFactor, TableWithJoins, + ConnectorSchema, CreateSink, CreateSinkStatement, EmitMode, Encode, Format, Query, Statement, }; use risingwave_sqlparser::parser::Parser; use super::create_mv::get_column_names; use super::create_source::UPSTREAM_SOURCE_KEY; +use super::util::gen_query_from_table_name; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::catalog_service::CatalogReadGuard; @@ -66,32 +66,6 @@ use crate::stream_fragmenter::build_graph; use crate::utils::resolve_privatelink_in_with_option; use crate::{Explain, Planner, TableCatalog, WithOptions}; -pub fn gen_sink_subscription_query_from_name(from_name: ObjectName) -> Result { - let table_factor = TableFactor::Table { - name: from_name, - alias: None, - as_of: None, - }; - let from = vec![TableWithJoins { - relation: table_factor, - joins: vec![], - }]; - let select = Select { - from, - projection: vec![SelectItem::Wildcard(None)], - ..Default::default() - }; - let body = SetExpr::Select(Box::new(select)); - Ok(Query { - with: None, - body, - order_by: vec![], - limit: None, - offset: None, - fetch: None, - }) -} - // used to store result of `gen_sink_plan` pub struct SinkPlanContext { pub query: Box, @@ -119,7 +93,7 @@ pub fn gen_sink_plan( CreateSink::From(from_name) => { sink_from_table_name = from_name.0.last().unwrap().real_value(); direct_sink = true; - Box::new(gen_sink_subscription_query_from_name(from_name)?) + Box::new(gen_query_from_table_name(from_name)) } CreateSink::AsQuery(query) => { sink_from_table_name = sink_table_name.clone(); diff --git a/src/frontend/src/handler/create_subscription.rs b/src/frontend/src/handler/create_subscription.rs index f44abbaa42f0..371806cc93a4 100644 --- a/src/frontend/src/handler/create_subscription.rs +++ b/src/frontend/src/handler/create_subscription.rs @@ -20,8 +20,8 @@ use risingwave_common::catalog::UserId; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{CreateSubscriptionStatement, Query}; -use super::create_sink::gen_sink_subscription_query_from_name; use super::privilege::resolve_query_privileges; +use super::util::gen_query_from_table_name; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::subscription_catalog::SubscriptionCatalog; use crate::error::Result; @@ -54,9 +54,7 @@ pub fn gen_subscription_plan( .unwrap() .real_value() .clone(); - let query = Box::new(gen_sink_subscription_query_from_name( - stmt.subscription_from, - )?); + let query = Box::new(gen_query_from_table_name(stmt.subscription_from)); let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name.clone())?; diff --git a/src/frontend/src/handler/declare_cursor.rs b/src/frontend/src/handler/declare_cursor.rs index dda676998edd..27f029bf3e4a 100644 --- a/src/frontend/src/handler/declare_cursor.rs +++ b/src/frontend/src/handler/declare_cursor.rs @@ -12,38 +12,123 @@ // See the License for the specific language governing permissions and // limitations under the License. +use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_sqlparser::ast::{ObjectName, Query, Statement}; +use risingwave_common::util::epoch::Epoch; +use risingwave_sqlparser::ast::{DeclareCursorStatement, ObjectName, Query, Since, Statement}; use super::query::{gen_batch_plan_by_statement, gen_batch_plan_fragmenter}; +use super::util::{convert_epoch_to_logstore_i64, convert_unix_millis_to_logstore_i64}; use super::RwPgResponse; -use crate::error::Result; +use crate::error::{ErrorCode, Result}; +use crate::handler::query::create_stream; use crate::handler::HandlerArgs; -use crate::session::cursor::Cursor; -use crate::OptimizerContext; +use crate::{Binder, OptimizerContext, PgResponseStream}; pub async fn handle_declare_cursor( - handler_args: HandlerArgs, + handle_args: HandlerArgs, + stmt: DeclareCursorStatement, +) -> Result { + match stmt.declare_cursor { + risingwave_sqlparser::ast::DeclareCursor::Query(query) => { + handle_declare_query_cursor(handle_args, stmt.cursor_name, query).await + } + risingwave_sqlparser::ast::DeclareCursor::Subscription(sub_name, rw_timestamp) => { + handle_declare_subscription_cursor( + handle_args, + sub_name, + stmt.cursor_name, + rw_timestamp, + ) + .await + } + } +} +async fn handle_declare_subscription_cursor( + handle_args: HandlerArgs, + sub_name: ObjectName, cursor_name: ObjectName, - query: Query, + rw_timestamp: Option, ) -> Result { - let session = handler_args.session.clone(); + let session = handle_args.session.clone(); + let db_name = session.database(); + let (schema_name, cursor_name) = + Binder::resolve_schema_qualified_name(db_name, cursor_name.clone())?; - let plan_fragmenter_result = { - let context = OptimizerContext::from_handler_args(handler_args); - let plan_result = gen_batch_plan_by_statement( - &session, - context.into(), - Statement::Query(Box::new(query.clone())), - )?; - gen_batch_plan_fragmenter(&session, plan_result)? + let cursor_from_subscription_name = sub_name.0.last().unwrap().real_value().clone(); + let subscription = + session.get_subscription_by_name(schema_name, &cursor_from_subscription_name)?; + // Start the first query of cursor, which includes querying the table and querying the subscription's logstore + let start_rw_timestamp = match rw_timestamp { + Some(risingwave_sqlparser::ast::Since::TimestampMsNum(start_rw_timestamp)) => { + check_cursor_unix_millis(start_rw_timestamp, subscription.get_retention_seconds()?)?; + Some(convert_unix_millis_to_logstore_i64(start_rw_timestamp)) + } + Some(risingwave_sqlparser::ast::Since::ProcessTime) => { + Some(convert_epoch_to_logstore_i64(Epoch::now().0)) + } + Some(risingwave_sqlparser::ast::Since::Begin) => { + let min_unix_millis = + Epoch::now().as_unix_millis() - subscription.get_retention_seconds()? * 1000; + Some(convert_unix_millis_to_logstore_i64(min_unix_millis)) + } + None => None, }; - + // Create cursor based on the response session - .add_cursor( - cursor_name, - Cursor::new(plan_fragmenter_result, session.clone()).await?, + .get_cursor_manager() + .add_subscription_cursor( + cursor_name.clone(), + start_rw_timestamp, + subscription, + &handle_args, + ) + .await?; + + Ok(PgResponse::empty_result(StatementType::DECLARE_CURSOR)) +} + +fn check_cursor_unix_millis(unix_millis: u64, retention_seconds: u64) -> Result<()> { + let now = Epoch::now().as_unix_millis(); + let min_unix_millis = now - retention_seconds * 1000; + if unix_millis > now { + return Err(ErrorCode::CatalogError( + "rw_timestamp is too large, need to be less than the current unix_millis" + .to_string() + .into(), ) + .into()); + } + if unix_millis < min_unix_millis { + return Err(ErrorCode::CatalogError("rw_timestamp is too small, need to be large than the current unix_millis - subscription's retention time".to_string().into()).into()); + } + Ok(()) +} + +async fn handle_declare_query_cursor( + handle_args: HandlerArgs, + cursor_name: ObjectName, + query: Box, +) -> Result { + let (row_stream, pg_descs) = + create_stream_for_cursor(handle_args.clone(), Statement::Query(query)).await?; + handle_args + .session + .get_cursor_manager() + .add_query_cursor(cursor_name, row_stream, pg_descs) .await?; Ok(PgResponse::empty_result(StatementType::DECLARE_CURSOR)) } + +pub async fn create_stream_for_cursor( + handle_args: HandlerArgs, + stmt: Statement, +) -> Result<(PgResponseStream, Vec)> { + let session = handle_args.session.clone(); + let plan_fragmenter_result = { + let context = OptimizerContext::from_handler_args(handle_args); + let plan_result = gen_batch_plan_by_statement(&session, context.into(), stmt)?; + gen_batch_plan_fragmenter(&session, plan_result)? + }; + create_stream(session, plan_fragmenter_result, vec![]).await +} diff --git a/src/frontend/src/handler/fetch_cursor.rs b/src/frontend/src/handler/fetch_cursor.rs index ac1e929d5187..05305a9657b1 100644 --- a/src/frontend/src/handler/fetch_cursor.rs +++ b/src/frontend/src/handler/fetch_cursor.rs @@ -12,21 +12,36 @@ // See the License for the specific language governing permissions and // limitations under the License. +use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_sqlparser::ast::ObjectName; +use pgwire::types::Row; +use risingwave_sqlparser::ast::FetchCursorStatement; use super::RwPgResponse; use crate::error::Result; use crate::handler::HandlerArgs; +use crate::{Binder, PgResponseStream}; pub async fn handle_fetch_cursor( - handler_args: HandlerArgs, - cursor_name: ObjectName, - count: Option, + handle_args: HandlerArgs, + stmt: FetchCursorStatement, ) -> Result { - let session = handler_args.session; - let (rows, pg_descs) = session.cursor_next(&cursor_name, count).await?; - Ok(PgResponse::builder(StatementType::FETCH_CURSOR) - .values(rows.into(), pg_descs) - .into()) + let session = handle_args.session.clone(); + let db_name = session.database(); + let (_, cursor_name) = + Binder::resolve_schema_qualified_name(db_name, stmt.cursor_name.clone())?; + + let cursor_manager = session.get_cursor_manager(); + + let (rows, pg_descs) = cursor_manager + .get_rows_with_cursor(cursor_name, stmt.count, handle_args) + .await?; + Ok(build_fetch_cursor_response(rows, pg_descs)) +} + +fn build_fetch_cursor_response(rows: Vec, pg_descs: Vec) -> RwPgResponse { + PgResponse::builder(StatementType::FETCH_CURSOR) + .row_cnt_opt(Some(rows.len() as i32)) + .values(PgResponseStream::from(rows), pg_descs) + .into() } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 8b60eeeeef2b..52ad26cc8373 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -362,6 +362,15 @@ pub async fn handle( if_not_exists, } => create_schema::handle_create_schema(handler_args, schema_name, if_not_exists).await, Statement::CreateUser(stmt) => create_user::handle_create_user(handler_args, stmt).await, + Statement::DeclareCursor { stmt } => { + declare_cursor::handle_declare_cursor(handler_args, stmt).await + } + Statement::FetchCursor { stmt } => { + fetch_cursor::handle_fetch_cursor(handler_args, stmt).await + } + Statement::CloseCursor { stmt } => { + close_cursor::handle_close_cursor(handler_args, stmt).await + } Statement::AlterUser(stmt) => alter_user::handle_alter_user(handler_args, stmt).await, Statement::Grant { .. } => { handle_privilege::handle_grant_privilege(handler_args, stmt).await @@ -929,15 +938,6 @@ pub async fn handle( object_name, comment, } => comment::handle_comment(handler_args, object_type, object_name, comment).await, - Statement::DeclareCursor { cursor_name, query } => { - declare_cursor::handle_declare_cursor(handler_args, cursor_name, *query).await - } - Statement::FetchCursor { cursor_name, count } => { - fetch_cursor::handle_fetch_cursor(handler_args, cursor_name, count).await - } - Statement::CloseCursor { cursor_name } => { - close_cursor::handle_close_cursor(handler_args, cursor_name).await - } _ => bail_not_implemented!("Unhandled statement: {}", stmt), } } diff --git a/src/frontend/src/handler/transaction.rs b/src/frontend/src/handler/transaction.rs index aba0020bdf46..a9eb08b48307 100644 --- a/src/frontend/src/handler/transaction.rs +++ b/src/frontend/src/handler/transaction.rs @@ -87,7 +87,7 @@ pub async fn handle_commit( } session.txn_commit_explicit(); - session.drop_all_cursors().await; + session.get_cursor_manager().remove_all_query_cursor().await; Ok(RwPgResponse::empty_result(stmt_type)) } @@ -104,7 +104,7 @@ pub async fn handle_rollback( } session.txn_rollback_explicit(); - session.drop_all_cursors().await; + session.get_cursor_manager().remove_all_query_cursor().await; Ok(RwPgResponse::empty_result(stmt_type)) } diff --git a/src/frontend/src/handler/util.rs b/src/frontend/src/handler/util.rs index d3ccb55e6a6a..011b07895894 100644 --- a/src/frontend/src/handler/util.rs +++ b/src/frontend/src/handler/util.rs @@ -29,10 +29,15 @@ use risingwave_common::array::DataChunk; use risingwave_common::catalog::Field; use risingwave_common::row::Row as _; use risingwave_common::types::{write_date_time_tz, DataType, ScalarRefImpl, Timestamptz}; +use risingwave_common::util::epoch::Epoch; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_sqlparser::ast::{CompatibleSourceSchema, ConnectorSchema}; +use risingwave_sqlparser::ast::{ + BinaryOperator, CompatibleSourceSchema, ConnectorSchema, Expr, ObjectName, OrderByExpr, Query, + Select, SelectItem, SetExpr, TableFactor, TableWithJoins, Value, +}; use crate::error::{ErrorCode, Result as RwResult}; +use crate::session::cursor_manager::{KV_LOG_STORE_EPOCH, KV_LOG_STORE_SEQ_ID, KV_LOG_STORE_VNODE}; use crate::session::{current, SessionImpl}; pin_project! { @@ -191,6 +196,87 @@ impl CompatibleSourceSchema { } } +pub fn gen_query_from_table_name(from_name: ObjectName) -> Query { + let table_factor = TableFactor::Table { + name: from_name, + alias: None, + as_of: None, + }; + let from = vec![TableWithJoins { + relation: table_factor, + joins: vec![], + }]; + let select = Select { + from, + projection: vec![SelectItem::Wildcard(None)], + ..Default::default() + }; + let body = SetExpr::Select(Box::new(select)); + Query { + with: None, + body, + order_by: vec![], + limit: None, + offset: None, + fetch: None, + } +} + +pub fn gen_query_from_logstore_ge_rw_timestamp(logstore_name: &str, rw_timestamp: i64) -> Query { + let table_factor = TableFactor::Table { + name: ObjectName(vec![logstore_name.into()]), + alias: None, + as_of: None, + }; + let from = vec![TableWithJoins { + relation: table_factor, + joins: vec![], + }]; + let selection = Some(Expr::BinaryOp { + left: Box::new(Expr::Identifier(KV_LOG_STORE_EPOCH.into())), + op: BinaryOperator::GtEq, + right: Box::new(Expr::Value(Value::Number(rw_timestamp.to_string()))), + }); + let except_columns = vec![ + Expr::Identifier(KV_LOG_STORE_SEQ_ID.into()), + Expr::Identifier(KV_LOG_STORE_VNODE.into()), + ]; + let select = Select { + from, + projection: vec![SelectItem::Wildcard(Some(except_columns))], + selection, + ..Default::default() + }; + let order_by = vec![OrderByExpr { + expr: Expr::Identifier(KV_LOG_STORE_EPOCH.into()), + asc: None, + nulls_first: None, + }]; + let body = SetExpr::Select(Box::new(select)); + Query { + with: None, + body, + order_by, + limit: None, + offset: None, + fetch: None, + } +} + +pub fn convert_unix_millis_to_logstore_i64(unix_millis: u64) -> i64 { + let epoch = Epoch::from_unix_millis(unix_millis); + convert_epoch_to_logstore_i64(epoch.0) +} + +pub fn convert_epoch_to_logstore_i64(epoch: u64) -> i64 { + epoch as i64 ^ (1i64 << 63) +} + +pub fn convert_logstore_i64_to_unix_millis(logstore_i64: i64) -> u64 { + let epoch = Epoch::from(logstore_i64 as u64 ^ (1u64 << 63)); + epoch.as_unix_millis() +} + #[cfg(test)] mod tests { use bytes::BytesMut; diff --git a/src/frontend/src/optimizer/plan_node/stream_subscription.rs b/src/frontend/src/optimizer/plan_node/stream_subscription.rs index e9d893696158..8b165d5bbacb 100644 --- a/src/frontend/src/optimizer/plan_node/stream_subscription.rs +++ b/src/frontend/src/optimizer/plan_node/stream_subscription.rs @@ -137,6 +137,7 @@ impl StreamSubscription { created_at_epoch: None, created_at_cluster_version: None, initialized_at_cluster_version: None, + subscription_internal_table_name: None, }; Ok((input, subscription_desc)) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 2059a40acab3..dc957735e676 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -355,17 +355,7 @@ pub fn infer_kv_log_store_table_catalog_inner( let read_prefix_len_hint = table_catalog_builder.get_current_pk_len(); - let payload_indices = table_catalog_builder.extend_columns( - &columns - .iter() - .map(|column| { - // make payload hidden column visible in kv log store batch query - let mut column = column.clone(); - column.is_hidden = false; - column - }) - .collect_vec(), - ); + let payload_indices = table_catalog_builder.extend_columns(columns); value_indices.extend(payload_indices); table_catalog_builder.set_value_indices(value_indices); diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 132108636f01..36285b59b5e3 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -60,6 +60,15 @@ impl ReadSnapshot { } } + /// Get the [`Option`] value for this snapshot, only `FrontendPinned`. + pub fn epoch_with_frontend_pinned(&self) -> Option { + match self.batch_query_epoch().epoch.unwrap() { + batch_query_epoch::Epoch::Committed(epoch) + | batch_query_epoch::Epoch::Current(epoch) => Some(epoch.into()), + batch_query_epoch::Epoch::Backup(_) => None, + } + } + /// Get the [`Epoch`] value for this snapshot. pub fn epoch(&self) -> Epoch { match self.batch_query_epoch().epoch.unwrap() { diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 9ecff36106e3..7ff790748a76 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -78,10 +78,12 @@ use tokio::sync::watch; use tokio::task::JoinHandle; use tracing::info; +use self::cursor_manager::CursorManager; use crate::binder::{Binder, BoundStatement, ResolveQualifiedNameError}; use crate::catalog::catalog_service::{CatalogReader, CatalogWriter, CatalogWriterImpl}; use crate::catalog::connection_catalog::ConnectionCatalog; use crate::catalog::root_catalog::Catalog; +use crate::catalog::subscription_catalog::SubscriptionCatalog; use crate::catalog::{ check_schema_writable, CatalogError, DatabaseId, OwnedByUserCatalog, SchemaId, }; @@ -112,7 +114,7 @@ use crate::user::UserId; use crate::{FrontendOpts, PgResponseStream}; pub(crate) mod current; -pub(crate) mod cursor; +pub(crate) mod cursor_manager; pub(crate) mod transaction; /// The global environment for the frontend server. @@ -608,8 +610,7 @@ pub struct SessionImpl { /// Last idle instant last_idle_instant: Arc>>, - /// The cursors declared in the transaction. - cursors: tokio::sync::Mutex>, + cursor_manager: Arc, } #[derive(Error, Debug)] @@ -650,7 +651,7 @@ impl SessionImpl { notices: Default::default(), exec_context: Mutex::new(None), last_idle_instant: Default::default(), - cursors: Default::default(), + cursor_manager: Arc::new(CursorManager::default()), } } @@ -677,7 +678,7 @@ impl SessionImpl { )) .into(), last_idle_instant: Default::default(), - cursors: Default::default(), + cursor_manager: Arc::new(CursorManager::default()), } } @@ -747,6 +748,10 @@ impl SessionImpl { .map(|context| context.running_sql.clone()) } + pub fn get_cursor_manager(&self) -> Arc { + self.cursor_manager.clone() + } + pub fn peer_addr(&self) -> &Address { &self.peer_addr } @@ -874,6 +879,32 @@ impl SessionImpl { Ok(connection.clone()) } + pub fn get_subscription_by_name( + &self, + schema_name: Option, + subscription_name: &str, + ) -> Result> { + let db_name = self.database(); + let search_path = self.config().search_path(); + let user_name = &self.auth_context().user_name; + + let catalog_reader = self.env().catalog_reader().read_guard(); + let schema = match schema_name { + Some(schema_name) => catalog_reader.get_schema_by_name(db_name, &schema_name)?, + None => catalog_reader.first_valid_schema(db_name, &search_path, user_name)?, + }; + let schema = catalog_reader.get_schema_by_name(db_name, schema.name().as_str())?; + let subscription = schema + .get_subscription_by_name(subscription_name) + .ok_or_else(|| { + RwError::from(ErrorCode::ItemNotFound(format!( + "subscription {} not found", + subscription_name + ))) + })?; + Ok(subscription.clone()) + } + pub fn clear_cancel_query_flag(&self) { let mut flag = self.current_query_cancel_flag.lock(); *flag = None; diff --git a/src/frontend/src/session/cursor.rs b/src/frontend/src/session/cursor.rs deleted file mode 100644 index 74a54b75123a..000000000000 --- a/src/frontend/src/session/cursor.rs +++ /dev/null @@ -1,136 +0,0 @@ -// 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::VecDeque; -use std::sync::Arc; - -use futures::StreamExt; -use pgwire::pg_field_descriptor::PgFieldDescriptor; -use pgwire::pg_response::StatementType; -use pgwire::types::Row; -use risingwave_sqlparser::ast::ObjectName; - -use super::PgResponseStream; -use crate::error::{ErrorCode, Result, RwError}; -use crate::handler::query::{create_stream, BatchPlanFragmenterResult}; -use crate::session::SessionImpl; - -pub struct Cursor { - row_stream: PgResponseStream, - pg_descs: Vec, - remaining_rows: VecDeque, -} - -impl Cursor { - pub async fn new( - plan_fragmenter_result: BatchPlanFragmenterResult, - session: Arc, - ) -> Result { - assert_eq!(plan_fragmenter_result.stmt_type, StatementType::SELECT); - let (row_stream, pg_descs) = create_stream(session, plan_fragmenter_result, vec![]).await?; - Ok(Self { - row_stream, - pg_descs, - remaining_rows: VecDeque::::new(), - }) - } - - pub async fn next_once(&mut self) -> Result> { - while self.remaining_rows.is_empty() { - let rows = self.row_stream.next().await; - let rows = match rows { - None => return Ok(None), - Some(row) => { - row.map_err(|err| RwError::from(ErrorCode::InternalError(format!("{}", err))))? - } - }; - self.remaining_rows = rows.into_iter().collect(); - } - let row = self.remaining_rows.pop_front().unwrap(); - Ok(Some(row)) - } - - pub async fn next(&mut self, count: Option) -> Result> { - // `FETCH NEXT` is equivalent to `FETCH 1`. - let fetch_count = count.unwrap_or(1); - if fetch_count <= 0 { - Err(crate::error::ErrorCode::InternalError( - "FETCH a non-positive count is not supported yet".to_string(), - ) - .into()) - } else { - // min with 100 to avoid allocating too many memory at once. - let mut ans = Vec::with_capacity(std::cmp::min(100, fetch_count) as usize); - let mut cur = 0; - while cur < fetch_count - && let Some(row) = self.next_once().await? - { - cur += 1; - ans.push(row); - } - Ok(ans) - } - } - - pub fn pg_descs(&self) -> Vec { - self.pg_descs.clone() - } -} - -impl SessionImpl { - pub async fn add_cursor(&self, cursor_name: ObjectName, cursor: Cursor) -> Result<()> { - if self - .cursors - .lock() - .await - .try_insert(cursor_name.clone(), cursor) - .is_err() - { - return Err(ErrorCode::CatalogError( - format!("cursor \"{cursor_name}\" already exists").into(), - ) - .into()); - } - Ok(()) - } - - pub async fn drop_all_cursors(&self) { - self.cursors.lock().await.clear(); - } - - pub async fn drop_cursor(&self, cursor_name: ObjectName) -> Result<()> { - match self.cursors.lock().await.remove(&cursor_name) { - Some(_) => Ok(()), - None => Err(ErrorCode::CatalogError( - format!("cursor \"{cursor_name}\" does not exist").into(), - ) - .into()), - } - } - - pub async fn cursor_next( - &self, - cursor_name: &ObjectName, - count: Option, - ) -> Result<(Vec, Vec)> { - if let Some(cursor) = self.cursors.lock().await.get_mut(cursor_name) { - Ok((cursor.next(count).await?, cursor.pg_descs())) - } else { - Err( - ErrorCode::CatalogError(format!("cursor \"{cursor_name}\" does not exist").into()) - .into(), - ) - } - } -} diff --git a/src/frontend/src/session/cursor_manager.rs b/src/frontend/src/session/cursor_manager.rs new file mode 100644 index 000000000000..214dac1a2b5a --- /dev/null +++ b/src/frontend/src/session/cursor_manager.rs @@ -0,0 +1,544 @@ +// 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 core::ops::Index; +use core::time::Duration; +use std::collections::{HashMap, VecDeque}; +use std::sync::Arc; +use std::time::Instant; + +use bytes::Bytes; +use futures::StreamExt; +use itertools::Itertools; +use pgwire::pg_field_descriptor::PgFieldDescriptor; +use pgwire::types::Row; +use risingwave_common::types::DataType; +use risingwave_sqlparser::ast::{Ident, ObjectName, Statement}; + +use crate::catalog::subscription_catalog::SubscriptionCatalog; +use crate::error::{ErrorCode, Result, RwError}; +use crate::handler::declare_cursor::create_stream_for_cursor; +use crate::handler::util::{ + convert_epoch_to_logstore_i64, convert_logstore_i64_to_unix_millis, + gen_query_from_logstore_ge_rw_timestamp, gen_query_from_table_name, +}; +use crate::handler::HandlerArgs; +use crate::PgResponseStream; + +pub const KV_LOG_STORE_EPOCH: &str = "kv_log_store_epoch"; +const KV_LOG_STORE_ROW_OP: &str = "kv_log_store_row_op"; +pub const KV_LOG_STORE_SEQ_ID: &str = "kv_log_store_seq_id"; +pub const KV_LOG_STORE_VNODE: &str = "kv_log_store_vnode"; + +pub enum Cursor { + Subscription(SubscriptionCursor), + Query(QueryCursor), +} +impl Cursor { + pub async fn next( + &mut self, + count: u32, + handle_args: HandlerArgs, + ) -> Result<(Vec, Vec)> { + match self { + Cursor::Subscription(cursor) => cursor.next(count, handle_args).await, + Cursor::Query(cursor) => cursor.next(count).await, + } + } +} + +pub struct QueryCursor { + row_stream: PgResponseStream, + pg_descs: Vec, + remaining_rows: VecDeque, +} + +impl QueryCursor { + pub fn new(row_stream: PgResponseStream, pg_descs: Vec) -> Result { + Ok(Self { + row_stream, + pg_descs, + remaining_rows: VecDeque::::new(), + }) + } + + pub async fn next_once(&mut self) -> Result> { + while self.remaining_rows.is_empty() { + let rows = self.row_stream.next().await; + let rows = match rows { + None => return Ok(None), + Some(row) => { + row.map_err(|err| RwError::from(ErrorCode::InternalError(format!("{}", err))))? + } + }; + self.remaining_rows = rows.into_iter().collect(); + } + let row = self.remaining_rows.pop_front().unwrap(); + Ok(Some(row)) + } + + pub async fn next(&mut self, count: u32) -> Result<(Vec, Vec)> { + // `FETCH NEXT` is equivalent to `FETCH 1`. + // min with 100 to avoid allocating too many memory at once. + let mut ans = Vec::with_capacity(std::cmp::min(100, count) as usize); + let mut cur = 0; + while cur < count + && let Some(row) = self.next_once().await? + { + cur += 1; + ans.push(row); + } + Ok((ans, self.pg_descs.clone())) + } +} + +enum State { + InitLogStoreQuery { + // The rw_timestamp used to initiate the query to read from subscription logstore. + seek_timestamp: i64, + + // If specified, the expected_timestamp must be an exact match for the next rw_timestamp. + expected_timestamp: Option, + }, + Fetch { + // Whether the query is reading from snapshot + // true: read from the upstream table snapshot + // false: read from subscription logstore + from_snapshot: bool, + + // The rw_timestamp used to initiate the query to read from subscription logstore. + rw_timestamp: i64, + + // The row stream to from the batch query read. + // It is returned from the batch execution. + row_stream: PgResponseStream, + + // The pg descs to from the batch query read. + // It is returned from the batch execution. + pg_descs: Vec, + + // A cache to store the remaining rows from the row stream. + remaining_rows: VecDeque, + }, + Invalid, +} + +pub struct SubscriptionCursor { + cursor_name: String, + subscription: Arc, + cursor_need_drop_time: Instant, + state: State, +} + +impl SubscriptionCursor { + pub async fn new( + cursor_name: String, + start_timestamp: Option, + subscription: Arc, + handle_args: &HandlerArgs, + ) -> Result { + let state = if let Some(start_timestamp) = start_timestamp { + State::InitLogStoreQuery { + seek_timestamp: start_timestamp, + expected_timestamp: None, + } + } else { + // The query stream needs to initiated on cursor creation to make sure + // future fetch on the cursor starts from the snapshot when the cursor is declared. + // + // TODO: is this the right behavior? Should we delay the query stream initiation till the first fetch? + let (row_stream, pg_descs) = + Self::initiate_query(None, &subscription, handle_args.clone()).await?; + let pinned_epoch = handle_args + .session + .get_pinned_snapshot() + .ok_or_else(|| { + ErrorCode::InternalError("Fetch Cursor can't find snapshot epoch".to_string()) + })? + .epoch_with_frontend_pinned() + .ok_or_else(|| { + ErrorCode::InternalError( + "Fetch Cursor can't support setting an epoch".to_string(), + ) + })? + .0; + let start_timestamp = convert_epoch_to_logstore_i64(pinned_epoch); + + State::Fetch { + from_snapshot: true, + rw_timestamp: start_timestamp, + row_stream, + pg_descs, + remaining_rows: VecDeque::new(), + } + }; + + let cursor_need_drop_time = + Instant::now() + Duration::from_secs(subscription.get_retention_seconds()?); + Ok(Self { + cursor_name, + subscription, + cursor_need_drop_time, + state, + }) + } + + pub async fn next_row( + &mut self, + handle_args: HandlerArgs, + ) -> Result<(Option, Vec)> { + loop { + match &mut self.state { + State::InitLogStoreQuery { + seek_timestamp, + expected_timestamp, + } => { + let from_snapshot = false; + + // Initiate a new batch query to continue fetching + let (mut row_stream, pg_descs) = Self::initiate_query( + Some(*seek_timestamp), + &self.subscription, + handle_args.clone(), + ) + .await?; + self.cursor_need_drop_time = Instant::now() + + Duration::from_secs(self.subscription.get_retention_seconds()?); + + // Try refill remaining rows + let mut remaining_rows = VecDeque::new(); + Self::try_refill_remaining_rows(&mut row_stream, &mut remaining_rows).await?; + + // Get the rw_timestamp in the first row returned by the query if any. + // new_row_rw_timestamp == None means the query returns empty result. + let new_row_rw_timestamp: Option = remaining_rows.front().map(|row| { + std::str::from_utf8(row.index(0).as_ref().unwrap()) + .unwrap() + .parse() + .unwrap() + }); + + // Check expected_timestamp against the rw_timestamp of the first row. + // Return an error if there is no new row or there is a mismatch. + if let Some(expected_timestamp) = expected_timestamp { + let expected_timestamp = *expected_timestamp; + if new_row_rw_timestamp.is_none() + || new_row_rw_timestamp.unwrap() != expected_timestamp + { + // Transition to Invalid state and return and error + self.state = State::Invalid; + return Err(ErrorCode::CatalogError( + format!( + " No data found for rw_timestamp {:?}, data may have been recycled, please recreate cursor", + convert_logstore_i64_to_unix_millis(expected_timestamp) + ) + .into(), + ) + .into()); + } + } + + // Return None if no data is found for the rw_timestamp in logstore. + // This happens when reaching EOF of logstore. This check cannot be moved before the + // expected_timestamp check to ensure that an error is returned on empty result when + // expected_timstamp is set. + if new_row_rw_timestamp.is_none() { + return Ok((None, pg_descs)); + } + + // Transition to the Fetch state + self.state = State::Fetch { + from_snapshot, + rw_timestamp: new_row_rw_timestamp.unwrap(), + row_stream, + pg_descs, + remaining_rows, + }; + } + State::Fetch { + from_snapshot, + rw_timestamp, + row_stream, + pg_descs, + remaining_rows, + } => { + let from_snapshot = *from_snapshot; + let rw_timestamp = *rw_timestamp; + + // Try refill remaining rows + Self::try_refill_remaining_rows(row_stream, remaining_rows).await?; + + if let Some(row) = remaining_rows.pop_front() { + // 1. Fetch the next row + let new_row = row.take(); + if from_snapshot { + // 1a. The rw_timestamp in the table is all the same, so don't need to check. + return Ok(( + Some(Row::new(Self::build_row_with_snapshot(new_row))), + pg_descs.clone(), + )); + } + + let new_row_rw_timestamp: i64 = new_row + .get(0) + .unwrap() + .as_ref() + .map(|bytes| std::str::from_utf8(bytes).unwrap().parse().unwrap()) + .unwrap(); + + if new_row_rw_timestamp != rw_timestamp { + // 1b. Find the next rw_timestamp. + // Initiate a new batch query to avoid query timeout and pinning version for too long. + // expected_timestamp shouold be set to ensure there is no data missing in the next query. + self.state = State::InitLogStoreQuery { + seek_timestamp: new_row_rw_timestamp, + expected_timestamp: Some(new_row_rw_timestamp), + }; + } else { + // 1c. The rw_timestamp of this row is equal to self.rw_timestamp, return row + return Ok(( + Some(Row::new(Self::build_row_with_logstore( + new_row, + rw_timestamp, + )?)), + pg_descs.clone(), + )); + } + } else { + // 2. Reach EOF for the current query. + // Initiate a new batch query using the rw_timestamp + 1. + // expected_timestamp don't need to be set as the next rw_timestamp is unknown. + self.state = State::InitLogStoreQuery { + seek_timestamp: rw_timestamp + 1, + expected_timestamp: None, + }; + } + } + State::Invalid => { + // TODO: auto close invalid cursor? + return Err(ErrorCode::InternalError( + "Cursor is in invalid state. Please close and re-create the cursor." + .to_string(), + ) + .into()); + } + } + } + } + + pub async fn next( + &mut self, + count: u32, + handle_args: HandlerArgs, + ) -> Result<(Vec, Vec)> { + if Instant::now() > self.cursor_need_drop_time { + return Err(ErrorCode::InternalError( + "The cursor has exceeded its maximum lifetime, please recreate it (close then declare cursor).".to_string(), + ) + .into()); + } + // `FETCH NEXT` is equivalent to `FETCH 1`. + if count != 1 { + Err(crate::error::ErrorCode::InternalError( + "FETCH count with subscription is not supported".to_string(), + ) + .into()) + } else { + let (row, pg_descs) = self.next_row(handle_args).await?; + if let Some(row) = row { + Ok((vec![row], pg_descs)) + } else { + Ok((vec![], pg_descs)) + } + } + } + + async fn initiate_query( + rw_timestamp: Option, + subscription: &SubscriptionCatalog, + handle_args: HandlerArgs, + ) -> Result<(PgResponseStream, Vec)> { + let query_stmt = if let Some(rw_timestamp) = rw_timestamp { + Statement::Query(Box::new(gen_query_from_logstore_ge_rw_timestamp( + &subscription.get_log_store_name(), + rw_timestamp, + ))) + } else { + let subscription_from_table_name = ObjectName(vec![Ident::from( + subscription.subscription_from_name.as_ref(), + )]); + Statement::Query(Box::new(gen_query_from_table_name( + subscription_from_table_name, + ))) + }; + let (row_stream, pg_descs) = create_stream_for_cursor(handle_args, query_stmt).await?; + Ok(( + row_stream, + Self::build_desc(pg_descs, rw_timestamp.is_none()), + )) + } + + async fn try_refill_remaining_rows( + row_stream: &mut PgResponseStream, + remaining_rows: &mut VecDeque, + ) -> Result<()> { + if remaining_rows.is_empty() + && let Some(row_set) = row_stream.next().await + { + remaining_rows.extend(row_set.map_err(|e| { + ErrorCode::InternalError(format!("Cursor get next chunk error {:?}", e.to_string())) + })?); + } + Ok(()) + } + + pub fn build_row_with_snapshot(row: Vec>) -> Vec> { + let mut new_row = vec![None, Some(Bytes::from(1i16.to_string()))]; + new_row.extend(row); + new_row + } + + pub fn build_row_with_logstore( + mut row: Vec>, + rw_timestamp: i64, + ) -> Result>> { + let mut new_row = vec![Some(Bytes::from( + convert_logstore_i64_to_unix_millis(rw_timestamp).to_string(), + ))]; + // need remove kv_log_store_epoch + new_row.extend(row.drain(1..row.len()).collect_vec()); + Ok(new_row) + } + + pub fn build_desc( + mut descs: Vec, + from_snapshot: bool, + ) -> Vec { + let mut new_descs = vec![ + PgFieldDescriptor::new( + "rw_timestamp".to_owned(), + DataType::Int64.to_oid(), + DataType::Int64.type_len(), + ), + PgFieldDescriptor::new( + "op".to_owned(), + DataType::Int16.to_oid(), + DataType::Int16.type_len(), + ), + ]; + // need remove kv_log_store_epoch and kv_log_store_row_op + if from_snapshot { + new_descs.extend(descs) + } else { + assert_eq!( + descs.get(0).unwrap().get_name(), + KV_LOG_STORE_EPOCH, + "Cursor query logstore: first column must be {}", + KV_LOG_STORE_EPOCH + ); + assert_eq!( + descs.get(1).unwrap().get_name(), + KV_LOG_STORE_ROW_OP, + "Cursor query logstore: first column must be {}", + KV_LOG_STORE_ROW_OP + ); + new_descs.extend(descs.drain(2..descs.len())); + } + new_descs + } +} + +#[derive(Default)] +pub struct CursorManager { + cursor_map: tokio::sync::Mutex>, +} + +impl CursorManager { + pub async fn add_subscription_cursor( + &self, + cursor_name: String, + start_timestamp: Option, + subscription: Arc, + handle_args: &HandlerArgs, + ) -> Result<()> { + let cursor = SubscriptionCursor::new( + cursor_name.clone(), + start_timestamp, + subscription, + handle_args, + ) + .await?; + self.cursor_map + .lock() + .await + .try_insert(cursor.cursor_name.clone(), Cursor::Subscription(cursor)) + .map_err(|_| { + ErrorCode::CatalogError(format!("cursor `{}` already exists", cursor_name).into()) + })?; + Ok(()) + } + + pub async fn add_query_cursor( + &self, + cursor_name: ObjectName, + row_stream: PgResponseStream, + pg_descs: Vec, + ) -> Result<()> { + let cursor = QueryCursor::new(row_stream, pg_descs)?; + self.cursor_map + .lock() + .await + .try_insert(cursor_name.to_string(), Cursor::Query(cursor)) + .map_err(|_| { + ErrorCode::CatalogError(format!("cursor `{}` already exists", cursor_name).into()) + })?; + + Ok(()) + } + + pub async fn remove_cursor(&self, cursor_name: String) -> Result<()> { + self.cursor_map + .lock() + .await + .remove(&cursor_name) + .ok_or_else(|| { + ErrorCode::CatalogError(format!("cursor `{}` don't exists", cursor_name).into()) + })?; + Ok(()) + } + + pub async fn remove_all_cursor(&self) { + self.cursor_map.lock().await.clear(); + } + + pub async fn remove_all_query_cursor(&self) { + self.cursor_map + .lock() + .await + .retain(|_, v| matches!(v, Cursor::Subscription(_))); + } + + pub async fn get_rows_with_cursor( + &self, + cursor_name: String, + count: u32, + handle_args: HandlerArgs, + ) -> Result<(Vec, Vec)> { + if let Some(cursor) = self.cursor_map.lock().await.get_mut(&cursor_name) { + cursor.next(count, handle_args).await + } else { + Err(ErrorCode::ItemNotFound(format!("Cannot find cursor `{}`", cursor_name)).into()) + } + } +} diff --git a/src/frontend/src/session/transaction.rs b/src/frontend/src/session/transaction.rs index 2d4a2484013a..cdc2b0e7d692 100644 --- a/src/frontend/src/session/transaction.rs +++ b/src/frontend/src/session/transaction.rs @@ -195,6 +195,10 @@ impl SessionImpl { }) } + pub fn get_pinned_snapshot(&self) -> Option { + self.txn_ctx().snapshot.clone() + } + /// Unpin snapshot by replacing the snapshot with None. pub fn unpin_snapshot(&self) { self.txn_ctx().snapshot = None; diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index c82a54cbc0e6..724355358b02 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -7,6 +7,7 @@ mod m20231008_020431_hummock; mod m20240304_074901_subscription; mod m20240410_082733_with_version_column_migration; mod m20240410_154406_session_params; +mod m20240417_062305_subscription_internal_table_name; pub struct Migrator; @@ -19,6 +20,7 @@ impl MigratorTrait for Migrator { Box::new(m20240304_074901_subscription::Migration), Box::new(m20240410_082733_with_version_column_migration::Migration), Box::new(m20240410_154406_session_params::Migration), + Box::new(m20240417_062305_subscription_internal_table_name::Migration), ] } } diff --git a/src/meta/model_v2/migration/src/m20240417_062305_subscription_internal_table_name.rs b/src/meta/model_v2/migration/src/m20240417_062305_subscription_internal_table_name.rs new file mode 100644 index 000000000000..a4c6f60928c9 --- /dev/null +++ b/src/meta/model_v2/migration/src/m20240417_062305_subscription_internal_table_name.rs @@ -0,0 +1,41 @@ +use sea_orm_migration::prelude::{Table as MigrationTable, *}; +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + MigrationTable::alter() + .table(Subscription::Table) + .add_column( + ColumnDef::new(Subscription::SubscriptionInternalTableName).integer(), + ) + .to_owned(), + ) + .await?; + + Ok(()) + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + MigrationTable::alter() + .table(Subscription::Table) + .drop_column(Alias::new( + Subscription::SubscriptionInternalTableName.to_string(), + )) + .to_owned(), + ) + .await?; + Ok(()) + } +} + +#[derive(DeriveIden)] +enum Subscription { + Table, + SubscriptionInternalTableName, +} diff --git a/src/meta/model_v2/src/subscription.rs b/src/meta/model_v2/src/subscription.rs index 096c63078a2a..8a695c2b4c65 100644 --- a/src/meta/model_v2/src/subscription.rs +++ b/src/meta/model_v2/src/subscription.rs @@ -30,6 +30,7 @@ pub struct Model { pub properties: Property, pub definition: String, pub subscription_from_name: String, + pub subscription_internal_table_name: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -63,6 +64,7 @@ impl From for ActiveModel { properties: Set(pb_subscription.properties.into()), definition: Set(pb_subscription.definition), subscription_from_name: Set(pb_subscription.subscription_from_name), + subscription_internal_table_name: Set(pb_subscription.subscription_internal_table_name), } } } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index bddc82d372d6..111e8e5ab9fe 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -18,7 +18,9 @@ use std::sync::Arc; use anyhow::anyhow; use itertools::Itertools; -use risingwave_common::catalog::{TableOption, DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMAS}; +use risingwave_common::catalog::{ + is_subscription_internal_table, TableOption, DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMAS, +}; use risingwave_common::util::stream_graph_visitor::visit_stream_node_cont; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::source::UPSTREAM_SOURCE_KEY; @@ -854,10 +856,10 @@ impl CatalogController { .all(&txn) .await?; let mut relations = internal_table_objs - .into_iter() + .iter() .map(|(table, obj)| PbRelation { relation_info: Some(PbRelationInfo::Table( - ObjectModel(table, obj.unwrap()).into(), + ObjectModel(table.clone(), obj.clone().unwrap()).into(), )), }) .collect_vec(); @@ -900,11 +902,21 @@ impl CatalogController { }); } ObjectType::Subscription => { - let (subscription, obj) = Subscription::find_by_id(job_id) + let (mut subscription, obj) = Subscription::find_by_id(job_id) .find_also_related(Object) .one(&txn) .await? .ok_or_else(|| MetaError::catalog_id_not_found("subscription", job_id))?; + let log_store_names: Vec<_> = internal_table_objs + .iter() + .filter(|a| is_subscription_internal_table(&subscription.name, &a.0.name)) + .map(|a| &a.0.name) + .collect(); + if log_store_names.len() != 1 { + bail!("A subscription can only have one log_store_name"); + } + subscription.subscription_internal_table_name = + log_store_names.get(0).cloned().cloned(); relations.push(PbRelation { relation_info: Some(PbRelationInfo::Subscription( ObjectModel(subscription, obj.unwrap()).into(), diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 645a580d8445..ff9845988839 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -240,6 +240,7 @@ impl From> for PbSubscription { subscription_from_name: value.0.subscription_from_name, initialized_at_cluster_version: value.1.initialized_at_cluster_version, created_at_cluster_version: value.1.created_at_cluster_version, + subscription_internal_table_name: value.0.subscription_internal_table_name, } } } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 6eae315bac72..5a316696e619 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -27,8 +27,8 @@ pub use database::*; pub use fragment::*; use itertools::Itertools; use risingwave_common::catalog::{ - valid_table_name, TableId as StreamingJobId, TableOption, DEFAULT_DATABASE_NAME, - DEFAULT_SCHEMA_NAME, DEFAULT_SUPER_USER, DEFAULT_SUPER_USER_FOR_PG, + is_subscription_internal_table, valid_table_name, TableId as StreamingJobId, TableOption, + DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, DEFAULT_SUPER_USER, DEFAULT_SUPER_USER_FOR_PG, DEFAULT_SUPER_USER_FOR_PG_ID, DEFAULT_SUPER_USER_ID, SYSTEM_SCHEMAS, }; use risingwave_common::{bail, ensure}; @@ -3231,6 +3231,15 @@ impl CatalogManager { subscription.schema_id, subscription.name.clone(), ); + let log_store_names: Vec<_> = internal_tables + .iter() + .filter(|a| is_subscription_internal_table(&subscription.name, a.get_name())) + .map(|a| a.get_name()) + .collect(); + if log_store_names.len() != 1 { + bail!("A subscription can only have one log_store_name"); + } + subscription.subscription_internal_table_name = log_store_names.get(0).cloned().cloned(); let mut tables = BTreeMapTransaction::new(&mut database_core.tables); let mut subscriptions = BTreeMapTransaction::new(&mut database_core.subscriptions); assert!( diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 82b59d1bf462..79c12c1b6cc6 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1286,6 +1286,22 @@ pub enum Statement { append_only: bool, params: CreateFunctionBody, }, + + /// DECLARE CURSOR + DeclareCursor { + stmt: DeclareCursorStatement, + }, + + // FETCH CURSOR + FetchCursor { + stmt: FetchCursorStatement, + }, + + // CLOSE CURSOR + CloseCursor { + stmt: CloseCursorStatement, + }, + /// ALTER DATABASE AlterDatabase { name: ObjectName, @@ -1497,21 +1513,6 @@ pub enum Statement { param: Ident, value: SetVariableValue, }, - /// DECLARE CURSOR - DeclareCursor { - cursor_name: ObjectName, - query: Box, - }, - /// FETCH FROM CURSOR - FetchCursor { - cursor_name: ObjectName, - /// Number of rows to fetch. `None` means `FETCH ALL`. - count: Option, - }, - /// CLOSE CURSOR - CloseCursor { - cursor_name: Option, - }, /// FLUSH the current barrier. /// /// Note: RisingWave specific statement. @@ -1854,6 +1855,9 @@ impl fmt::Display for Statement { Statement::CreateSink { stmt } => write!(f, "CREATE SINK {}", stmt,), Statement::CreateSubscription { stmt } => write!(f, "CREATE SUBSCRIPTION {}", stmt,), Statement::CreateConnection { stmt } => write!(f, "CREATE CONNECTION {}", stmt,), + Statement::DeclareCursor { stmt } => write!(f, "DECLARE {}", stmt,), + Statement::FetchCursor { stmt } => write!(f, "FETCH {}", stmt), + Statement::CloseCursor { stmt } => write!(f, "CLOSE {}", stmt), Statement::AlterDatabase { name, operation } => { write!(f, "ALTER DATABASE {} {}", name, operation) } @@ -2072,23 +2076,6 @@ impl fmt::Display for Statement { "{param} = {value}", ) } - Statement::DeclareCursor { cursor_name, query } => { - write!(f, "DECLARE {} CURSOR FOR {}", cursor_name, query) - }, - Statement::FetchCursor { cursor_name , count} => { - if let Some(count) = count { - write!(f, "FETCH {} FROM {}", count, cursor_name) - } else { - write!(f, "FETCH NEXT FROM {}", cursor_name) - } - }, - Statement::CloseCursor { cursor_name } => { - if let Some(name) = cursor_name { - write!(f, "CLOSE {}", name) - } else { - write!(f, "CLOSE ALL") - } - } Statement::Flush => { write!(f, "FLUSH") } diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index ff50c31a29d4..a821cd77e70e 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -634,6 +634,147 @@ impl fmt::Display for CreateSubscriptionStatement { } } +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub enum DeclareCursor { + Query(Box), + Subscription(ObjectName, Option), +} + +impl fmt::Display for DeclareCursor { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let mut v: Vec = vec![]; + match self { + DeclareCursor::Query(query) => v.push(format!("{}", query.as_ref())), + DeclareCursor::Subscription(name, since) => { + v.push(format!("{}", name)); + v.push(format!("{:?}", since)); + } + } + v.iter().join(" ").fmt(f) + } +} +// sql_grammar!(DeclareCursorStatement { +// cursor_name: Ident, +// [Keyword::SUBSCRIPTION] +// [Keyword::CURSOR], +// [Keyword::FOR], +// subscription: Ident or query: Query, +// [Keyword::SINCE], +// rw_timestamp: Ident, +// }); +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub struct DeclareCursorStatement { + pub cursor_name: ObjectName, + pub declare_cursor: DeclareCursor, +} + +impl ParseTo for DeclareCursorStatement { + fn parse_to(p: &mut Parser) -> Result { + impl_parse_to!(cursor_name: ObjectName, p); + + let declare_cursor = if !p.parse_keyword(Keyword::SUBSCRIPTION) { + p.expect_keyword(Keyword::CURSOR)?; + p.expect_keyword(Keyword::FOR)?; + DeclareCursor::Query(Box::new(p.parse_query()?)) + } else { + p.expect_keyword(Keyword::CURSOR)?; + p.expect_keyword(Keyword::FOR)?; + let cursor_for_name = p.parse_object_name()?; + let rw_timestamp = p.parse_since()?; + DeclareCursor::Subscription(cursor_for_name, rw_timestamp) + }; + + Ok(Self { + cursor_name, + declare_cursor, + }) + } +} +impl fmt::Display for DeclareCursorStatement { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let mut v: Vec = vec![]; + impl_fmt_display!(cursor_name, v, self); + v.push("CURSOR FOR ".to_string()); + impl_fmt_display!(declare_cursor, v, self); + v.iter().join(" ").fmt(f) + } +} + +// sql_grammar!(FetchCursorStatement { +// cursor_name: Ident, +// }); +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub struct FetchCursorStatement { + pub cursor_name: ObjectName, + pub count: u32, +} + +impl ParseTo for FetchCursorStatement { + fn parse_to(p: &mut Parser) -> Result { + let count = if p.parse_keyword(Keyword::NEXT) { + 1 + } else { + let count_str = p.parse_number_value()?; + count_str.parse::().map_err(|e| { + ParserError::ParserError(format!("Could not parse '{}' as i32: {}", count_str, e)) + })? + }; + p.expect_keyword(Keyword::FROM)?; + impl_parse_to!(cursor_name: ObjectName, p); + + Ok(Self { cursor_name, count }) + } +} + +impl fmt::Display for FetchCursorStatement { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let mut v: Vec = vec![]; + if self.count == 1 { + v.push("NEXT ".to_string()); + } else { + impl_fmt_display!(count, v, self); + } + v.push("FROM ".to_string()); + impl_fmt_display!(cursor_name, v, self); + v.iter().join(" ").fmt(f) + } +} + +// sql_grammar!(CloseCursorStatement { +// cursor_name: Ident, +// }); +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub struct CloseCursorStatement { + pub cursor_name: Option, +} + +impl ParseTo for CloseCursorStatement { + fn parse_to(p: &mut Parser) -> Result { + let cursor_name = if p.parse_keyword(Keyword::ALL) { + None + } else { + Some(p.parse_object_name()?) + }; + + Ok(Self { cursor_name }) + } +} +impl fmt::Display for CloseCursorStatement { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let mut v: Vec = vec![]; + if let Some(cursor_name) = &self.cursor_name { + v.push(format!("{}", cursor_name)); + } else { + v.push("ALL".to_string()); + } + v.iter().join(" ").fmt(f) + } +} + // sql_grammar!(CreateConnectionStatement { // if_not_exists => [Keyword::IF, Keyword::NOT, Keyword::EXISTS], // connection_name: Ident, @@ -708,6 +849,25 @@ impl fmt::Display for WithProperties { } } +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub enum Since { + TimestampMsNum(u64), + ProcessTime, + Begin, +} + +impl fmt::Display for Since { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + use Since::*; + match self { + TimestampMsNum(ts) => write!(f, " SINCE {}", ts), + ProcessTime => write!(f, " SINCE PROCTIME()"), + Begin => write!(f, " SINCE BEGIN()"), + } + } +} + #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub struct RowSchemaLocation { diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 208a642eb484..5d282427d453 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -459,6 +459,7 @@ define_keywords!( SETS, SHOW, SIMILAR, + SINCE, SINK, SINKS, SMALLINT, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index aaf400b43879..92009ecb016e 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -241,6 +241,9 @@ impl Parser { self.prev_token(); Ok(Statement::Query(Box::new(self.parse_query()?))) } + Keyword::DECLARE => Ok(self.parse_declare()?), + Keyword::FETCH => Ok(self.parse_fetch_cursor()?), + Keyword::CLOSE => Ok(self.parse_close_cursor()?), Keyword::TRUNCATE => Ok(self.parse_truncate()?), Keyword::CREATE => Ok(self.parse_create()?), Keyword::DROP => Ok(self.parse_drop()?), @@ -278,9 +281,6 @@ impl Parser { Keyword::EXECUTE => Ok(self.parse_execute()?), Keyword::PREPARE => Ok(self.parse_prepare()?), Keyword::COMMENT => Ok(self.parse_comment()?), - Keyword::DECLARE => Ok(self.parse_declare_cursor()?), - Keyword::FETCH => Ok(self.parse_fetch_cursor()?), - Keyword::CLOSE => Ok(self.parse_close_cursor()?), Keyword::FLUSH => Ok(Statement::Flush), Keyword::WAIT => Ok(Statement::Wait), Keyword::RECOVER => Ok(Statement::Recover), @@ -2362,6 +2362,24 @@ impl Parser { }) } + pub fn parse_declare(&mut self) -> Result { + Ok(Statement::DeclareCursor { + stmt: DeclareCursorStatement::parse_to(self)?, + }) + } + + pub fn parse_fetch_cursor(&mut self) -> Result { + Ok(Statement::FetchCursor { + stmt: FetchCursorStatement::parse_to(self)?, + }) + } + + pub fn parse_close_cursor(&mut self) -> Result { + Ok(Statement::CloseCursor { + stmt: CloseCursorStatement::parse_to(self)?, + }) + } + fn parse_table_column_def(&mut self) -> Result { Ok(TableColumnDef { name: self.parse_identifier_non_reserved()?, @@ -2988,6 +3006,44 @@ impl Parser { Ok(SqlOption { name, value }) } + pub fn parse_since(&mut self) -> Result, ParserError> { + if self.parse_keyword(Keyword::SINCE) { + let token = self.next_token(); + match token.token { + Token::Word(w) => { + let ident = w.to_ident()?; + // Backward compatibility for now. + if ident.real_value() == "proctime" || ident.real_value() == "now" { + self.expect_token(&Token::LParen)?; + self.expect_token(&Token::RParen)?; + Ok(Some(Since::ProcessTime)) + } else if ident.real_value() == "begin" { + self.expect_token(&Token::LParen)?; + self.expect_token(&Token::RParen)?; + Ok(Some(Since::Begin)) + } else { + parser_err!(format!( + "Expected proctime(), begin() or now(), found: {}", + ident.real_value() + )) + } + } + Token::Number(s) => { + let num = s.parse::().map_err(|e| { + ParserError::ParserError(format!("Could not parse '{}' as u64: {}", s, e)) + }); + Ok(Some(Since::TimestampMsNum(num?))) + } + unexpected => self.expected( + "proctime(), begin() , now(), Number", + unexpected.with_location(token.location), + ), + } + } else { + Ok(None) + } + } + pub fn parse_emit_mode(&mut self) -> Result, ParserError> { if self.parse_keyword(Keyword::EMIT) { match self.parse_one_of_keywords(&[Keyword::IMMEDIATELY, Keyword::ON]) { @@ -3394,7 +3450,6 @@ impl Parser { self.peek_token(), ); } - let value = self.parse_set_variable()?; let deferred = self.parse_keyword(Keyword::DEFERRED); @@ -5427,40 +5482,6 @@ impl Parser { comment, }) } - - /// Parse a SQL DECLARE statement - pub fn parse_declare_cursor(&mut self) -> Result { - let cursor_name = self.parse_object_name()?; - self.expect_keyword(Keyword::CURSOR)?; - self.expect_keyword(Keyword::FOR)?; - let query = Box::new(self.parse_query()?); - Ok(Statement::DeclareCursor { cursor_name, query }) - } - - /// Parse a SQL FETCH statement - pub fn parse_fetch_cursor(&mut self) -> Result { - let count = if self.parse_keyword(Keyword::NEXT) { - None - } else { - let count_str = self.parse_number_value()?; - Some(count_str.parse::().map_err(|e| { - ParserError::ParserError(format!("Could not parse '{}' as i32: {}", count_str, e)) - })?) - }; - self.expect_keyword(Keyword::FROM)?; - let cursor_name = self.parse_object_name()?; - Ok(Statement::FetchCursor { cursor_name, count }) - } - - /// Parse a SQL CLOSE statement - pub fn parse_close_cursor(&mut self) -> Result { - let cursor_name = if self.parse_keyword(Keyword::ALL) { - None - } else { - Some(self.parse_object_name()?) - }; - Ok(Statement::CloseCursor { cursor_name }) - } } impl Word { diff --git a/src/stream/src/common/log_store_impl/subscription_log_store.rs b/src/stream/src/common/log_store_impl/subscription_log_store.rs index 39ada926826d..c7de7073f85e 100644 --- a/src/stream/src/common/log_store_impl/subscription_log_store.rs +++ b/src/stream/src/common/log_store_impl/subscription_log_store.rs @@ -18,7 +18,6 @@ use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::TableId; -use risingwave_common::hash::VnodeBitmapExt; use risingwave_connector::sink::log_store::LogStoreResult; use risingwave_hummock_sdk::table_watermark::{VnodeWatermark, WatermarkDirection}; use risingwave_storage::store::{InitOptions, LocalStateStore, SealCurrentEpochOptions}; @@ -83,15 +82,9 @@ impl SubscriptionLogStoreWriter { pub async fn flush_current_epoch( &mut self, next_epoch: u64, - is_checkpoint: bool, truncate_offset: Option, ) -> LogStoreResult<()> { - let epoch = self.state_store.epoch(); - for vnode in self.serde.vnodes().iter_vnodes() { - let (key, value) = self.serde.serialize_barrier(epoch, vnode, is_checkpoint); - self.state_store.insert(key, value, None)?; - } - + // Because barrier has no effect on subscription, barrier will not be inserted here let watermark = truncate_offset.map(|truncate_offset| { VnodeWatermark::new( self.serde.vnodes().clone(), diff --git a/src/stream/src/executor/subscription.rs b/src/stream/src/executor/subscription.rs index 8eed385a2b59..cd752dfdf28c 100644 --- a/src/stream/src/executor/subscription.rs +++ b/src/stream/src/executor/subscription.rs @@ -114,11 +114,7 @@ impl SubscriptionExecutor { None }; self.log_store - .flush_current_epoch( - barrier.epoch.curr, - barrier.kind.is_checkpoint(), - truncate_offset, - ) + .flush_current_epoch(barrier.epoch.curr, truncate_offset) .await?; if let Some(vnode_bitmap) = diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 1431909f5366..aeb08b5f0842 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -42,6 +42,7 @@ pub enum StatementType { FETCH, COPY, EXPLAIN, + CLOSE_CURSOR, CREATE_TABLE, CREATE_MATERIALIZED_VIEW, CREATE_VIEW, @@ -55,6 +56,7 @@ pub enum StatementType { CREATE_FUNCTION, CREATE_CONNECTION, COMMENT, + DECLARE_CURSOR, DESCRIBE, GRANT_PRIVILEGE, DROP_TABLE, @@ -100,9 +102,7 @@ pub enum StatementType { ROLLBACK, SET_TRANSACTION, CANCEL_COMMAND, - DECLARE_CURSOR, FETCH_CURSOR, - CLOSE_CURSOR, WAIT, KILL, RECOVER, diff --git a/src/utils/pgwire/src/types.rs b/src/utils/pgwire/src/types.rs index 45f289b053e6..d4d37e1168ea 100644 --- a/src/utils/pgwire/src/types.rs +++ b/src/utils/pgwire/src/types.rs @@ -45,6 +45,10 @@ impl Row { pub fn values(&self) -> &[Option] { &self.0 } + + pub fn take(self) -> Vec> { + self.0 + } } impl Index for Row { From 2a2cf33034fafa2eebcd1eb1dd125371dd0a5987 Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Thu, 18 Apr 2024 16:15:50 +0800 Subject: [PATCH 55/64] refactor(frontend): don't use OptimizerContext in async context (#16284) Signed-off-by: TennyZhuang --- src/frontend/src/handler/create_connection.rs | 7 +- src/frontend/src/handler/create_source.rs | 7 +- src/frontend/src/handler/create_table.rs | 90 +++++++++++-------- src/frontend/src/handler/explain.rs | 26 +++--- .../src/optimizer/optimizer_context.rs | 7 ++ src/frontend/src/utils/overwrite_options.rs | 2 +- src/frontend/src/utils/with_options.rs | 9 ++ 7 files changed, 85 insertions(+), 63 deletions(-) diff --git a/src/frontend/src/handler/create_connection.rs b/src/frontend/src/handler/create_connection.rs index 40fd6f0d3729..22491f9cb0ee 100644 --- a/src/frontend/src/handler/create_connection.rs +++ b/src/frontend/src/handler/create_connection.rs @@ -124,12 +124,7 @@ pub async fn handle_create_connection( }; } let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name)?; - let with_properties = handler_args - .with_options - .inner() - .clone() - .into_iter() - .collect(); + let with_properties = handler_args.with_options.clone().into_connector_props(); let create_connection_payload = resolve_create_connection_payload(&with_properties)?; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index d3f7ea12434a..364c7aafed12 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1317,12 +1317,7 @@ pub async fn handle_create_source( let source_schema = stmt.source_schema.into_v2_with_warning(); - let mut with_properties = handler_args - .with_options - .clone() - .into_inner() - .into_iter() - .collect(); + let mut with_properties = handler_args.with_options.clone().into_connector_props(); validate_compatibility(&source_schema, &mut with_properties)?; ensure_table_constraints_supported(&stmt.constraints)?; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 9e93d843401c..9c1290b59ba5 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -44,8 +44,8 @@ use risingwave_pb::plan_common::{ use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_sqlparser::ast::{ - CdcTableInfo, ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Format, - ObjectName, OnConflict, SourceWatermark, TableConstraint, + CdcTableInfo, ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, + ExplainOptions, Format, ObjectName, OnConflict, SourceWatermark, TableConstraint, }; use risingwave_sqlparser::parser::IncludeOption; @@ -456,7 +456,8 @@ pub fn bind_pk_on_relation( /// stream source. #[allow(clippy::too_many_arguments)] pub(crate) async fn gen_create_table_plan_with_source( - context: OptimizerContext, + handler_args: HandlerArgs, + explain_options: ExplainOptions, table_name: ObjectName, column_defs: Vec, wildcard_idx: Option, @@ -480,8 +481,8 @@ pub(crate) async fn gen_create_table_plan_with_source( .into()); } - let session = context.session_ctx(); - let mut with_properties = context.with_options().inner().clone().into_iter().collect(); + let session = &handler_args.session; + let mut with_properties = handler_args.with_options.clone().into_connector_props(); validate_compatibility(&source_schema, &mut with_properties)?; ensure_table_constraints_supported(&constraints)?; @@ -489,7 +490,7 @@ pub(crate) async fn gen_create_table_plan_with_source( let sql_pk_names = bind_sql_pk_names(&column_defs, &constraints)?; let (columns_from_resolve_source, source_info) = - bind_columns_from_source(context.session_ctx(), &source_schema, &with_properties).await?; + bind_columns_from_source(session, &source_schema, &with_properties).await?; let columns_from_sql = bind_sql_columns(&column_defs)?; let mut columns = bind_all_columns( @@ -531,7 +532,7 @@ pub(crate) async fn gen_create_table_plan_with_source( // TODO(yuhao): allow multiple watermark on source. assert!(watermark_descs.len() <= 1); - let definition = context.normalized_sql().to_owned(); + let definition = handler_args.normalized_sql.clone(); bind_sql_column_constraints( session, @@ -543,6 +544,8 @@ pub(crate) async fn gen_create_table_plan_with_source( check_source_schema(&with_properties, row_id_index, &columns).await?; + let context = OptimizerContext::new(handler_args, explain_options); + gen_table_plan_inner( context.into(), table_name, @@ -939,7 +942,8 @@ fn derive_connect_properties( #[allow(clippy::too_many_arguments)] pub(super) async fn handle_create_table_plan( - context: OptimizerContext, + handler_args: HandlerArgs, + explain_options: ExplainOptions, col_id_gen: ColumnIdGenerator, source_schema: Option, cdc_table_info: Option, @@ -954,7 +958,7 @@ pub(super) async fn handle_create_table_plan( include_column_options: IncludeOption, ) -> Result<(PlanRef, Option, PbTable, TableJobType)> { let source_schema = check_create_table_with_source( - context.with_options(), + &handler_args.with_options, source_schema, &include_column_options, )?; @@ -963,7 +967,8 @@ pub(super) async fn handle_create_table_plan( match (source_schema, cdc_table_info.as_ref()) { (Some(source_schema), None) => ( gen_create_table_plan_with_source( - context, + handler_args, + explain_options, table_name.clone(), column_defs, wildcard_idx, @@ -979,22 +984,26 @@ pub(super) async fn handle_create_table_plan( .await?, TableJobType::General, ), - (None, None) => ( - gen_create_table_plan( - context, - table_name.clone(), - column_defs, - constraints, - col_id_gen, - source_watermarks, - append_only, - on_conflict, - with_version_column, - )?, - TableJobType::General, - ), + (None, None) => { + let context = OptimizerContext::new(handler_args, explain_options); + ( + gen_create_table_plan( + context, + table_name.clone(), + column_defs, + constraints, + col_id_gen, + source_watermarks, + append_only, + on_conflict, + with_version_column, + )?, + TableJobType::General, + ) + } (None, Some(cdc_table)) => { + let context = OptimizerContext::new(handler_args, explain_options); let (plan, table) = gen_create_table_plan_for_cdc_source( context.into(), cdc_table.source_name.clone(), @@ -1050,10 +1059,10 @@ pub async fn handle_create_table( } let (graph, source, table, job_type) = { - let context = OptimizerContext::from_handler_args(handler_args); let col_id_gen = ColumnIdGenerator::new_initial(); let (plan, source, table, job_type) = handle_create_table_plan( - context, + handler_args, + ExplainOptions::default(), col_id_gen, source_schema, cdc_table_info, @@ -1132,11 +1141,11 @@ pub async fn generate_stream_graph_for_table( ) -> Result<(StreamFragmentGraph, Table, Option)> { use risingwave_pb::catalog::table::OptionalAssociatedSourceId; - let context = OptimizerContext::from_handler_args(handler_args); let (plan, source, table) = match source_schema { Some(source_schema) => { gen_create_table_plan_with_source( - context, + handler_args, + ExplainOptions::default(), table_name, columns, wildcard_idx, @@ -1151,17 +1160,20 @@ pub async fn generate_stream_graph_for_table( ) .await? } - None => gen_create_table_plan( - context, - table_name, - columns, - constraints, - col_id_gen, - source_watermarks, - append_only, - on_conflict, - with_version_column, - )?, + None => { + let context = OptimizerContext::from_handler_args(handler_args); + gen_create_table_plan( + context, + table_name, + columns, + constraints, + col_id_gen, + source_watermarks, + append_only, + on_conflict, + with_version_column, + )? + } }; // TODO: avoid this backward conversion. diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 5235088b0d62..cfc416f0d8b0 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -39,16 +39,16 @@ use crate::utils::explain_stream_graph; use crate::OptimizerContextRef; async fn do_handle_explain( - context: OptimizerContext, + handler_args: HandlerArgs, + explain_options: ExplainOptions, stmt: Statement, blocks: &mut Vec, ) -> Result<()> { // Workaround to avoid `Rc` across `await` point. let mut batch_plan_fragmenter = None; + let session = handler_args.session.clone(); { - let session = context.session_ctx().clone(); - let (plan, context) = match stmt { // `CREATE TABLE` takes the ownership of the `OptimizerContext` to avoid `Rc` across // `await` point. We can only take the reference back from the `PlanRef` if it's @@ -72,7 +72,8 @@ async fn do_handle_explain( let source_schema = source_schema.map(|s| s.into_v2_with_warning()); let (plan, _source, _table, _job_type) = handle_create_table_plan( - context, + handler_args, + explain_options, col_id_gen, source_schema, cdc_table_info, @@ -91,7 +92,8 @@ async fn do_handle_explain( (Ok(plan), context) } Statement::CreateSink { stmt } => { - let partition_info = get_partition_compute_info(context.with_options()).await?; + let partition_info = get_partition_compute_info(&handler_args.with_options).await?; + let context = OptimizerContext::new(handler_args, explain_options); let plan = gen_sink_plan(&session, context.into(), stmt, partition_info) .map(|plan| plan.sink_plan)?; let context = plan.ctx(); @@ -102,7 +104,8 @@ async fn do_handle_explain( // `OptimizerContext` even if the planning fails. This enables us to log the partial // traces for better debugging experience. _ => { - let context: OptimizerContextRef = context.into(); + let context: OptimizerContextRef = + OptimizerContext::new(handler_args, explain_options).into(); let plan = match stmt { // -- Streaming DDLs -- Statement::CreateView { @@ -164,13 +167,16 @@ async fn do_handle_explain( | Statement::Delete { .. } | Statement::Update { .. } | Statement::Query { .. } => { - gen_batch_plan_by_statement(&session, context.clone(), stmt).map(|x| x.plan) + gen_batch_plan_by_statement(&session, context, stmt).map(|x| x.plan) } _ => bail_not_implemented!("unsupported statement {:?}", stmt), }; - (plan, context) + let plan = plan?; + let context = plan.ctx().clone(); + + (Ok(plan) as Result<_>, context) } }; @@ -247,10 +253,8 @@ pub async fn handle_explain( bail_not_implemented!(issue = 4856, "explain analyze"); } - let context = OptimizerContext::new(handler_args.clone(), options.clone()); - let mut blocks = Vec::new(); - let result = do_handle_explain(context, stmt, &mut blocks).await; + let result = do_handle_explain(handler_args, options.clone(), stmt, &mut blocks).await; if let Err(e) = result { if options.trace { diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index 215b0aaf8bcb..eb6e41037874 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -15,6 +15,7 @@ use core::convert::Into; use core::fmt::Formatter; use std::cell::{RefCell, RefMut}; +use std::marker::PhantomData; use std::rc::Rc; use std::sync::Arc; @@ -28,6 +29,8 @@ use crate::utils::{OverwriteOptions, WithOptions}; const RESERVED_ID_NUM: u16 = 10000; +type PhantomUnsend = PhantomData>; + pub struct OptimizerContext { session_ctx: Arc, /// Store plan node id @@ -55,6 +58,8 @@ pub struct OptimizerContext { /// Store the configs can be overwritten in with clause /// if not specified, use the value from session variable. overwrite_options: OverwriteOptions, + + _phantom: PhantomUnsend, } pub type OptimizerContextRef = Rc; @@ -86,6 +91,7 @@ impl OptimizerContext { next_expr_display_id: RefCell::new(RESERVED_ID_NUM.into()), total_rule_applied: RefCell::new(0), overwrite_options, + _phantom: Default::default(), } } @@ -107,6 +113,7 @@ impl OptimizerContext { next_expr_display_id: RefCell::new(0), total_rule_applied: RefCell::new(0), overwrite_options: OverwriteOptions::default(), + _phantom: Default::default(), } .into() } diff --git a/src/frontend/src/utils/overwrite_options.rs b/src/frontend/src/utils/overwrite_options.rs index a88a4c12d7fb..e1719c348107 100644 --- a/src/frontend/src/utils/overwrite_options.rs +++ b/src/frontend/src/utils/overwrite_options.rs @@ -20,7 +20,7 @@ pub struct OverwriteOptions { } impl OverwriteOptions { - const STREAMING_RATE_LIMIT_KEY: &'static str = "streaming_rate_limit"; + pub(crate) const STREAMING_RATE_LIMIT_KEY: &'static str = "streaming_rate_limit"; pub fn new(args: &mut HandlerArgs) -> Self { let streaming_rate_limit = { diff --git a/src/frontend/src/utils/with_options.rs b/src/frontend/src/utils/with_options.rs index 43091335be97..dc3e66bdfbac 100644 --- a/src/frontend/src/utils/with_options.rs +++ b/src/frontend/src/utils/with_options.rs @@ -25,6 +25,7 @@ use risingwave_sqlparser::ast::{ CreateSubscriptionStatement, SqlOption, Statement, Value, }; +use super::OverwriteOptions; use crate::catalog::connection_catalog::resolve_private_link_connection; use crate::catalog::ConnectionId; use crate::error::{ErrorCode, Result as RwResult, RwError}; @@ -81,6 +82,14 @@ impl WithOptions { self.inner } + /// Convert to connector props, remove the key-value pairs used in the top-level. + pub fn into_connector_props(self) -> HashMap { + self.inner + .into_iter() + .filter(|(key, _)| key != OverwriteOptions::STREAMING_RATE_LIMIT_KEY) + .collect() + } + /// Parse the retention seconds from the options. pub fn retention_seconds(&self) -> Option { self.inner From e15e74dc984ad8740f9be68c2254f79607f0e695 Mon Sep 17 00:00:00 2001 From: xfz <73645462+xuefengze@users.noreply.github.com> Date: Thu, 18 Apr 2024 16:44:05 +0800 Subject: [PATCH 56/64] test(source): add kafka-cdc compatibility test (#16360) --- ci/scripts/gen-integration-test-yaml.py | 1 + ci/scripts/notify.py | 1 + .../compatible_data/compatible_data.go | 199 ++++++++++++++++++ integration_tests/datagen/gen/generator.go | 3 + integration_tests/datagen/load_gen.go | 10 + integration_tests/datagen/main.go | 9 +- integration_tests/kafka-cdc/create_source.sql | 39 ++++ integration_tests/kafka-cdc/data_check | 1 + .../kafka-cdc/docker-compose.yml | 51 +++++ 9 files changed, 313 insertions(+), 1 deletion(-) create mode 100644 integration_tests/datagen/compatible_data/compatible_data.go create mode 100644 integration_tests/kafka-cdc/create_source.sql create mode 100644 integration_tests/kafka-cdc/data_check create mode 100644 integration_tests/kafka-cdc/docker-compose.yml diff --git a/ci/scripts/gen-integration-test-yaml.py b/ci/scripts/gen-integration-test-yaml.py index 6332b98ecca9..61fd7721acc3 100644 --- a/ci/scripts/gen-integration-test-yaml.py +++ b/ci/scripts/gen-integration-test-yaml.py @@ -43,6 +43,7 @@ 'pinot-sink': ['json'], 'presto-trino': ['json'], 'client-library': ['none'], + 'kafka-cdc': ['json'], } def gen_pipeline_steps(): diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py index 0ebd01f8621e..8823c2f0ee92 100755 --- a/ci/scripts/notify.py +++ b/ci/scripts/notify.py @@ -103,6 +103,7 @@ "pinot-sink-json": ["yiming"], "presto-trino-json": ["wutao"], "client-library-none": ["wutao"], + "kafka-cdc-json": ["bohan"], } def get_failed_tests(get_test_status, test_map): diff --git a/integration_tests/datagen/compatible_data/compatible_data.go b/integration_tests/datagen/compatible_data/compatible_data.go new file mode 100644 index 000000000000..a04441cb4366 --- /dev/null +++ b/integration_tests/datagen/compatible_data/compatible_data.go @@ -0,0 +1,199 @@ +package compatible_data + +import ( + "context" + "datagen/gen" + "datagen/sink" + "encoding/json" + "fmt" + "math" + "strings" +) + +type Struct struct { + S_int32 int32 `json:"s_int32"` + S_bool bool `json:"s_bool"` +} + +type compatibleData struct { + sink.BaseSinkRecord + + Id int32 `json:"id"` + C_boolean bool `json:"c_boolean"` + C_smallint int16 `json:"c_smallint"` + C_integer int32 `json:"c_integer"` + C_bigint int64 `json:"c_bigint"` + C_decimal string `json:"c_decimal"` + C_real float32 `json:"c_real"` + C_double_precision float64 `json:"c_double_precision"` + C_varchar string `json:"c_varchar"` + C_bytea string `json:"c_bytea"` + C_date string `json:"c_date"` + C_time string `json:"c_time"` + C_timestamp string `json:"c_timestamp"` + C_timestamptz string `json:"c_timestamptz"` + C_interval string `json:"c_interval"` + C_jsonb string `json:"c_jsonb"` + C_boolean_array []bool `json:"c_boolean_array"` + C_smallint_array []int16 `json:"c_smallint_array"` + C_integer_array []int32 `json:"c_integer_array"` + C_bigint_array []int64 `json:"c_bigint_array"` + C_decimal_array []string `json:"c_decimal_array"` + C_real_array []float32 `json:"c_real_array"` + C_double_precision_array []float64 `json:"c_double_precision_array"` + C_varchar_array []string `json:"c_varchar_array"` + C_bytea_array []string `json:"c_bytea_array"` + C_date_array []string `json:"c_date_array"` + C_time_array []string `json:"c_time_array"` + C_timestamp_array []string `json:"c_timestamp_array"` + C_timestamptz_array []string `json:"c_timestamptz_array"` + C_interval_array []string `json:"c_interval_array"` + C_jsonb_array []string `json:"c_jsonb_array"` + C_struct Struct `json:"c_struct"` +} + +func (c *compatibleData) Topic() string { + return "compatible_data" +} + +func (c *compatibleData) Key() string { + return fmt.Sprintf("%d", c.Id) +} + +func (c *compatibleData) ToPostgresSql() string { + panic("unimplemented") +} + +func (c *compatibleData) ToJson() []byte { + data, err := json.Marshal(c) + if err != nil { + panic("failed to marshal compatible data to JSON") + } + return data +} + +func (c *compatibleData) ToProtobuf() []byte { + panic("unimplemented") +} + +func (c *compatibleData) ToAvro() []byte { + panic("unimplemented") +} + +type compatibleDataGen struct { + recordSum int32 +} + +func NewCompatibleDataGen() gen.LoadGenerator { + return &compatibleDataGen{} +} + +func (g *compatibleDataGen) GenData() compatibleData { + g.recordSum++ + recordType := g.recordSum % 3 + if recordType == 0 { + return compatibleData{ + Id: g.recordSum, + C_boolean: true, + C_smallint: 0, + C_integer: 0, + C_bigint: 0, + C_decimal: "nan", + C_real: 0, + C_double_precision: 0, + C_varchar: "", + C_bytea: "", + C_date: "0001-01-01", + C_time: "00:00:00", + C_timestamp: "0001-01-01 00:00:00", + C_timestamptz: "0001-01-01 00:00:00Z", + C_interval: "P0Y0M0DT0H0M0S", + C_jsonb: "{}", + } + } else if recordType == 1 { + return compatibleData{ + Id: g.recordSum, + C_boolean: false, + C_smallint: math.MinInt16, + C_integer: math.MinInt32, + C_bigint: math.MinInt64, + C_decimal: "-123456789.123456789", + C_real: -9999.999999, + C_double_precision: -10000.0, + C_varchar: "a", + C_bytea: "a", + C_date: "1970-01-01", + C_time: "00:00:00.123456", + C_timestamp: "1970-01-01 00:00:00.123456", + C_timestamptz: "1970-01-01 00:00:00.123456Z", + C_interval: "P1Y2M3DT4H5M6S", + C_jsonb: "{}", + C_boolean_array: []bool{true, false}, + C_smallint_array: []int16{1}, + C_integer_array: []int32{1}, + C_bigint_array: []int64{1}, + C_decimal_array: []string{"1.0"}, + C_real_array: []float32{1.0}, + C_double_precision_array: []float64{1.0}, + C_varchar_array: []string{"aa"}, + C_bytea_array: []string{"aa"}, + C_date_array: []string{"1970-01-01"}, + C_time_array: []string{"00:00:00.123456"}, + C_timestamp_array: []string{"1970-01-01 00:00:00.123456"}, + C_timestamptz_array: []string{"1970-01-01 00:00:00.123456Z"}, + C_interval_array: []string{"P0Y0M0DT0H0M2S"}, + C_jsonb_array: []string{"{}"}, + C_struct: Struct{1, true}, + } + } else { + return compatibleData{ + Id: g.recordSum, + C_boolean: true, + C_smallint: math.MaxInt16, + C_integer: math.MaxInt32, + C_bigint: math.MaxInt64, + C_decimal: "123456789.123456789", + C_real: 9999.999999, + C_double_precision: 10000.0, + C_varchar: strings.Repeat("a", 100), + C_bytea: strings.Repeat("b", 100), + C_date: "9999-12-31", + C_time: "23:59:59.999999", + C_timestamp: "9999-12-31 23:59:59.999999", + C_timestamptz: "9999-12-31 23:59:59.999999Z", + C_interval: "P1Y2M3DT4H5M6S", + C_jsonb: "{\"mean\":1}", + C_boolean_array: []bool{true, false}, + C_smallint_array: []int16{1}, + C_integer_array: []int32{1}, + C_bigint_array: []int64{1}, + C_decimal_array: []string{"1.0"}, + C_real_array: []float32{1.0}, + C_double_precision_array: []float64{1.0}, + C_varchar_array: []string{"aa"}, + C_bytea_array: []string{"aa"}, + C_date_array: []string{"1970-01-01"}, + C_time_array: []string{"00:00:00.123456"}, + C_timestamp_array: []string{"1970-01-01 00:00:00.123456"}, + C_timestamptz_array: []string{"1970-01-01 00:00:00.123456Z"}, + C_interval_array: []string{"P1Y2M3DT4H5M6S"}, + C_jsonb_array: []string{"{}"}, + C_struct: Struct{-1, false}, + } + } +} + +func (g *compatibleDataGen) KafkaTopics() []string { + return []string{"compatible_data"} +} + +func (g *compatibleDataGen) Load(ctx context.Context, outCh chan<- sink.SinkRecord) { + for { + record := g.GenData() + select { + case <-ctx.Done(): + return + case outCh <- &record: + } + } +} diff --git a/integration_tests/datagen/gen/generator.go b/integration_tests/datagen/gen/generator.go index 5cc7dfd2acef..28b5245cdf9a 100644 --- a/integration_tests/datagen/gen/generator.go +++ b/integration_tests/datagen/gen/generator.go @@ -42,6 +42,9 @@ type GeneratorConfig struct { // The topic to filter. If not specified, all topics will be used. Topic string + + // The total number of events to generate. + TotalEvents int64 } type LoadGenerator interface { diff --git a/integration_tests/datagen/load_gen.go b/integration_tests/datagen/load_gen.go index 77735983fb6d..a0df429482b0 100644 --- a/integration_tests/datagen/load_gen.go +++ b/integration_tests/datagen/load_gen.go @@ -6,6 +6,7 @@ import ( "datagen/ad_ctr" "datagen/cdn_metrics" "datagen/clickstream" + "datagen/compatible_data" "datagen/delivery" "datagen/ecommerce" "datagen/gen" @@ -67,6 +68,8 @@ func newGen(cfg gen.GeneratorConfig) (gen.LoadGenerator, error) { return livestream.NewLiveStreamMetricsGen(cfg), nil } else if cfg.Mode == "nexmark" { return nexmark.NewNexmarkGen(cfg), nil + } else if cfg.Mode == "compatible-data" { + return compatible_data.NewCompatibleDataGen(), nil } else { return nil, fmt.Errorf("invalid mode: %s", cfg.Mode) } @@ -148,6 +151,13 @@ func generateLoad(ctx context.Context, cfg gen.GeneratorConfig) error { return err } } + if cfg.TotalEvents > 0 && count >= cfg.TotalEvents { + if err := sinkImpl.Flush(ctx); err != nil { + return err + } + log.Printf("Sent %d records in total (Elapsed: %s)", count, time.Since(initTime).String()) + return nil + } } } } diff --git a/integration_tests/datagen/main.go b/integration_tests/datagen/main.go index 135e470ea204..cd3335925a04 100644 --- a/integration_tests/datagen/main.go +++ b/integration_tests/datagen/main.go @@ -244,7 +244,7 @@ func main() { }, cli.StringFlag{ Name: "mode", - Usage: "ad-click | ad-ctr | twitter | cdn-metrics | clickstream | ecommerce | delivery | livestream", + Usage: "ad-click | ad-ctr | twitter | cdn-metrics | clickstream | ecommerce | delivery | livestream | compatible-data", Required: true, Destination: &cfg.Mode, }, @@ -267,6 +267,13 @@ func main() { Required: false, Destination: &cfg.Topic, }, + cli.Int64Flag{ + Name: "total_event", + Usage: "The total number of events to generate. If not specified, the generator will run indefinitely.", + Value: 0, + Required: false, + Destination: &cfg.TotalEvents, + }, }, } err := app.Run(os.Args) diff --git a/integration_tests/kafka-cdc/create_source.sql b/integration_tests/kafka-cdc/create_source.sql new file mode 100644 index 000000000000..cbc64ad7362d --- /dev/null +++ b/integration_tests/kafka-cdc/create_source.sql @@ -0,0 +1,39 @@ +CREATE TABLE compatible_data ( + id integer PRIMARY KEY, + c_boolean boolean, + c_smallint smallint, + c_integer integer, + c_bigint bigint, + c_decimal decimal, + c_real real, + c_double_precision double precision, + c_varchar varchar, + c_bytea bytea, + c_date date, + c_time time, + c_timestamp timestamp, + c_timestamptz timestamptz, + c_interval interval, + c_jsonb jsonb, + c_boolean_array boolean[], + c_smallint_array smallint[], + c_integer_array integer[], + c_bigint_array bigint[], + c_decimal_array decimal[], + c_real_array real[], + c_double_precision_array double precision[], + c_varchar_array varchar[], + c_bytea_array bytea[], + c_date_array date[], + c_time_array time[], + c_timestamp_array timestamp[], + c_timestamptz_array timestamptz[], + c_interval_array interval[], + c_jsonb_array jsonb[], + c_struct STRUCT, +) WITH ( + connector = 'kafka', + topic = 'compatible_data', + properties.bootstrap.server = 'message_queue:29092', + scan.startup.mode = 'earliest' +) FORMAT PLAIN ENCODE JSON; diff --git a/integration_tests/kafka-cdc/data_check b/integration_tests/kafka-cdc/data_check new file mode 100644 index 000000000000..3c3ac8c12f43 --- /dev/null +++ b/integration_tests/kafka-cdc/data_check @@ -0,0 +1 @@ +compatible_data diff --git a/integration_tests/kafka-cdc/docker-compose.yml b/integration_tests/kafka-cdc/docker-compose.yml new file mode 100644 index 000000000000..d62fa2acd9df --- /dev/null +++ b/integration_tests/kafka-cdc/docker-compose.yml @@ -0,0 +1,51 @@ +--- +version: "3" +services: + risingwave-standalone: + extends: + file: ../../docker/docker-compose.yml + service: risingwave-standalone + etcd-0: + extends: + file: ../../docker/docker-compose.yml + service: etcd-0 + grafana-0: + extends: + file: ../../docker/docker-compose.yml + service: grafana-0 + minio-0: + extends: + file: ../../docker/docker-compose.yml + service: minio-0 + prometheus-0: + extends: + file: ../../docker/docker-compose.yml + service: prometheus-0 + message_queue: + extends: + file: ../../docker/docker-compose.yml + service: message_queue + datagen: + build: ../datagen + depends_on: [message_queue] + command: + - /bin/sh + - -c + - /datagen --mode compatible-data --qps 2 --total_event 3 kafka --brokers message_queue:29092 + restart: always + container_name: datagen + +volumes: + risingwave-standalone: + external: false + etcd-0: + external: false + grafana-0: + external: false + minio-0: + external: false + prometheus-0: + external: false + message_queue: + external: false +name: risingwave-compose From b37228e3a11bb2d176920c40ee636ce3703beb2a Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Thu, 18 Apr 2024 17:27:46 +0800 Subject: [PATCH 57/64] fix(frontend): split chunks in system catalog (#16375) Signed-off-by: Runji Wang --- src/batch/src/executor/sys_row_seq_scan.rs | 13 ++++------ src/common/src/catalog/mod.rs | 6 ++--- src/frontend/macro/src/lib.rs | 24 ++++++++++++------- .../src/catalog/system_catalog/mod.rs | 14 ++++------- 4 files changed, 29 insertions(+), 28 deletions(-) diff --git a/src/batch/src/executor/sys_row_seq_scan.rs b/src/batch/src/executor/sys_row_seq_scan.rs index fd427b132909..07521f73d74f 100644 --- a/src/batch/src/executor/sys_row_seq_scan.rs +++ b/src/batch/src/executor/sys_row_seq_scan.rs @@ -101,19 +101,16 @@ impl Executor for SysRowSeqScanExecutor { } fn execute(self: Box) -> BoxedDataChunkStream { - self.do_executor() + self.do_execute() } } impl SysRowSeqScanExecutor { #[try_stream(boxed, ok = DataChunk, error = BatchError)] - async fn do_executor(self: Box) { - let chunk = self - .sys_catalog_reader - .read_table(&self.table_id) - .await - .map_err(BatchError::SystemTable)?; - if chunk.cardinality() != 0 { + async fn do_execute(self: Box) { + #[for_await] + for chunk in self.sys_catalog_reader.read_table(self.table_id) { + let chunk = chunk.map_err(BatchError::SystemTable)?; yield chunk.project(&self.column_indices); } } diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index a7bf6f980f36..0fc23af8c105 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -21,9 +21,9 @@ pub mod test_utils; use std::sync::Arc; -use async_trait::async_trait; pub use column::*; pub use external_table::*; +use futures::stream::BoxStream; pub use internal_table::*; use parse_display::Display; pub use physical_table::*; @@ -148,9 +148,9 @@ pub fn cdc_table_name_column_desc() -> ColumnDesc { } /// The local system catalog reader in the frontend node. -#[async_trait] pub trait SysCatalogReader: Sync + Send + 'static { - async fn read_table(&self, table_id: &TableId) -> Result; + /// Reads the data of the system catalog table. + fn read_table(&self, table_id: TableId) -> BoxStream<'_, Result>; } pub type SysCatalogReaderRef = Arc; diff --git a/src/frontend/macro/src/lib.rs b/src/frontend/macro/src/lib.rs index 36b7f33eb99c..f6ede6400d89 100644 --- a/src/frontend/macro/src/lib.rs +++ b/src/frontend/macro/src/lib.rs @@ -112,6 +112,7 @@ fn gen_sys_table(attr: Attr, item_fn: ItemFn) -> Result { let struct_type = strip_outer_type(ty, "Vec").ok_or_else(return_type_error)?; let _await = item_fn.sig.asyncness.map(|_| quote!(.await)); let handle_error = return_result.then(|| quote!(?)); + let chunk_size = 1024usize; Ok(quote! { #[linkme::distributed_slice(crate::catalog::system_catalog::SYS_CATALOGS_SLICE)] @@ -121,19 +122,26 @@ fn gen_sys_table(attr: Attr, item_fn: ItemFn) -> Result { assert!(#struct_type::PRIMARY_KEY.is_some(), "primary key is required for system table"); }; + #[futures_async_stream::try_stream(boxed, ok = risingwave_common::array::DataChunk, error = risingwave_common::error::BoxedError)] + async fn function(reader: &crate::catalog::system_catalog::SysCatalogReaderImpl) { + let rows = #user_fn_name(reader) #_await #handle_error; + let mut builder = #struct_type::data_chunk_builder(#chunk_size); + for row in rows { + if let Some(chunk) = builder.append_one_row(row.into_owned_row()) { + yield chunk; + } + } + if let Some(chunk) = builder.consume_all() { + yield chunk; + } + } + crate::catalog::system_catalog::BuiltinCatalog::Table(crate::catalog::system_catalog::BuiltinTable { name: #table_name, schema: #schema_name, columns: #struct_type::fields(), pk: #struct_type::PRIMARY_KEY.unwrap(), - function: |reader| std::boxed::Box::pin(async { - let rows = #user_fn_name(reader) #_await #handle_error; - let mut builder = #struct_type::data_chunk_builder(rows.len() + 1); - for row in rows { - _ = builder.append_one_row(row.into_owned_row()); - } - Ok(builder.finish()) - }), + function, }) } }) diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 81137f402ad0..a3b16216d818 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -19,8 +19,7 @@ pub mod rw_catalog; use std::collections::HashMap; use std::sync::{Arc, LazyLock}; -use async_trait::async_trait; -use futures::future::BoxFuture; +use futures::stream::BoxStream; use itertools::Itertools; use parking_lot::RwLock; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeManagerRef; @@ -142,7 +141,7 @@ pub struct BuiltinTable { schema: &'static str, columns: Vec>, pk: &'static [usize], - function: for<'a> fn(&'a SysCatalogReaderImpl) -> BoxFuture<'a, Result>, + function: for<'a> fn(&'a SysCatalogReaderImpl) -> BoxStream<'a, Result>, } pub struct BuiltinView { @@ -340,18 +339,15 @@ pub static SYS_CATALOGS: LazyLock = LazyLock::new(|| { #[linkme::distributed_slice] pub static SYS_CATALOGS_SLICE: [fn() -> BuiltinCatalog]; -#[async_trait] impl SysCatalogReader for SysCatalogReaderImpl { - async fn read_table(&self, table_id: &TableId) -> Result { + fn read_table(&self, table_id: TableId) -> BoxStream<'_, Result> { let table_name = SYS_CATALOGS .catalogs .get((table_id.table_id - SYS_CATALOG_START_ID as u32) as usize) .unwrap(); match table_name { - BuiltinCatalog::Table(t) => (t.function)(self).await, - BuiltinCatalog::View(_) => { - panic!("read_table should not be called on a view") - } + BuiltinCatalog::Table(t) => (t.function)(self), + BuiltinCatalog::View(_) => panic!("read_table should not be called on a view"), } } } From d3fc9b8169b82441a90cd41c08e7acec93bb5bd9 Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Thu, 18 Apr 2024 17:36:26 +0800 Subject: [PATCH 58/64] feat: alter parallelism use deferred mode during recovery state (#16361) Co-authored-by: Noel Kwan --- src/meta/src/rpc/ddl_controller.rs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 78fb65089eb4..1c551f46b6fc 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -356,8 +356,15 @@ impl DdlController { &self, table_id: u32, parallelism: PbTableParallelism, - deferred: bool, + mut deferred: bool, ) -> MetaResult<()> { + if self.barrier_manager.check_status_running().is_err() { + tracing::info!( + "alter parallelism is set to deferred mode because the system is in recovery state" + ); + deferred = true; + } + if !deferred && !self .metadata_manager From aa9d4baa7986fb2eb0ffde550a8fbf25c9335e7f Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 18 Apr 2024 17:48:35 +0800 Subject: [PATCH 59/64] feat(stream): support non-append-only process time temporal join (#16286) --- .../{ => append_only}/issue_15257.slt | 0 .../{ => append_only}/temporal_join.slt | 0 .../temporal_join_multiple_rows.slt | 0 .../temporal_join_non_loopup_cond.slt | 0 .../temporal_join_watermark.slt | 0 .../temporal_join_with_index.slt | 0 .../non_append_only/temporal_join.slt | 47 +++++ .../temporal_join_non_lookup_cond.slt | 100 +++++++++ .../temporal_join_with_index.slt | 44 ++++ .../temporal_join_with_index2.slt | 44 ++++ .../temporal_join_with_index3.slt | 44 ++++ proto/stream_plan.proto | 2 + src/common/src/util/stream_graph_visitor.rs | 3 + .../tests/testdata/input/temporal_join.yaml | 26 +-- .../tests/testdata/output/nexmark.yaml | 6 +- .../testdata/output/temporal_filter.yaml | 2 +- .../tests/testdata/output/temporal_join.yaml | 71 ++++--- .../src/optimizer/plan_node/logical_join.rs | 7 - .../plan_node/stream_temporal_join.rs | 71 ++++++- .../plan_visitor/temporal_join_validator.rs | 19 +- src/stream/src/executor/temporal_join.rs | 198 +++++++++++++++--- src/stream/src/from_proto/temporal_join.rs | 46 +++- 22 files changed, 637 insertions(+), 93 deletions(-) rename e2e_test/streaming/temporal_join/{ => append_only}/issue_15257.slt (100%) rename e2e_test/streaming/temporal_join/{ => append_only}/temporal_join.slt (100%) rename e2e_test/streaming/temporal_join/{ => append_only}/temporal_join_multiple_rows.slt (100%) rename e2e_test/streaming/temporal_join/{ => append_only}/temporal_join_non_loopup_cond.slt (100%) rename e2e_test/streaming/temporal_join/{ => append_only}/temporal_join_watermark.slt (100%) rename e2e_test/streaming/temporal_join/{ => append_only}/temporal_join_with_index.slt (100%) create mode 100644 e2e_test/streaming/temporal_join/non_append_only/temporal_join.slt create mode 100644 e2e_test/streaming/temporal_join/non_append_only/temporal_join_non_lookup_cond.slt create mode 100644 e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index.slt create mode 100644 e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index2.slt create mode 100644 e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index3.slt diff --git a/e2e_test/streaming/temporal_join/issue_15257.slt b/e2e_test/streaming/temporal_join/append_only/issue_15257.slt similarity index 100% rename from e2e_test/streaming/temporal_join/issue_15257.slt rename to e2e_test/streaming/temporal_join/append_only/issue_15257.slt diff --git a/e2e_test/streaming/temporal_join/temporal_join.slt b/e2e_test/streaming/temporal_join/append_only/temporal_join.slt similarity index 100% rename from e2e_test/streaming/temporal_join/temporal_join.slt rename to e2e_test/streaming/temporal_join/append_only/temporal_join.slt diff --git a/e2e_test/streaming/temporal_join/temporal_join_multiple_rows.slt b/e2e_test/streaming/temporal_join/append_only/temporal_join_multiple_rows.slt similarity index 100% rename from e2e_test/streaming/temporal_join/temporal_join_multiple_rows.slt rename to e2e_test/streaming/temporal_join/append_only/temporal_join_multiple_rows.slt diff --git a/e2e_test/streaming/temporal_join/temporal_join_non_loopup_cond.slt b/e2e_test/streaming/temporal_join/append_only/temporal_join_non_loopup_cond.slt similarity index 100% rename from e2e_test/streaming/temporal_join/temporal_join_non_loopup_cond.slt rename to e2e_test/streaming/temporal_join/append_only/temporal_join_non_loopup_cond.slt diff --git a/e2e_test/streaming/temporal_join/temporal_join_watermark.slt b/e2e_test/streaming/temporal_join/append_only/temporal_join_watermark.slt similarity index 100% rename from e2e_test/streaming/temporal_join/temporal_join_watermark.slt rename to e2e_test/streaming/temporal_join/append_only/temporal_join_watermark.slt diff --git a/e2e_test/streaming/temporal_join/temporal_join_with_index.slt b/e2e_test/streaming/temporal_join/append_only/temporal_join_with_index.slt similarity index 100% rename from e2e_test/streaming/temporal_join/temporal_join_with_index.slt rename to e2e_test/streaming/temporal_join/append_only/temporal_join_with_index.slt diff --git a/e2e_test/streaming/temporal_join/non_append_only/temporal_join.slt b/e2e_test/streaming/temporal_join/non_append_only/temporal_join.slt new file mode 100644 index 000000000000..b98658b6e8d2 --- /dev/null +++ b/e2e_test/streaming/temporal_join/non_append_only/temporal_join.slt @@ -0,0 +1,47 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table stream(id1 int, a1 int, b1 int); + +statement ok +create table version(id2 int, a2 int, b2 int, primary key (id2)); + +statement ok +create materialized view v as select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 + +statement ok +insert into stream values(1, 11, 111); + +statement ok +insert into version values(1, 11, 111); + +statement ok +insert into stream values(1, 11, 111); + +statement ok +delete from version; + +query IIII rowsort +select * from v; +---- +1 11 1 11 +1 11 NULL NULL + +statement ok +update stream set a1 = 22, b1 = 222 + +query IIII rowsort +select * from v; +---- +1 22 NULL NULL +1 22 NULL NULL + +statement ok +drop materialized view v; + +statement ok +drop table stream; + +statement ok +drop table version; diff --git a/e2e_test/streaming/temporal_join/non_append_only/temporal_join_non_lookup_cond.slt b/e2e_test/streaming/temporal_join/non_append_only/temporal_join_non_lookup_cond.slt new file mode 100644 index 000000000000..2c0cf094db99 --- /dev/null +++ b/e2e_test/streaming/temporal_join/non_append_only/temporal_join_non_lookup_cond.slt @@ -0,0 +1,100 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table stream(id1 int, a1 int, b1 int); + +statement ok +create table version(id2 int, a2 int, b2 int, primary key (id2)); + +statement ok +create materialized view v as select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 and a1 > a2; + +statement ok +insert into stream values(1, 11, 111); + +statement ok +insert into version values(1, 12, 111); + +statement ok +insert into stream values(1, 13, 111); + +statement ok +delete from version; + +query IIII rowsort +select * from v; +---- +1 11 NULL NULL +1 13 1 12 + +statement ok +delete from stream; + +statement ok +insert into version values(2, 22, 222); + +statement ok +insert into stream values(2, 23, 222); + +query IIII rowsort +select * from v; +---- +2 23 2 22 + +statement ok +delete from stream; + +query IIII rowsort +select * from v; +---- + + +statement ok +drop materialized view v; + +statement ok +drop table stream; + +statement ok +drop table version; + +statement ok +create table stream(id1 int, a1 int, b1 int); + +statement ok +create table version(id2 int, a2 int, b2 int, primary key (id2)); + +statement ok +create materialized view v as select id1, a1, id2, a2 from stream join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 and a1 > a2; + +statement ok +insert into version values (1, 12, 111), (2, 12, 111); + +statement ok +insert into stream values (1, 11, 111), (2, 13, 111); + +query IIII rowsort +select * from v; +---- +2 13 2 12 + +statement ok +update stream set a1 = 222, b1 = 333 where id1 = 1; + +statement ok +update stream set a1 = 2, b1 = 3 where id1 = 2; + +query IIII rowsort +select * from v; +---- +1 222 1 12 + +statement ok +drop materialized view v; + +statement ok +drop table stream; + +statement ok +drop table version; diff --git a/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index.slt b/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index.slt new file mode 100644 index 000000000000..e9e7e59a483d --- /dev/null +++ b/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index.slt @@ -0,0 +1,44 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table stream(id1 int, a1 int, b1 int); + +statement ok +create table version(id2 int, a2 int, b2 int, primary key (id2)); + +statement ok +create index idx on version (a2); + +statement ok +create materialized view v as select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on b1 = b2 and a1 = a2; + +statement ok +insert into version values(1, 11, 111); + +statement ok +insert into stream values(1, 11, 111); + +query IIII rowsort +select * from v; +---- +1 11 1 11 + +statement ok +update stream set a1 = 22 where id1 = 1; + +query IIII rowsort +select * from v; +---- +1 22 NULL NULL + +statement ok +drop materialized view v; + +statement ok +drop table stream; + +statement ok +drop table version; + + diff --git a/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index2.slt b/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index2.slt new file mode 100644 index 000000000000..c560c135f07c --- /dev/null +++ b/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index2.slt @@ -0,0 +1,44 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table stream(id1 int, a1 int, b1 int); + +statement ok +create table version(id2 int, a2 int, b2 int, primary key (id2)); + +statement ok +create index idx on version(a2, b2) distributed by (a2); + +statement ok +create materialized view v as select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on b1 = b2 and a1 = a2; + +statement ok +insert into version values(1, 11, 111); + +statement ok +insert into stream values(1, 11, 111); + +query IIII rowsort +select * from v; +---- +1 11 1 11 + +statement ok +update stream set a1 = 22 where id1 = 1; + +query IIII rowsort +select * from v; +---- +1 22 NULL NULL + +statement ok +drop materialized view v; + +statement ok +drop table stream; + +statement ok +drop table version; + + diff --git a/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index3.slt b/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index3.slt new file mode 100644 index 000000000000..d9c52e2db8a3 --- /dev/null +++ b/e2e_test/streaming/temporal_join/non_append_only/temporal_join_with_index3.slt @@ -0,0 +1,44 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table stream(id1 int, a1 int, b1 int); + +statement ok +create table version(id2 int, a2 int, b2 int, primary key (id2)); + +statement ok +create index idx on version(a2, b2) distributed by (a2, b2); + +statement ok +create materialized view v as select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on b1 = b2 and a1 = a2; + +statement ok +insert into version values(1, 11, 111); + +statement ok +insert into stream values(1, 11, 111); + +query IIII rowsort +select * from v; +---- +1 11 1 11 + +statement ok +update stream set a1 = 22 where id1 = 1; + +query IIII rowsort +select * from v; +---- +1 22 NULL NULL + +statement ok +drop materialized view v; + +statement ok +drop table stream; + +statement ok +drop table version; + + diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index b4393153b57a..89c052137897 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -435,6 +435,8 @@ message TemporalJoinNode { plan_common.StorageTableDesc table_desc = 7; // The output indices of the lookup side table repeated uint32 table_output_indices = 8; + // The state table used for non-append-only temporal join. + optional catalog.Table memo_table = 9; } message DynamicFilterNode { diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index a3bb869b5527..81f1189693c0 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -109,6 +109,9 @@ pub fn visit_stream_node_tables_inner( always!(node.right_table, "HashJoinRight"); always!(node.right_degree_table, "HashJoinDegreeRight"); } + NodeBody::TemporalJoin(node) => { + optional!(node.memo_table, "TemporalJoinMemo"); + } NodeBody::DynamicFilter(node) => { if node.condition_always_relax { always!(node.left_table, "DynamicFilterLeftNotSatisfy"); diff --git a/src/frontend/planner_test/tests/testdata/input/temporal_join.yaml b/src/frontend/planner_test/tests/testdata/input/temporal_join.yaml index 500e4ae2c298..7bf9769b4c43 100644 --- a/src/frontend/planner_test/tests/testdata/input/temporal_join.yaml +++ b/src/frontend/planner_test/tests/testdata/input/temporal_join.yaml @@ -43,11 +43,11 @@ - stream_error - name: Temporal join append only test sql: | - create table stream(id1 int, a1 int, b1 int); + create table stream(id1 int, a1 int, b1 int) append only; create table version(id2 int, a2 int, b2 int, primary key (id2)); select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where a2 < 10; expected_outputs: - - stream_error + - stream_plan - name: Temporal join type test sql: | create table stream(id1 int, a1 int, b1 int); @@ -57,7 +57,7 @@ - stream_error - name: multi-way temporal join with the same key sql: | - create table stream(k int, a1 int, b1 int) APPEND ONLY; + create table stream(k int, a1 int, b1 int); create table version1(k int, x1 int, y2 int, primary key (k)); create table version2(k int, x2 int, y2 int, primary key (k)); select stream.k, x1, x2, a1, b1 @@ -68,7 +68,7 @@ - stream_plan - name: multi-way temporal join with different keys sql: | - create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, id2 int, a1 int, b1 int); create table version1(id1 int, x1 int, y2 int, primary key (id1)); create table version2(id2 int, x2 int, y2 int, primary key (id2)); select stream.id1, x1, stream.id2, x2, a1, b1 @@ -79,7 +79,7 @@ - stream_plan - name: multi-way temporal join with different keys sql: | - create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, id2 int, a1 int, b1 int); create table version1(id1 int, x1 int, y2 int, primary key (id1)); create table version2(id2 int, x2 int, y2 int, primary key (id2)); select stream.id1, x1, stream.id2, x2, a1, b1 @@ -90,7 +90,7 @@ - stream_plan - name: temporal join with an index (distribution key size = 1) sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx2 on version (a2, b2) distributed by (a2); select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; @@ -98,7 +98,7 @@ - stream_plan - name: temporal join with an index (distribution key size = 2) sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx2 on version (a2, b2); select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; @@ -106,7 +106,7 @@ - stream_plan - name: temporal join with an index (index column size = 1) sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx2 on version (b2); select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; @@ -114,14 +114,14 @@ - stream_plan - name: temporal join with singleton table sql: | - create table t (a int) append only; + create table t (a int); create materialized view v as select count(*) from t; select * from t left join v FOR SYSTEM_TIME AS OF PROCTIME() on a = count; expected_outputs: - stream_plan - name: index selection for temporal join (with one index). sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx on version (a2, b2); select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; @@ -129,7 +129,7 @@ - stream_plan - name: index selection for temporal join (with two indexes) and should choose the index with a longer prefix.. sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx1 on version (a2); create index idx2 on version (a2, b2); @@ -138,7 +138,7 @@ - stream_plan - name: index selection for temporal join (with three indexes) and should choose primary table. sql: | - create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int, c1 int); create table version(id2 int, a2 int, b2 int, c2 int, primary key (id2)); create index idx1 on version (a2); create index idx2 on version (b2); @@ -148,7 +148,7 @@ - stream_plan - name: index selection for temporal join (two index) and no one matches. sql: | - create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int, c1 int); create table version(id2 int, a2 int, b2 int, c2 int, primary key (id2)); create index idx1 on version (a2); create index idx2 on version (a2, b2); diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 08916d1539c8..dcdd34ed2c15 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -1002,7 +1002,7 @@ ON mod(B.auction, 10000) = S.key sink_plan: |- StreamSink { type: append-only, columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr10018(hidden), side_input.key(hidden)] } - └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } ├─StreamExchange { dist: HashShard($expr1) } │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } @@ -1011,7 +1011,7 @@ stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(bid._row_id, $expr1) } - └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } ├─StreamExchange { dist: HashShard($expr1) } │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } @@ -1024,7 +1024,7 @@ └── StreamExchange Hash([5, 6]) from 1 Fragment 1 - StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } + StreamTemporalJoin { type: Inner, append_only: true, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } ├── StreamExchange Hash([4]) from 2 └── StreamExchange NoShuffle from 3 diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml index fd5f6aec627f..7bbd43ce3c35 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml @@ -333,7 +333,7 @@ StreamMaterialize { columns: [id1, a1, id2, v1, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamProject { exprs: [stream.id1, stream.a1, version.id2, stream.v1, stream._row_id], output_watermarks: [stream.v1] } └─StreamDynamicFilter { predicate: (stream.v1 > now), output_watermarks: [stream.v1], output: [stream.id1, stream.a1, stream.v1, version.id2, stream._row_id], cleaned_by_watermark: true } - ├─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, stream.v1, version.id2, stream._row_id] } + ├─StreamTemporalJoin { type: LeftOuter, append_only: true, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, stream.v1, version.id2, stream._row_id] } │ ├─StreamExchange { dist: HashShard(stream.id1) } │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.v1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml index ebf7af980d23..aa8887e98bef 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml @@ -7,7 +7,7 @@ stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + └─StreamTemporalJoin { type: LeftOuter, append_only: true, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } @@ -23,7 +23,7 @@ stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } @@ -36,7 +36,7 @@ stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } @@ -49,7 +49,7 @@ stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1, a1], pk_columns: [stream._row_id, id1, a1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.id1, stream.a1, stream._row_id) } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2, version.a2) } @@ -65,7 +65,7 @@ └─StreamSimpleAgg [append_only] { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream._row_id, stream.id1, version.id2] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream._row_id, stream.id1, version.id2] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } @@ -80,12 +80,17 @@ HINT: Please add the primary key of the lookup table to the join condition and remove any other conditions - name: Temporal join append only test sql: | - create table stream(id1 int, a1 int, b1 int); + create table stream(id1 int, a1 int, b1 int) append only; create table version(id2 int, a2 int, b2 int, primary key (id2)); select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where a2 < 10; - stream_error: |- - Not supported: Temporal join requires an append-only left input - HINT: Please ensure your left input is append-only + stream_plan: |- + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2], stream_scan_type: UpstreamOnly, stream_key: [version.id2], pk: [id2], dist: UpstreamHashShard(version.id2) } - name: Temporal join type test sql: | create table stream(id1 int, a1 int, b1 int); @@ -96,7 +101,7 @@ HINT: please check your temporal join syntax e.g. consider removing the right outer join if it is being used. - name: multi-way temporal join with the same key sql: | - create table stream(k int, a1 int, b1 int) APPEND ONLY; + create table stream(k int, a1 int, b1 int); create table version1(k int, x1 int, y2 int, primary key (k)); create table version2(k int, x2 int, y2 int, primary key (k)); select stream.k, x1, x2, a1, b1 @@ -106,9 +111,9 @@ stream_plan: |- StreamMaterialize { columns: [k, x1, x2, a1, b1, stream._row_id(hidden), version2.k(hidden)], stream_key: [stream._row_id, k], pk_columns: [stream._row_id, k], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.k, stream._row_id) } - └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] } ├─StreamExchange { dist: HashShard(stream.k) } - │ └─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } + │ └─StreamTemporalJoin { type: Inner, append_only: false, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } │ ├─StreamExchange { dist: HashShard(stream.k) } │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } @@ -118,7 +123,7 @@ └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], stream_scan_type: UpstreamOnly, stream_key: [version2.k], pk: [k], dist: UpstreamHashShard(version2.k) } - name: multi-way temporal join with different keys sql: | - create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, id2 int, a1 int, b1 int); create table version1(id1 int, x1 int, y2 int, primary key (id1)); create table version2(id2 int, x2 int, y2 int, primary key (id2)); select stream.id1, x1, stream.id2, x2, a1, b1 @@ -128,9 +133,9 @@ stream_plan: |- StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.id1, stream.id2, stream._row_id) } - └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } ├─StreamExchange { dist: HashShard(stream.id2) } - │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } + │ └─StreamTemporalJoin { type: Inner, append_only: false, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } │ ├─StreamExchange { dist: HashShard(stream.id1) } │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } @@ -140,7 +145,7 @@ └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], stream_scan_type: UpstreamOnly, stream_key: [version2.id2], pk: [id2], dist: UpstreamHashShard(version2.id2) } - name: multi-way temporal join with different keys sql: | - create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, id2 int, a1 int, b1 int); create table version1(id1 int, x1 int, y2 int, primary key (id1)); create table version2(id2 int, x2 int, y2 int, primary key (id2)); select stream.id1, x1, stream.id2, x2, a1, b1 @@ -150,9 +155,9 @@ stream_plan: |- StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.id1, stream.id2, stream._row_id) } - └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } + └─StreamTemporalJoin { type: Inner, append_only: true, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } ├─StreamExchange { dist: HashShard(stream.id2) } - │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } + │ └─StreamTemporalJoin { type: Inner, append_only: false, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } │ ├─StreamExchange { dist: HashShard(stream.id1) } │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } @@ -162,54 +167,54 @@ └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], stream_scan_type: UpstreamOnly, stream_key: [version2.id2], pk: [id2], dist: UpstreamHashShard(version2.id2) } - name: temporal join with an index (distribution key size = 1) sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx2 on version (a2, b2) distributed by (a2); select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + └─StreamTemporalJoin { type: LeftOuter, append_only: false, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } ├─StreamExchange { dist: HashShard(stream.a1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], stream_scan_type: UpstreamOnly, stream_key: [idx2.id2], pk: [a2, b2, id2], dist: UpstreamHashShard(idx2.a2) } - name: temporal join with an index (distribution key size = 2) sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx2 on version (a2, b2); select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + └─StreamTemporalJoin { type: LeftOuter, append_only: false, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } ├─StreamExchange { dist: HashShard(stream.a1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], stream_scan_type: UpstreamOnly, stream_key: [idx2.id2], pk: [a2, b2, id2], dist: UpstreamHashShard(idx2.a2) } - name: temporal join with an index (index column size = 1) sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx2 on version (b2); select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, stream.b1, a1], pk_columns: [stream._row_id, id2, stream.b1, a1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.b1 = idx2.b2 AND (stream.a1 = idx2.a2), output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + └─StreamTemporalJoin { type: LeftOuter, append_only: false, predicate: stream.b1 = idx2.b2 AND (stream.a1 = idx2.a2), output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } ├─StreamExchange { dist: HashShard(stream.b1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.b2) } └─StreamTableScan { table: idx2, columns: [idx2.b2, idx2.id2, idx2.a2], stream_scan_type: UpstreamOnly, stream_key: [idx2.id2], pk: [b2, id2], dist: UpstreamHashShard(idx2.b2) } - name: temporal join with singleton table sql: | - create table t (a int) append only; + create table t (a int); create materialized view v as select count(*) from t; select * from t left join v FOR SYSTEM_TIME AS OF PROCTIME() on a = count; stream_plan: |- StreamMaterialize { columns: [a, count, t._row_id(hidden), $expr1(hidden)], stream_key: [t._row_id, $expr1], pk_columns: [t._row_id, $expr1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: AND ($expr1 = v.count), output: [t.a, v.count, t._row_id, $expr1] } + └─StreamTemporalJoin { type: LeftOuter, append_only: false, predicate: AND ($expr1 = v.count), output: [t.a, v.count, t._row_id, $expr1] } ├─StreamExchange { dist: Single } │ └─StreamProject { exprs: [t.a, t.a::Int64 as $expr1, t._row_id] } │ └─StreamTableScan { table: t, columns: [t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -217,21 +222,21 @@ └─StreamTableScan { table: v, columns: [v.count], stream_scan_type: UpstreamOnly, stream_key: [], pk: [], dist: Single } - name: index selection for temporal join (with one index). sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx on version (a2, b2); select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.a1, idx.id2, stream._row_id, stream.b1) } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx.a2 AND stream.b1 = idx.b2, output: [stream.id1, stream.a1, idx.id2, idx.a2, stream._row_id, stream.b1] } + └─StreamTemporalJoin { type: LeftOuter, append_only: false, predicate: stream.a1 = idx.a2 AND stream.b1 = idx.b2, output: [stream.id1, stream.a1, idx.id2, idx.a2, stream._row_id, stream.b1] } ├─StreamExchange { dist: HashShard(stream.a1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx.a2) } └─StreamTableScan { table: idx, columns: [idx.id2, idx.a2, idx.b2], stream_scan_type: UpstreamOnly, stream_key: [idx.id2], pk: [a2, b2, id2], dist: UpstreamHashShard(idx.a2) } - name: index selection for temporal join (with two indexes) and should choose the index with a longer prefix.. sql: | - create table stream(id1 int, a1 int, b1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int); create table version(id2 int, a2 int, b2 int, primary key (id2)); create index idx1 on version (a2); create index idx2 on version (a2, b2); @@ -239,14 +244,14 @@ stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + └─StreamTemporalJoin { type: LeftOuter, append_only: false, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } ├─StreamExchange { dist: HashShard(stream.a1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } └─StreamTableScan { table: idx2, columns: [idx2.id2, idx2.a2, idx2.b2], stream_scan_type: UpstreamOnly, stream_key: [idx2.id2], pk: [a2, b2, id2], dist: UpstreamHashShard(idx2.a2) } - name: index selection for temporal join (with three indexes) and should choose primary table. sql: | - create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int, c1 int); create table version(id2 int, a2 int, b2 int, c2 int, primary key (id2)); create index idx1 on version (a2); create index idx2 on version (b2); @@ -255,14 +260,14 @@ stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden), stream.c1(hidden)], stream_key: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_columns: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.id1, stream.a1, stream._row_id, stream.b1, stream.c1) } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] } + └─StreamTemporalJoin { type: LeftOuter, append_only: false, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream.c1, stream._row_id], stream_scan_type: ArrangementBackfill, stream_key: [stream._row_id], pk: [_row_id], dist: UpstreamHashShard(stream._row_id) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2, version.c2], stream_scan_type: UpstreamOnly, stream_key: [version.id2], pk: [id2], dist: UpstreamHashShard(version.id2) } - name: index selection for temporal join (two index) and no one matches. sql: | - create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY; + create table stream(id1 int, a1 int, b1 int, c1 int); create table version(id2 int, a2 int, b2 int, c2 int, primary key (id2)); create index idx1 on version (a2); create index idx2 on version (a2, b2); diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 47c5238bde2b..571efee542c2 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -1071,13 +1071,6 @@ impl LogicalJoin { // Enforce a shuffle for the temporal join LHS to let the scheduler be able to schedule the join fragment together with the RHS with a `no_shuffle` exchange. let left = required_dist.enforce(left, &Order::any()); - if !left.append_only() { - return Err(RwError::from(ErrorCode::NotSupported( - "Temporal join requires an append-only left input".into(), - "Please ensure your left input is append-only".into(), - ))); - } - // Extract the predicate from logical scan. Only pure scan is supported. let (new_scan, scan_predicate, project_expr) = logical_scan.predicate_pull_up(); // Construct output column to require column mapping diff --git a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs index ecbdba1b3226..ce8753b9ddbc 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -14,6 +14,7 @@ use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::util::sort_util::OrderType; use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::TemporalJoinNode; @@ -26,26 +27,29 @@ use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary}; use crate::expr::{Expr, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::generic::GenericPlanNode; use crate::optimizer::plan_node::plan_tree_node::PlanTreeNodeUnary; -use crate::optimizer::plan_node::utils::IndicesDisplay; +use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder}; use crate::optimizer::plan_node::{ EqJoinPredicate, EqJoinPredicateDisplay, StreamExchange, StreamTableScan, TryToStreamPb, }; use crate::scheduler::SchedulerResult; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::ColIndexMappingRewriteExt; +use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTemporalJoin { pub base: PlanBase, core: generic::Join, eq_join_predicate: EqJoinPredicate, + append_only: bool, } impl StreamTemporalJoin { pub fn new(core: generic::Join, eq_join_predicate: EqJoinPredicate) -> Self { assert!(core.join_type == JoinType::Inner || core.join_type == JoinType::LeftOuter); - assert!(core.left.append_only()); + let append_only = core.left.append_only(); let right = core.right.clone(); let exchange: &StreamExchange = right .as_stream_exchange() @@ -79,6 +83,7 @@ impl StreamTemporalJoin { base, core, eq_join_predicate, + append_only, } } @@ -90,6 +95,58 @@ impl StreamTemporalJoin { pub fn eq_join_predicate(&self) -> &EqJoinPredicate { &self.eq_join_predicate } + + pub fn append_only(&self) -> bool { + self.append_only + } + + /// Return memo-table catalog and its `pk_indices`. + /// (`join_key` + `left_pk` + `right_pk`) -> (`right_scan_schema` + `join_key` + `left_pk`) + /// + /// Write pattern: + /// for each left input row (with insert op), construct the memo table pk and insert the row into the memo table. + /// + /// Read pattern: + /// for each left input row (with delete op), construct pk prefix (`join_key` + `left_pk`) to fetch rows and delete them from the memo table. + pub fn infer_memo_table_catalog(&self, right_scan: &StreamTableScan) -> TableCatalog { + let left_eq_indexes = self.eq_join_predicate.left_eq_indexes(); + let read_prefix_len_hint = left_eq_indexes.len() + self.left().stream_key().unwrap().len(); + + // Build internal table + let mut internal_table_catalog_builder = TableCatalogBuilder::default(); + // Add right table fields + let right_scan_schema = right_scan.core().schema(); + for field in right_scan_schema.fields() { + internal_table_catalog_builder.add_column(field); + } + // Add join_key + left_pk + for field in left_eq_indexes + .iter() + .chain(self.core.left.stream_key().unwrap()) + .map(|idx| &self.core.left.schema().fields()[*idx]) + { + internal_table_catalog_builder.add_column(field); + } + + let mut pk_indices = vec![]; + pk_indices + .extend(right_scan_schema.len()..(right_scan_schema.len() + read_prefix_len_hint)); + pk_indices.extend(right_scan.stream_key().unwrap()); + + pk_indices.iter().for_each(|idx| { + internal_table_catalog_builder.add_order_column(*idx, OrderType::ascending()) + }); + + let dist_key_len = right_scan + .core() + .distribution_key() + .map(|keys| keys.len()) + .unwrap_or(0); + + let internal_table_dist_keys = + (right_scan_schema.len()..(right_scan_schema.len() + dist_key_len)).collect(); + internal_table_catalog_builder.build(internal_table_dist_keys, read_prefix_len_hint) + } } impl Distill for StreamTemporalJoin { @@ -97,6 +154,7 @@ impl Distill for StreamTemporalJoin { let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("type", Pretty::debug(&self.core.join_type))); + vec.push(("append_only", Pretty::debug(&self.append_only))); let concat_schema = self.core.concat_schema(); vec.push(( @@ -142,7 +200,7 @@ impl_plan_tree_node_for_binary! { StreamTemporalJoin } impl TryToStreamPb for StreamTemporalJoin { fn try_to_stream_prost_body( &self, - _state: &mut BuildFragmentGraphState, + state: &mut BuildFragmentGraphState, ) -> SchedulerResult { let left_jk_indices = self.eq_join_predicate.left_eq_indexes(); let right_jk_indices = self.eq_join_predicate.right_eq_indexes(); @@ -174,6 +232,13 @@ impl TryToStreamPb for StreamTemporalJoin { output_indices: self.core.output_indices.iter().map(|&x| x as u32).collect(), table_desc: Some(scan.core().table_desc.try_to_protobuf()?), table_output_indices: scan.core().output_col_idx.iter().map(|&i| i as _).collect(), + memo_table: if self.append_only { + None + } else { + let mut memo_table = self.infer_memo_table_catalog(scan); + memo_table = memo_table.with_id(state.gen_table_id_wrapped()); + Some(memo_table.to_internal_table_prost()) + }, })) } } diff --git a/src/frontend/src/optimizer/plan_visitor/temporal_join_validator.rs b/src/frontend/src/optimizer/plan_visitor/temporal_join_validator.rs index 2f8d6b3fc89b..cbdd2c695ad8 100644 --- a/src/frontend/src/optimizer/plan_visitor/temporal_join_validator.rs +++ b/src/frontend/src/optimizer/plan_visitor/temporal_join_validator.rs @@ -15,6 +15,7 @@ use risingwave_sqlparser::ast::AsOf; use super::{DefaultBehavior, Merge}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ BatchSeqScan, LogicalScan, PlanTreeNodeBinary, StreamTableScan, StreamTemporalJoin, }; @@ -22,12 +23,21 @@ use crate::optimizer::plan_visitor::PlanVisitor; use crate::PlanRef; #[derive(Debug, Clone, Default)] -pub struct TemporalJoinValidator {} +pub struct TemporalJoinValidator { + found_non_append_only_temporal_join: bool, +} impl TemporalJoinValidator { pub fn exist_dangling_temporal_scan(plan: PlanRef) -> bool { - let mut decider = TemporalJoinValidator {}; - decider.visit(plan) + let mut decider = TemporalJoinValidator { + found_non_append_only_temporal_join: false, + }; + let ctx = plan.ctx(); + let has_dangling_temporal_scan = decider.visit(plan); + if decider.found_non_append_only_temporal_join { + ctx.session_ctx().notice_to_user("A non-append-only temporal join is used in the query. It would introduce a additional memo-table comparing to append-only temporal join."); + } + has_dangling_temporal_scan } } @@ -53,6 +63,9 @@ impl PlanVisitor for TemporalJoinValidator { } fn visit_stream_temporal_join(&mut self, stream_temporal_join: &StreamTemporalJoin) -> bool { + if !stream_temporal_join.append_only() { + self.found_non_append_only_temporal_join = true; + } self.visit(stream_temporal_join.left()) } } diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index 1d374189cb5e..f365706317a9 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -22,6 +22,7 @@ use either::Either; use futures::stream::{self, PollNext}; use futures::{pin_mut, StreamExt, TryStreamExt}; use futures_async_stream::{for_await, try_stream}; +use itertools::Itertools; use local_stats_alloc::{SharedStatsAlloc, StatsAlloc}; use lru::DefaultHasher; use risingwave_common::array::{Op, StreamChunk}; @@ -45,12 +46,18 @@ use super::{ }; use crate::cache::{cache_may_stale, new_with_hasher_in, ManagedLruCache}; use crate::common::metrics::MetricsInfo; +use crate::common::table::state_table::StateTable; use crate::executor::join::builder::JoinStreamChunkBuilder; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ActorContextRef, Executor, Watermark}; use crate::task::AtomicU64Ref; -pub struct TemporalJoinExecutor { +pub struct TemporalJoinExecutor< + K: HashKey, + S: StateStore, + const T: JoinTypePrimitive, + const APPEND_ONLY: bool, +> { ctx: ActorContextRef, #[allow(dead_code)] info: ExecutorInfo, @@ -63,9 +70,7 @@ pub struct TemporalJoinExecutor, output_indices: Vec, chunk_size: usize, - // TODO: update metrics - #[allow(dead_code)] - metrics: Arc, + memo_table: Option>, } #[derive(Default)] @@ -310,17 +315,21 @@ async fn align_input(left: Executor, right: Executor) { } mod phase1 { + use std::ops::Bound; + + use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::stream_chunk_builder::StreamChunkBuilder; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::hash::{HashKey, NullBitmap}; - use risingwave_common::row::{self, Row, RowExt}; + use risingwave_common::row::{self, OwnedRow, Row, RowExt}; use risingwave_common::types::{DataType, DatumRef}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_hummock_sdk::HummockEpoch; use risingwave_storage::StateStore; use super::{StreamExecutorError, TemporalSide}; + use crate::common::table::state_table::StateTable; pub(super) trait Phase1Evaluation { /// Called when a matched row is found. @@ -425,13 +434,21 @@ mod phase1 { #[try_stream(ok = StreamChunk, error = StreamExecutorError)] #[allow(clippy::too_many_arguments)] - pub(super) async fn handle_chunk<'a, K: HashKey, S: StateStore, E: Phase1Evaluation>( + pub(super) async fn handle_chunk< + 'a, + K: HashKey, + S: StateStore, + E: Phase1Evaluation, + const APPEND_ONLY: bool, + >( chunk_size: usize, right_size: usize, full_schema: Vec, epoch: HummockEpoch, left_join_keys: &'a [usize], right_table: &'a mut TemporalSide, + memo_table_lookup_prefix: &'a [usize], + memo_table: &'a mut Option>, null_matched: &'a K::Bitmap, chunk: StreamChunk, ) { @@ -441,39 +458,142 @@ mod phase1 { .visibility() .iter() .zip_eq_debug(keys.iter()) - .filter_map(|(vis, key)| if vis { Some(key) } else { None }); + .zip_eq_debug(chunk.ops()) + .filter_map(|((vis, key), op)| { + if vis { + if APPEND_ONLY { + assert_eq!(*op, Op::Insert); + Some(key) + } else { + match op { + Op::Insert | Op::UpdateInsert => Some(key), + Op::Delete | Op::UpdateDelete => None, + } + } + } else { + None + } + }); right_table .fetch_or_promote_keys(to_fetch_keys, epoch) .await?; + for (r, key) in chunk.rows_with_holes().zip_eq_debug(keys.into_iter()) { let Some((op, left_row)) = r else { continue; }; + let mut matched = false; - if key.null_bitmap().is_subset(null_matched) - && let join_entry = right_table.force_peek(&key) - && !join_entry.is_empty() - { - matched = true; - for right_row in join_entry.cached.values() { - if let Some(chunk) = - E::append_matched_row(op, &mut builder, left_row, right_row) - { - yield chunk; + + if APPEND_ONLY { + // Append-only temporal join + if key.null_bitmap().is_subset(null_matched) + && let join_entry = right_table.force_peek(&key) + && !join_entry.is_empty() + { + matched = true; + for right_row in join_entry.cached.values() { + if let Some(chunk) = + E::append_matched_row(op, &mut builder, left_row, right_row) + { + yield chunk; + } + } + } + } else { + // Non-append-only temporal join + // The memo-table pk and columns: + // (`join_key` + `left_pk` + `right_pk`) -> (`right_scan_schema` + `join_key` + `left_pk`) + // + // Write pattern: + // for each left input row (with insert op), construct the memo table pk and insert the row into the memo table. + // Read pattern: + // for each left input row (with delete op), construct pk prefix (`join_key` + `left_pk`) to fetch rows and delete them from the memo table. + // + // Temporal join supports inner join and left outer join, additionally, it could contain other conditions. + // Surprisingly, we could handle them in a unified way with memo table. + // The memo table would persist rows fetched from the right table and appending the `join_key` and `left_pk` from the left row. + // The null rows generated by outer join and the other condition somehow is a stateless operation which means we can handle them without the memo table. + let memo_table = memo_table.as_mut().unwrap(); + match op { + Op::Insert | Op::UpdateInsert => { + if key.null_bitmap().is_subset(null_matched) + && let join_entry = right_table.force_peek(&key) + && !join_entry.is_empty() + { + matched = true; + for right_row in join_entry.cached.values() { + let right_row: OwnedRow = right_row.clone(); + // Insert into memo table + memo_table.insert(right_row.clone().chain( + left_row.project(memo_table_lookup_prefix).into_owned_row(), + )); + if let Some(chunk) = E::append_matched_row( + Op::Insert, + &mut builder, + left_row, + right_row, + ) { + yield chunk; + } + } + } + } + Op::Delete | Op::UpdateDelete => { + let mut memo_rows_to_delete = vec![]; + if key.null_bitmap().is_subset(null_matched) { + let sub_range: &(Bound, Bound) = + &(Bound::Unbounded, Bound::Unbounded); + let prefix = left_row.project(memo_table_lookup_prefix); + let state_table_iter = memo_table + .iter_with_prefix(prefix, sub_range, Default::default()) + .await?; + pin_mut!(state_table_iter); + + while let Some(memo_row) = state_table_iter.next().await { + matched = true; + let memo_row = memo_row?.into_owned_row(); + memo_rows_to_delete.push(memo_row.clone()); + if let Some(chunk) = E::append_matched_row( + Op::Delete, + &mut builder, + left_row, + memo_row.slice(0..right_size), + ) { + yield chunk; + } + } + } + for memo_row in memo_rows_to_delete { + // Delete from memo table + memo_table.delete(memo_row); + } } } } - if let Some(chunk) = E::match_end(&mut builder, op, left_row, right_size, matched) { + if let Some(chunk) = E::match_end( + &mut builder, + match op { + Op::Insert | Op::UpdateInsert => Op::Insert, + Op::Delete | Op::UpdateDelete => Op::Delete, + }, + left_row, + right_size, + matched, + ) { yield chunk; } } + if let Some(chunk) = builder.take() { yield chunk; } } } -impl TemporalJoinExecutor { +impl + TemporalJoinExecutor +{ #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, @@ -492,6 +612,7 @@ impl TemporalJoinExecutor metrics: Arc, chunk_size: usize, join_key_data_types: Vec, + memo_table: Option>, ) -> Self { let alloc = StatsAlloc::new(Global).shared(); @@ -528,7 +649,7 @@ impl TemporalJoinExecutor condition, output_indices, chunk_size, - metrics, + memo_table, } } @@ -555,7 +676,14 @@ impl TemporalJoinExecutor let left_to_output: HashMap = HashMap::from_iter(left_map.iter().cloned()); + let left_stream_key_indices = self.left.pk_indices().to_vec(); let right_stream_key_indices = self.right.pk_indices().to_vec(); + let memo_table_lookup_prefix = self + .left_join_keys + .iter() + .cloned() + .chain(left_stream_key_indices) + .collect_vec(); let null_matched = K::Bitmap::from_bool_vec(self.null_safe); @@ -572,6 +700,8 @@ impl TemporalJoinExecutor .chain(self.right.schema().data_types().into_iter()) .collect(); + let mut wait_first_barrier = true; + #[for_await] for msg in align_input(self.left, self.right) { self.right_table.cache.evict(); @@ -591,13 +721,15 @@ impl TemporalJoinExecutor let full_schema = full_schema.clone(); if T == JoinType::Inner { - let st1 = phase1::handle_chunk::( + let st1 = phase1::handle_chunk::( self.chunk_size, right_size, full_schema, epoch, &self.left_join_keys, &mut self.right_table, + &memo_table_lookup_prefix, + &mut self.memo_table, &null_matched, chunk, ); @@ -621,13 +753,15 @@ impl TemporalJoinExecutor } } else if let Some(ref cond) = self.condition { // Joined result without evaluating non-lookup conditions. - let st1 = phase1::handle_chunk::( + let st1 = phase1::handle_chunk::( self.chunk_size, right_size, full_schema, epoch, &self.left_join_keys, &mut self.right_table, + &memo_table_lookup_prefix, + &mut self.memo_table, &null_matched, chunk, ); @@ -670,13 +804,15 @@ impl TemporalJoinExecutor // The last row should always be marker row, assert_eq!(matched_count, 0); } else { - let st1 = phase1::handle_chunk::( + let st1 = phase1::handle_chunk::( self.chunk_size, right_size, full_schema, epoch, &self.left_join_keys, &mut self.right_table, + &memo_table_lookup_prefix, + &mut self.memo_table, &null_matched, chunk, ); @@ -689,6 +825,18 @@ impl TemporalJoinExecutor } } InternalMessage::Barrier(updates, barrier) => { + if !A { + if wait_first_barrier { + wait_first_barrier = false; + self.memo_table.as_mut().unwrap().init_epoch(barrier.epoch); + } else { + self.memo_table + .as_mut() + .unwrap() + .commit(barrier.epoch) + .await?; + } + } if let Some(vnodes) = barrier.as_update_vnode_bitmap(self.ctx.id) { let prev_vnodes = self.right_table.source.update_vnode_bitmap(vnodes.clone()); @@ -710,8 +858,8 @@ impl TemporalJoinExecutor } } -impl Execute - for TemporalJoinExecutor +impl Execute + for TemporalJoinExecutor { fn execute(self: Box) -> super::BoxedMessageStream { self.into_stream().boxed() diff --git a/src/stream/src/from_proto/temporal_join.rs b/src/stream/src/from_proto/temporal_join.rs index 15badec97e5c..805049a29688 100644 --- a/src/stream/src/from_proto/temporal_join.rs +++ b/src/stream/src/from_proto/temporal_join.rs @@ -22,6 +22,7 @@ use risingwave_pb::plan_common::{JoinType as JoinTypeProto, StorageTableDesc}; use risingwave_storage::table::batch_table::storage_table::StorageTable; use super::*; +use crate::common::table::state_table::StateTable; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ActorContextRef, JoinType, TemporalJoinExecutor}; use crate::task::AtomicU64Ref; @@ -45,9 +46,9 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { .collect_vec(); StorageTable::new_partial( - store, + store.clone(), column_ids, - params.vnode_bitmap.map(Into::into), + params.vnode_bitmap.clone().map(Into::into), table_desc, ) }; @@ -99,6 +100,23 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { .map(|idx| source_l.schema().fields[*idx].data_type()) .collect_vec(); + let memo_table = node.get_memo_table(); + let memo_table = match memo_table { + Ok(memo_table) => { + let vnodes = Arc::new( + params + .vnode_bitmap + .expect("vnodes not set for temporal join"), + ); + Some( + StateTable::from_table_catalog(memo_table, store.clone(), Some(vnodes.clone())) + .await, + ) + } + Err(_) => None, + }; + let append_only = memo_table.is_none(); + let dispatcher_args = TemporalJoinExecutorDispatcherArgs { ctx: params.actor_context, info: params.info.clone(), @@ -117,6 +135,8 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { metrics: params.executor_stats, join_type_proto: node.get_join_type()?, join_key_data_types, + memo_table, + append_only, }; Ok((params.info, dispatcher_args.dispatch()?).into()) @@ -141,6 +161,8 @@ struct TemporalJoinExecutorDispatcherArgs { metrics: Arc, join_type_proto: JoinTypeProto, join_key_data_types: Vec, + memo_table: Option>, + append_only: bool, } impl HashKeyDispatcher for TemporalJoinExecutorDispatcherArgs { @@ -149,11 +171,12 @@ impl HashKeyDispatcher for TemporalJoinExecutorDispatcherArgs fn dispatch_impl(self) -> Self::Output { /// This macro helps to fill the const generic type parameter. macro_rules! build { - ($join_type:ident) => { + ($join_type:ident, $append_only:ident) => { Ok(Box::new(TemporalJoinExecutor::< K, S, { JoinType::$join_type }, + { $append_only }, >::new( self.ctx, self.info, @@ -171,12 +194,25 @@ impl HashKeyDispatcher for TemporalJoinExecutorDispatcherArgs self.metrics, self.chunk_size, self.join_key_data_types, + self.memo_table, ))) }; } match self.join_type_proto { - JoinTypeProto::Inner => build!(Inner), - JoinTypeProto::LeftOuter => build!(LeftOuter), + JoinTypeProto::Inner => { + if self.append_only { + build!(Inner, true) + } else { + build!(Inner, false) + } + } + JoinTypeProto::LeftOuter => { + if self.append_only { + build!(LeftOuter, true) + } else { + build!(LeftOuter, false) + } + } _ => unreachable!(), } } From 9d9d2052bdb3055deddc36d7d0fca37a5018e6ad Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 18 Apr 2024 18:01:48 +0800 Subject: [PATCH 60/64] feat(optimizer): add kafka scan operator (#16371) --- e2e_test/streaming/aggregate/count_star.slt | 6 +- .../tests/testdata/output/batch_source.yaml | 4 +- .../testdata/output/generated_columns.yaml | 2 +- .../tests/testdata/output/nexmark_source.yaml | 98 ++--- .../testdata/output/nexmark_watermark.yaml | 98 ++--- .../tests/testdata/output/share.yaml | 16 +- .../tests/testdata/output/shared_source.yml | 8 +- .../tests/testdata/output/subquery.yaml | 10 +- .../tests/testdata/output/tpch_variant.yaml | 276 +++++++------- .../tests/testdata/output/watermark.yaml | 2 +- .../src/optimizer/logical_optimization.rs | 9 + src/frontend/src/optimizer/mod.rs | 2 + .../optimizer/plan_node/batch_kafka_scan.rs | 140 +++++++ .../src/optimizer/plan_node/batch_source.rs | 5 - .../src/optimizer/plan_node/generic/source.rs | 27 +- .../optimizer/plan_node/logical_kafka_scan.rs | 346 ++++++++++++++++++ .../src/optimizer/plan_node/logical_source.rs | 207 +---------- src/frontend/src/optimizer/plan_node/mod.rs | 8 + src/frontend/src/optimizer/rule/mod.rs | 3 + .../rule/source_to_kafka_scan_rule.rs | 35 ++ .../src/scheduler/distributed/stage.rs | 2 +- src/frontend/src/scheduler/local.rs | 2 +- src/frontend/src/scheduler/plan_fragmenter.rs | 23 +- 23 files changed, 839 insertions(+), 490 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/batch_kafka_scan.rs create mode 100644 src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs create mode 100644 src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs diff --git a/e2e_test/streaming/aggregate/count_star.slt b/e2e_test/streaming/aggregate/count_star.slt index 134a850930ea..2bc6c88c96fd 100644 --- a/e2e_test/streaming/aggregate/count_star.slt +++ b/e2e_test/streaming/aggregate/count_star.slt @@ -1,3 +1,6 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + statement ok create table t (v int); @@ -7,9 +10,6 @@ insert into t values (114), (514); statement ok create materialized view mv as select * from t; -statement ok -flush; - query I select count(*) from t; ---- diff --git a/src/frontend/planner_test/tests/testdata/output/batch_source.yaml b/src/frontend/planner_test/tests/testdata/output/batch_source.yaml index 8613f2830566..63daa9c43b2d 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_source.yaml @@ -3,11 +3,11 @@ select * from s logical_plan: |- LogicalProject { exprs: [id, value] } - └─LogicalSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [id, value] } - └─BatchSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], filter: (None, None) } create_source: format: plain encode: protobuf diff --git a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml index e88df797de0c..3ed95c1ac146 100644 --- a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml @@ -17,7 +17,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [(v2 + 1:Int32) as $expr1] } - └─BatchSource { source: s1, columns: [v2, _row_id], filter: (None, None) } + └─BatchSource { source: s1, columns: [v2, _row_id] } - name: select proctime() sql: | select proctime(); diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 2e9484a4c1ed..823fa85459df 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -51,7 +51,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, price, date_time] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } @@ -87,7 +87,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time, _row_id] } @@ -118,7 +118,7 @@ BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) } └─BatchProject { exprs: [auction, price] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, price, _row_id] } @@ -154,11 +154,11 @@ ├─BatchExchange { order: [], dist: HashShard(seller) } │ └─BatchFilter { predicate: (category = 10:Int32) } │ └─BatchProject { exprs: [id, seller, category] } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } └─BatchProject { exprs: [id, name, city, state] } - └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, seller, _row_id) } @@ -244,9 +244,9 @@ └─BatchHashAgg { group_key: [id, category], aggs: [max(price)] } └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, category, price] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [category, avg], stream_key: [category], pk_columns: [category], pk_conflict: NoCheck } └─StreamProject { exprs: [category, (sum(max(price)) / count(max(price))::Decimal) as $expr1] } @@ -380,7 +380,7 @@ │ └─LogicalProject { exprs: [window_start, auction] } │ └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } │ └─LogicalFilter { predicate: IsNotNull(date_time) } - │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─LogicalProject { exprs: [max(count), window_start] } └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } └─LogicalProject { exprs: [window_start, count] } @@ -389,7 +389,7 @@ └─LogicalProject { exprs: [auction, window_start] } └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } └─LogicalFilter { predicate: IsNotNull(date_time) } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [auction, count] } @@ -399,7 +399,7 @@ │ └─BatchExchange { order: [], dist: HashShard(auction) } │ └─BatchProject { exprs: [auction, date_time] } │ └─BatchFilter { predicate: IsNotNull(date_time) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchHashAgg { group_key: [window_start], aggs: [max(count)] } └─BatchExchange { order: [], dist: HashShard(window_start) } └─BatchHashAgg { group_key: [auction, window_start], aggs: [count] } @@ -407,7 +407,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchProject { exprs: [auction, date_time] } └─BatchFilter { predicate: IsNotNull(date_time) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, count, window_start, window_start] } @@ -614,13 +614,13 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: price = max(price) AND (date_time >= $expr2) AND (date_time <= $expr1), output: [auction, price, bidder, date_time] } ├─BatchExchange { order: [], dist: HashShard(price) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(max(price)) } └─BatchProject { exprs: [max(price), $expr1, ($expr1 - '00:00:10':Interval) as $expr2] } └─BatchHashAgg { group_key: [$expr1], aggs: [max(price)] } └─BatchExchange { order: [], dist: HashShard($expr1) } └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden)], stream_key: [_row_id, $expr1, price], pk_columns: [_row_id, $expr1, price], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1] } @@ -731,12 +731,12 @@ │ └─BatchExchange { order: [], dist: HashShard(id, name, $expr1, $expr2) } │ └─BatchProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } │ └─BatchProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } - │ └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } └─BatchHashAgg { group_key: [seller, $expr3, $expr4], aggs: [] } └─BatchExchange { order: [], dist: HashShard(seller, $expr3, $expr4) } └─BatchProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } @@ -827,21 +827,21 @@ └─LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, auction, bidder, price, channel, url, date_time, extra, _row_id] } └─LogicalFilter { predicate: (id = auction) AND (date_time >= date_time) AND (date_time <= expires) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } └─LogicalJoin { type: Inner, on: (id = auction) AND (date_time >= date_time) AND (date_time <= expires), output: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } - ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck } └─StreamGroupTopN [append_only] { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } @@ -912,7 +912,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2, _row_id] } @@ -1012,7 +1012,7 @@ └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra] } └─BatchFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } └─BatchProject { exprs: [auction, bidder, price, date_time, extra] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra, _row_id] } @@ -1059,7 +1059,7 @@ └─BatchExchange { order: [], dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } └─BatchExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } └─BatchProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } @@ -1265,7 +1265,7 @@ └─BatchExchange { order: [], dist: HashShard(channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag) } └─BatchExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } └─BatchProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } └─StreamHashAgg [append_only] { group_key: [channel, $expr1], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } @@ -1391,7 +1391,7 @@ └─BatchHashAgg { group_key: [auction, $expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } └─BatchExchange { order: [], dist: HashShard(auction, $expr1) } └─BatchProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], stream_key: [auction, day], pk_columns: [auction, day], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price) / count(price)::Decimal) as $expr2, sum(price)] } @@ -1438,13 +1438,13 @@ └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [date_time DESC], limit: 1, offset: 0, group_key: [bidder, auction] } └─BatchExchange { order: [], dist: HashShard(bidder, auction) } └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction], pk_columns: [bidder, auction], pk_conflict: NoCheck } └─StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, group_key: [bidder, auction] } @@ -1494,13 +1494,13 @@ └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, rank] } └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bidder, auction] } └─BatchExchange { order: [], dist: HashShard(bidder, auction) } └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction, _row_id], pk_columns: [bidder, auction, _row_id], pk_conflict: NoCheck } └─StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bidder, auction] } @@ -1549,7 +1549,7 @@ └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1557,7 +1557,7 @@ └─BatchGroupTopN { order: [price DESC], limit: 10, offset: 0, group_key: [auction] } └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden), rank_number], stream_key: [auction, _row_id], pk_columns: [auction, _row_id], pk_conflict: NoCheck } └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1616,11 +1616,11 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } ├─BatchExchange { order: [], dist: HashShard(auction) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (category = 10:Int32) } └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(auction, _row_id, _row_id) } @@ -1693,7 +1693,7 @@ └─BatchProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1] } └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } └─BatchProject { exprs: [auction, bidder, price, channel, url] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1, _row_id] } @@ -1724,7 +1724,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3] } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3, _row_id] } @@ -1765,10 +1765,10 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price)] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchHashAgg { group_key: [auction], aggs: [max(price)] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id) } @@ -1853,16 +1853,16 @@ │ └─BatchHashAgg { group_key: [id, item_name], aggs: [count(auction)] } │ └─BatchHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction] } │ ├─BatchExchange { order: [], dist: HashShard(id) } - │ │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(auction) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } └─BatchSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum0(count), count(auction)] } └─BatchHashAgg { group_key: [auction], aggs: [count] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [auction_id, auction_item_name], pk_conflict: NoCheck } └─StreamDynamicFilter { predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } @@ -1982,12 +1982,12 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchProject { exprs: [auction] } └─BatchFilter { predicate: (count >= 20:Int32) } └─BatchHashAgg { group_key: [auction], aggs: [count] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id) } @@ -2071,12 +2071,12 @@ BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchProject { exprs: [auction] } └─BatchFilter { predicate: (count < 20:Int32) } └─BatchHashAgg { group_key: [auction], aggs: [count] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id) } @@ -2163,9 +2163,9 @@ └─BatchHashAgg { group_key: [id, item_name], aggs: [count(auction)] } └─BatchHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } └─StreamProject { exprs: [id, item_name, count(auction)] } @@ -2272,9 +2272,9 @@ └─BatchHashAgg { group_key: [id], aggs: [max(price)] } └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, price] } ├─BatchExchange { order: [], dist: HashShard(id) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(auction) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [min(min(max(price)))] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index 12593221419d..6dd731cffffb 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -11,7 +11,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } @@ -35,7 +35,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, (0.908:Decimal * Field(bid, 2:Int32)::Decimal) as $expr4, $expr1] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, (0.908:Decimal * Field(bid, 2:Int32)::Decimal) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } @@ -79,7 +79,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 2:Int32) as $expr3] } └─BatchFilter { predicate: (((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR (Field(bid, 0:Int32) = 2001:Int32)) OR (Field(bid, 0:Int32) = 2019:Int32)) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, _row_id] } @@ -124,12 +124,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr1, Field(auction, 7:Int32) as $expr2] } │ └─BatchFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, auction] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr3) } └─BatchProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, Field(person, 4:Int32) as $expr5, Field(person, 5:Int32) as $expr6] } └─BatchFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) } └─BatchProject { exprs: [event_type, person] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), $expr3(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, $expr3], pk_columns: [_row_id, _row_id#1, $expr3], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, $expr3, _row_id) } @@ -226,12 +226,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, $expr1, Field(auction, 6:Int32) as $expr3, Field(auction, 8:Int32) as $expr4] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr6) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr6, Field(bid, 2:Int32) as $expr7, $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr5, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [category, avg], stream_key: [category], pk_columns: [category], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr4, (sum(max($expr6)) / count(max($expr6))::Decimal) as $expr7] } @@ -359,7 +359,7 @@ │ └─LogicalShare { id: 3 } │ └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } │ └─LogicalFilter { predicate: (event_type = 2:Int32) } - │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } └─LogicalProject { exprs: [max(count), window_start] } └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } └─LogicalProject { exprs: [window_start, count] } @@ -371,7 +371,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [$expr2, count] } @@ -382,7 +382,7 @@ │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, $expr1] } │ └─BatchFilter { predicate: IsNotNull($expr1) AND (event_type = 2:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchHashAgg { group_key: [window_start], aggs: [max(count)] } └─BatchExchange { order: [], dist: HashShard(window_start) } └─BatchHashAgg { group_key: [$expr4, window_start], aggs: [count] } @@ -391,7 +391,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr4, $expr3] } └─BatchFilter { predicate: IsNotNull($expr3) AND (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr3, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck, watermark_columns: [window_start(hidden), window_start#1(hidden)] } └─StreamProject { exprs: [$expr2, count, window_start, window_start], output_watermarks: [window_start, window_start] } @@ -679,7 +679,7 @@ │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1] } │ └─BatchFilter { predicate: (event_type = 2:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard(max($expr7)) } └─BatchProject { exprs: [max($expr7), $expr6, ($expr6 - '00:00:10':Interval) as $expr8] } └─BatchHashAgg { group_key: [$expr6], aggs: [max($expr7)] } @@ -687,7 +687,7 @@ └─BatchProject { exprs: [(TumbleStart($expr5, '00:00:10':Interval) + '00:00:10':Interval) as $expr6, Field(bid, 2:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr5, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } └─StreamExchange { dist: HashShard($expr4, _row_id, $expr5) } @@ -831,14 +831,14 @@ │ └─BatchProject { exprs: [event_type, person, auction, bid, $expr1, _row_id, TumbleStart($expr1, '00:00:10':Interval) as $expr2] } │ └─BatchFilter { predicate: (event_type = 0:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchHashAgg { group_key: [$expr8, $expr7, $expr9], aggs: [] } └─BatchExchange { order: [], dist: HashShard($expr8, $expr7, $expr9) } └─BatchProject { exprs: [Field(auction, 7:Int32) as $expr8, $expr7, ($expr7 + '00:00:10':Interval) as $expr9] } └─BatchProject { exprs: [event_type, person, auction, bid, $expr6, _row_id, TumbleStart($expr6, '00:00:10':Interval) as $expr7] } └─BatchFilter { predicate: (event_type = 1:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr6, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) } @@ -976,20 +976,20 @@ ├─LogicalShare { id: 3 } │ └─LogicalProject { exprs: [Field(auction, 0:Int32) as $expr1, Field(auction, 1:Int32) as $expr2, Field(auction, 2:Int32) as $expr3, Field(auction, 3:Int32) as $expr4, Field(auction, 4:Int32) as $expr5, date_time, Field(auction, 6:Int32) as $expr6, Field(auction, 7:Int32) as $expr7, Field(auction, 8:Int32) as $expr8, Field(auction, 9:Int32) as $expr9] } │ └─LogicalFilter { predicate: (event_type = 1:Int32) } - │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } └─LogicalShare { id: 6 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr10, Field(bid, 1:Int32) as $expr11, Field(bid, 2:Int32) as $expr12, Field(bid, 3:Int32) as $expr13, Field(bid, 4:Int32) as $expr14, date_time, Field(bid, 6:Int32) as $expr15] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [$expr11 DESC, date_time ASC], limit: 1, offset: 0, group_key: [$expr1] } └─LogicalJoin { type: Inner, on: ($expr1 = $expr9) AND (date_time >= date_time) AND (date_time <= $expr6), output: all } ├─LogicalProject { exprs: [Field(auction, 0:Int32) as $expr1, Field(auction, 1:Int32) as $expr2, Field(auction, 2:Int32) as $expr3, Field(auction, 3:Int32) as $expr4, Field(auction, 4:Int32) as $expr5, date_time, Field(auction, 6:Int32) as $expr6, Field(auction, 7:Int32) as $expr7, Field(auction, 8:Int32) as $expr8] } │ └─LogicalFilter { predicate: (event_type = 1:Int32) } - │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr9, Field(bid, 1:Int32) as $expr10, Field(bid, 2:Int32) as $expr11, date_time] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [$expr13 DESC, $expr10 ASC], limit: 1, offset: 0, group_key: [$expr2] } @@ -998,12 +998,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, Field(auction, 2:Int32) as $expr4, Field(auction, 3:Int32) as $expr5, Field(auction, 4:Int32) as $expr6, $expr1, Field(auction, 6:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, Field(auction, 8:Int32) as $expr9] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr11) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr11, Field(bid, 1:Int32) as $expr12, Field(bid, 2:Int32) as $expr13, $expr10] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr10, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck, watermark_columns: [date_time, bid_date_time] } └─StreamGroupTopN [append_only] { order: [$expr12 DESC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr2], output_watermarks: [$expr1, $expr1] } @@ -1106,7 +1106,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, ToChar($expr1, 'YYYY-MM-DD':Varchar) as $expr5, ToChar($expr1, 'HH:MI':Varchar) as $expr6] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, ToChar($expr1, 'YYYY-MM-DD':Varchar) as $expr5, ToChar($expr1, 'HH:MI':Varchar) as $expr6, _row_id], output_watermarks: [$expr1] } @@ -1217,7 +1217,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, (0.908:Decimal * Field(bid, 2:Int32)::Decimal) as $expr4, Case(((Extract('HOUR':Varchar, $expr1) >= 8:Decimal) AND (Extract('HOUR':Varchar, $expr1) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, $expr1) <= 6:Decimal) OR (Extract('HOUR':Varchar, $expr1) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr5, $expr1, Field(bid, 6:Int32) as $expr6] } └─BatchFilter { predicate: ((0.908:Decimal * Field(bid, 2:Int32)::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * Field(bid, 2:Int32)::Decimal) < 50000000:Decimal) AND (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, (0.908:Decimal * Field(bid, 2:Int32)::Decimal) as $expr4, Case(((Extract('HOUR':Varchar, $expr1) >= 8:Decimal) AND (Extract('HOUR':Varchar, $expr1) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, $expr1) <= 6:Decimal) OR (Extract('HOUR':Varchar, $expr1) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr5, $expr1, Field(bid, 6:Int32) as $expr6, _row_id], output_watermarks: [$expr1] } @@ -1281,7 +1281,7 @@ └─BatchProject { exprs: [ToChar($expr1, 'yyyy-MM-dd':Varchar) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } └─StreamHashAgg [append_only] { group_key: [$expr2], aggs: [count, count filter(($expr3 < 10000:Int32)), count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count filter(($expr3 >= 1000000:Int32)), count(distinct $expr4), count(distinct $expr4) filter(($expr3 < 10000:Int32)), count(distinct $expr4) filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count(distinct $expr4) filter(($expr3 >= 1000000:Int32)), count(distinct $expr5), count(distinct $expr5) filter(($expr3 < 10000:Int32)), count(distinct $expr5) filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count(distinct $expr5) filter(($expr3 >= 1000000:Int32))] } @@ -1359,7 +1359,7 @@ └─BatchProject { exprs: [Field(bid, 3:Int32) as $expr2, ToChar($expr1, 'yyyy-MM-dd':Varchar) as $expr3, ToChar($expr1, 'HH:mm':Varchar) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 1:Int32) as $expr6, Field(bid, 0:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } └─StreamHashAgg [append_only] { group_key: [$expr2, $expr3], aggs: [max($expr4), count, count filter(($expr5 < 10000:Int32)), count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count filter(($expr5 >= 1000000:Int32)), count(distinct $expr6), count(distinct $expr6) filter(($expr5 < 10000:Int32)), count(distinct $expr6) filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count(distinct $expr6) filter(($expr5 >= 1000000:Int32)), count(distinct $expr7), count(distinct $expr7) filter(($expr5 < 10000:Int32)), count(distinct $expr7) filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count(distinct $expr7) filter(($expr5 >= 1000000:Int32))] } @@ -1430,7 +1430,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, ToChar($expr1, 'YYYY-MM-DD':Varchar) as $expr3, Field(bid, 2:Int32) as $expr4] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], stream_key: [auction, day], pk_columns: [auction, day], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr2, $expr3, count, count filter(($expr4 < 10000:Int32)), count filter(($expr4 >= 10000:Int32) AND ($expr4 < 1000000:Int32)), count filter(($expr4 >= 1000000:Int32)), min($expr4), max($expr4), (sum($expr4) / count($expr4)::Decimal) as $expr5, sum($expr4)] } @@ -1491,7 +1491,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [$expr1 DESC], limit: 1, offset: 0, group_key: [$expr3, $expr2] } @@ -1499,7 +1499,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, Field(bid, 6:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction], pk_columns: [bidder, auction], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamGroupTopN [append_only] { order: [$expr1 DESC], limit: 1, offset: 0, group_key: [$expr3, $expr2], output_watermarks: [$expr1] } @@ -1561,7 +1561,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [$expr1 DESC], limit: 1, offset: 0, with_ties: true, group_key: [$expr3, $expr2] } @@ -1569,7 +1569,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, Field(bid, 6:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction, _row_id], pk_columns: [bidder, auction, _row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } └─StreamGroupTopN [append_only] { order: [$expr1 DESC], limit: 1, offset: 0, with_ties: true, group_key: [$expr3, $expr2], output_watermarks: [$expr1] } @@ -1630,7 +1630,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5, date_time, Field(bid, 6:Int32) as $expr6] } └─LogicalFilter { predicate: (event_type = 2:Int32) } - └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: nexmark, columns: [event_type, person, auction, bid, date_time, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY $expr2 ORDER BY $expr4 DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1640,7 +1640,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1, Field(bid, 6:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden), rank_number], stream_key: [auction, _row_id], pk_columns: [auction, _row_id], pk_conflict: NoCheck } └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY $expr2 ORDER BY $expr4 DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1696,12 +1696,12 @@ │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1] } │ └─BatchFilter { predicate: (event_type = 2:Int32) } │ └─BatchProject { exprs: [event_type, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr8) } └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr8, Field(auction, 1:Int32) as $expr9, Field(auction, 2:Int32) as $expr10, Field(auction, 3:Int32) as $expr11, Field(auction, 4:Int32) as $expr12, $expr7, Field(auction, 6:Int32) as $expr13, Field(auction, 7:Int32) as $expr14, Field(auction, 8:Int32) as $expr15] } └─BatchFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } └─BatchProject { exprs: [event_type, auction, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr7] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr2, _row_id, _row_id) } @@ -1800,7 +1800,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5] } └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(Field(bid, 4:Int32), '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(Field(bid, 3:Int32)), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) AND (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, bid] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr2, $expr3, $expr4, $expr5, Case((Lower($expr5) = 'apple':Varchar), '0':Varchar, (Lower($expr5) = 'google':Varchar), '1':Varchar, (Lower($expr5) = 'facebook':Varchar), '2':Varchar, (Lower($expr5) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch($expr6, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr7, _row_id] } @@ -1842,7 +1842,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, bid] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr2, $expr3, $expr4, $expr5, SplitPart($expr6, '/':Varchar, 4:Int32) as $expr7, SplitPart($expr6, '/':Varchar, 5:Int32) as $expr8, SplitPart($expr6, '/':Varchar, 6:Int32) as $expr9, _row_id] } @@ -1899,13 +1899,13 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchHashAgg { group_key: [$expr5], aggs: [max($expr6)] } └─BatchExchange { order: [], dist: HashShard($expr5) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr2, _row_id) } @@ -2015,12 +2015,12 @@ │ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } │ └─BatchExchange { order: [], dist: HashShard($expr5) } │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5] } │ └─BatchFilter { predicate: (event_type = 2:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchProject { exprs: [(sum0(sum0(count)) / sum0(count($expr7))) as $expr8] } └─BatchSimpleAgg { aggs: [sum0(sum0(count)), sum0(count($expr7))] } └─BatchExchange { order: [], dist: Single } @@ -2030,7 +2030,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr6, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [auction_id, auction_item_name], pk_conflict: NoCheck } └─StreamDynamicFilter { predicate: (count($expr4) >= $expr5), output: [$expr2, $expr3, count($expr4)] } @@ -2174,7 +2174,7 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchProject { exprs: [$expr5] } └─BatchFilter { predicate: (count >= 20:Int32) } └─BatchHashAgg { group_key: [$expr5], aggs: [count] } @@ -2182,7 +2182,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr2, _row_id) } @@ -2285,7 +2285,7 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchProject { exprs: [$expr5] } └─BatchFilter { predicate: (count < 20:Int32) } └─BatchHashAgg { group_key: [$expr5], aggs: [count] } @@ -2293,7 +2293,7 @@ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr2, _row_id) } @@ -2399,12 +2399,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr5) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr2, $expr3, count($expr4)] } @@ -2525,12 +2525,12 @@ │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, $expr1, Field(auction, 6:Int32) as $expr3] } │ └─BatchFilter { predicate: (event_type = 1:Int32) } │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } └─BatchExchange { order: [], dist: HashShard($expr5) } └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, $expr4] } └─BatchFilter { predicate: (event_type = 2:Int32) } └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } - └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } + └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [min(min(max($expr5)))] } diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 5df21816d1fe..cfe5d841cb2c 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -27,10 +27,10 @@ └─BatchHashJoin { type: Inner, predicate: id = id, output: [] } ├─BatchExchange { order: [], dist: HashShard(id) } │ └─BatchFilter { predicate: (initial_bid = 1:Int32) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (initial_bid = 2:Int32) } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [sum0(count)] } @@ -92,7 +92,7 @@ │ └─LogicalProject { exprs: [window_start, auction] } │ └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } │ └─LogicalFilter { predicate: IsNotNull(date_time) } - │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─LogicalProject { exprs: [max(count), window_start] } └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } └─LogicalProject { exprs: [window_start, count] } @@ -101,7 +101,7 @@ └─LogicalProject { exprs: [auction, window_start] } └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } └─LogicalFilter { predicate: IsNotNull(date_time) } - └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [auction, count] } @@ -111,7 +111,7 @@ │ └─BatchExchange { order: [], dist: HashShard(auction) } │ └─BatchProject { exprs: [auction, date_time] } │ └─BatchFilter { predicate: IsNotNull(date_time) } - │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } └─BatchHashAgg { group_key: [window_start], aggs: [max(count)] } └─BatchExchange { order: [], dist: HashShard(window_start) } └─BatchHashAgg { group_key: [auction, window_start], aggs: [count] } @@ -119,7 +119,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchProject { exprs: [auction, date_time] } └─BatchFilter { predicate: IsNotNull(date_time) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, count, window_start, window_start] } @@ -298,11 +298,11 @@ ├─BatchExchange { order: [], dist: HashShard(id) } │ └─BatchFilter { predicate: (initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz) } │ └─BatchProject { exprs: [id, initial_bid, date_time] } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (initial_bid = 2:Int32) } └─BatchProject { exprs: [id, initial_bid, date_time] } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [a_id, b_id, a_ts, b_ts, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a_id], pk_columns: [_row_id, _row_id#1, a_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id, _row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/shared_source.yml b/src/frontend/planner_test/tests/testdata/output/shared_source.yml index 54f50fc2c687..5bf3739f2841 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_source.yml +++ b/src/frontend/planner_test/tests/testdata/output/shared_source.yml @@ -36,7 +36,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } @@ -52,7 +52,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } @@ -68,7 +68,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } @@ -84,7 +84,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 451ce0b70a7b..e17f2f3cf699 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -486,21 +486,21 @@ └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(a1, a1), output: [] } ├─LogicalShare { id: 2 } │ └─LogicalProject { exprs: [a1] } - │ └─LogicalSource { source: a, columns: [a1, a2, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: a, columns: [a1, a2, _row_id] } └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(a1, a1), output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalAgg { group_key: [a1], aggs: [] } │ │ └─LogicalShare { id: 2 } │ │ └─LogicalProject { exprs: [a1] } - │ │ └─LogicalSource { source: a, columns: [a1, a2, _row_id], time_range: (Unbounded, Unbounded) } + │ │ └─LogicalSource { source: a, columns: [a1, a2, _row_id] } │ └─LogicalAgg { aggs: [] } - │ └─LogicalSource { source: b, columns: [b1, b2, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: b, columns: [b1, b2, _row_id] } └─LogicalJoin { type: Inner, on: true, output: [a1] } ├─LogicalAgg { group_key: [a1], aggs: [] } │ └─LogicalShare { id: 2 } │ └─LogicalProject { exprs: [a1] } - │ └─LogicalSource { source: a, columns: [a1, a2, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: c, columns: [c1, c2, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: a, columns: [a1, a2, _row_id] } + └─LogicalSource { source: c, columns: [c1, c2, _row_id] } - name: test subquery in table function sql: | create table t(x int[], y int[], k int primary key); diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml index 89e884f96223..7b192ccceed4 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -136,11 +136,11 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } └─LogicalProject { exprs: [min(ps_supplycost)] } └─LogicalAgg { aggs: [min(ps_supplycost)] } └─LogicalProject { exprs: [ps_supplycost] } @@ -148,29 +148,29 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (n_regionkey = r_regionkey), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment] } ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, n_name, n_regionkey] } │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [p_partkey, p_mfgr, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] } │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = ps_partkey) AND (ps_supplycost = min(ps_supplycost)), output: [p_partkey, p_mfgr, ps_suppkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = ps_partkey), output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost] } - │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ │ │ └─LogicalAgg { group_key: [ps_partkey], aggs: [min(ps_supplycost)] } │ │ │ └─LogicalJoin { type: Inner, on: (n_regionkey = r_regionkey), output: [ps_supplycost, ps_partkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [ps_partkey, ps_supplycost, n_regionkey] } │ │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [ps_partkey, ps_supplycost, s_nationkey] } - │ │ │ │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } batch_plan: |- BatchExchange { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], dist: Single } └─BatchSort { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC] } @@ -183,9 +183,9 @@ │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND ps_supplycost = min(ps_supplycost), output: [p_partkey, p_mfgr, ps_suppkey] } │ │ │ ├─BatchHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } - │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ │ │ └─BatchHashAgg { group_key: [ps_partkey], aggs: [min(ps_supplycost)] } │ │ │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } │ │ │ └─BatchHashJoin { type: Inner, predicate: n_regionkey = r_regionkey, output: [ps_supplycost, ps_partkey] } @@ -194,19 +194,19 @@ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey] } │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(ps_suppkey) } - │ │ │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ │ │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - │ │ │ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), min(ps_supplycost)(hidden), ps_partkey(hidden), _row_id#1(hidden), _row_id#2(hidden), r_regionkey(hidden), _row_id#3(hidden), n_nationkey(hidden), _row_id#4(hidden), s_suppkey(hidden)], stream_key: [_row_id, p_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(p_partkey, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey) } @@ -488,12 +488,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [n_name], aggs: [sum($expr1)] } └─LogicalProject { exprs: [n_name, (l_extendedprice * (1:Int32::Decimal - l_discount)) as $expr1] } @@ -502,12 +502,12 @@ │ ├─LogicalJoin { type: Inner, on: (l_orderkey = o_orderkey) AND (l_suppkey = s_suppkey), output: [s_nationkey, l_extendedprice, l_discount] } │ │ ├─LogicalJoin { type: Inner, on: (c_nationkey = s_nationkey), output: [o_orderkey, s_suppkey, s_nationkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (c_custkey = o_custkey), output: [c_nationkey, o_orderkey] } - │ │ │ │ ├─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } batch_plan: |- BatchExchange { order: [sum($expr1) DESC], dist: Single } └─BatchSort { order: [sum($expr1) DESC] } @@ -524,17 +524,17 @@ │ │ │ ├─BatchExchange { order: [], dist: HashShard(c_nationkey) } │ │ │ │ └─BatchHashJoin { type: Inner, predicate: c_custkey = o_custkey, output: [c_nationkey, o_orderkey] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(c_custkey) } - │ │ │ │ │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(o_custkey) } - │ │ │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(s_nationkey) } - │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(l_orderkey, l_suppkey) } - │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [n_name, revenue], stream_key: [n_name], pk_columns: [revenue, n_name], pk_conflict: NoCheck } └─StreamProject { exprs: [n_name, sum($expr1)] } @@ -750,12 +750,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [n_name, n_name, $expr1], aggs: [sum($expr2)] } └─LogicalProject { exprs: [n_name, n_name, Extract('YEAR':Varchar, l_shipdate) as $expr1, (l_extendedprice * (1:Int32::Decimal - l_discount)) as $expr2] } @@ -764,12 +764,12 @@ │ ├─LogicalJoin { type: Inner, on: (o_orderkey = l_orderkey), output: [l_extendedprice, l_discount, l_shipdate, n_name, o_custkey] } │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [l_orderkey, l_extendedprice, l_discount, l_shipdate, n_name] } │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = l_suppkey), output: [s_nationkey, l_orderkey, l_extendedprice, l_discount, l_shipdate] } - │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } batch_plan: |- BatchExchange { order: [n_name ASC, n_name ASC, $expr1 ASC], dist: Single } └─BatchSort { order: [n_name ASC, n_name ASC, $expr1 ASC] } @@ -786,17 +786,17 @@ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } │ │ │ │ └─BatchHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_nationkey, l_orderkey, l_extendedprice, l_discount, l_shipdate] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(l_suppkey) } - │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(o_orderkey) } - │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(c_custkey) } - │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [supp_nation, cust_nation, l_year], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2)] } @@ -1020,14 +1020,14 @@ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [$expr1, RoundDigit((sum($expr3) / sum($expr2)), 6:Int32) as $expr4] } └─LogicalAgg { group_key: [$expr1], aggs: [sum($expr3), sum($expr2)] } @@ -1040,14 +1040,14 @@ │ │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [l_orderkey, l_extendedprice, l_discount, n_name] } │ │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = l_suppkey), output: [l_orderkey, l_extendedprice, l_discount, s_nationkey] } │ │ │ │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = l_partkey), output: [l_orderkey, l_suppkey, l_extendedprice, l_discount] } - │ │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ └─LogicalSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } batch_plan: |- BatchExchange { order: [$expr1 ASC], dist: Single } └─BatchProject { exprs: [$expr1, RoundDigit((sum($expr3) / sum($expr2)), 6:Int32) as $expr4] } @@ -1070,21 +1070,21 @@ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(l_suppkey) } │ │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_suppkey, l_extendedprice, l_discount] } │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ │ │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(l_partkey) } - │ │ │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(o_orderkey) } - │ │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(c_custkey) } - │ │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [o_year, mkt_share], stream_key: [o_year], pk_columns: [o_year], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr1, RoundDigit((sum($expr3) / sum($expr2)), 6:Int32) as $expr4] } @@ -1359,12 +1359,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } └─LogicalAgg { group_key: [n_name, $expr1], aggs: [sum($expr2)] } @@ -1374,12 +1374,12 @@ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, n_name] } │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = l_suppkey), output: [l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, s_nationkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = l_partkey), output: [l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] } - │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } batch_plan: |- BatchExchange { order: [n_name ASC, $expr1 DESC], dist: Single } └─BatchProject { exprs: [n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } @@ -1397,17 +1397,17 @@ │ │ │ ├─BatchExchange { order: [], dist: HashShard(l_suppkey) } │ │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(l_partkey) } - │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(ps_partkey, ps_suppkey) } - │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(o_orderkey) } - └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [nation, o_year], pk_columns: [nation, o_year], pk_conflict: NoCheck } └─StreamProject { exprs: [n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) as $expr3] } @@ -1607,34 +1607,34 @@ └─LogicalFilter { predicate: (s_nationkey = n_nationkey) } └─LogicalApply { type: LeftSemi, on: (s_suppkey = ps_suppkey), correlated_id: 1 } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─LogicalProject { exprs: [ps_suppkey] } └─LogicalFilter { predicate: (ps_availqty::Decimal > $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 3, max_one_row: true } ├─LogicalApply { type: LeftSemi, on: (ps_partkey = p_partkey), correlated_id: 2 } - │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ └─LogicalProject { exprs: [p_partkey] } - │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } └─LogicalProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr1] } └─LogicalAgg { aggs: [sum(l_quantity)] } └─LogicalProject { exprs: [l_quantity] } └─LogicalFilter { predicate: (l_partkey = CorrelatedInputRef { index: 0, correlated_id: 3 }) AND (l_suppkey = CorrelatedInputRef { index: 1, correlated_id: 3 }) } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (s_suppkey = ps_suppkey), output: [s_name, s_address] } ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [s_suppkey, s_name, s_address] } - │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─LogicalJoin { type: Inner, on: (ps_partkey = l_partkey) AND (ps_suppkey = l_suppkey) AND ($expr1 > $expr2), output: [ps_suppkey] } ├─LogicalProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1] } │ └─LogicalJoin { type: LeftSemi, on: (ps_partkey = p_partkey), output: [ps_partkey, ps_suppkey, ps_availqty] } - │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ └─LogicalProject { exprs: [p_partkey] } - │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } └─LogicalProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr2, l_partkey, l_suppkey] } └─LogicalAgg { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity)] } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } batch_plan: |- BatchExchange { order: [s_name ASC], dist: Single } └─BatchSort { order: [s_name ASC] } @@ -1642,23 +1642,23 @@ ├─BatchExchange { order: [], dist: HashShard(s_suppkey) } │ └─BatchHashJoin { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address] } │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } - │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(ps_suppkey) } └─BatchHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey AND ($expr1 > $expr2), output: [ps_suppkey] } ├─BatchExchange { order: [], dist: HashShard(ps_partkey, ps_suppkey) } │ └─BatchProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1] } │ └─BatchHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty] } │ ├─BatchExchange { order: [], dist: HashShard(ps_partkey) } - │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(p_partkey) } │ └─BatchProject { exprs: [p_partkey] } - │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } └─BatchProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr2, l_partkey, l_suppkey] } └─BatchHashAgg { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity)] } └─BatchExchange { order: [], dist: HashShard(l_partkey, l_suppkey) } - └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, _row_id, s_nationkey, s_suppkey) } @@ -1840,16 +1840,16 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ └─LogicalProject { exprs: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment] } │ └─LogicalFilter { predicate: (l_orderkey = CorrelatedInputRef { index: 8, correlated_id: 1 }) AND (l_suppkey <> CorrelatedInputRef { index: 10, correlated_id: 1 }) } - │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } └─LogicalProject { exprs: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment] } └─LogicalFilter { predicate: (l_orderkey = CorrelatedInputRef { index: 8, correlated_id: 2 }) AND (l_suppkey <> CorrelatedInputRef { index: 10, correlated_id: 2 }) AND (l_receiptdate > l_commitdate) } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [s_name], aggs: [count] } └─LogicalJoin { type: LeftAnti, on: (l_orderkey = l_orderkey) AND (l_suppkey <> l_suppkey), output: [s_name] } @@ -1857,16 +1857,16 @@ │ ├─LogicalJoin { type: Inner, on: (o_orderkey = l_orderkey), output: [s_name, l_orderkey, l_suppkey] } │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [s_name, l_orderkey, l_suppkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = l_suppkey), output: [s_name, s_nationkey, l_orderkey, l_suppkey] } - │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─LogicalFilter { predicate: (l_receiptdate > l_commitdate) } - │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ └─LogicalSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ └─LogicalProject { exprs: [l_orderkey, l_suppkey] } - │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } └─LogicalProject { exprs: [l_orderkey, l_suppkey] } └─LogicalFilter { predicate: (l_receiptdate > l_commitdate) } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } batch_plan: |- BatchExchange { order: [count DESC, s_name ASC], dist: Single } └─BatchSort { order: [count DESC, s_name ASC] } @@ -1880,21 +1880,21 @@ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } │ │ │ │ └─BatchHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, s_nationkey, l_orderkey, l_suppkey] } │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } │ │ │ │ └─BatchExchange { order: [], dist: HashShard(l_suppkey) } │ │ │ │ └─BatchFilter { predicate: (l_receiptdate > l_commitdate) } - │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } │ │ └─BatchExchange { order: [], dist: HashShard(o_orderkey) } - │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id], filter: (None, None) } + │ │ └─BatchSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } │ └─BatchExchange { order: [], dist: HashShard(l_orderkey) } │ └─BatchProject { exprs: [l_orderkey, l_suppkey] } - │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + │ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } └─BatchExchange { order: [], dist: HashShard(l_orderkey) } └─BatchProject { exprs: [l_orderkey, l_suppkey] } └─BatchFilter { predicate: (l_receiptdate > l_commitdate) } - └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } └─StreamHashAgg { group_key: [s_name], aggs: [count] } diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index 954734459861..67e098154ece 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -5,7 +5,7 @@ select t.v1 - INTERVAL '2' SECOND as v1 from t; logical_plan: |- LogicalProject { exprs: [(v1 - '00:00:02':Interval) as $expr1] } - └─LogicalSource { source: t, columns: [v1, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalSource { source: t, columns: [v1, _row_id] } stream_plan: |- StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamProject { exprs: [SubtractWithTimeZone(v1, '00:00:02':Interval, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 6448bd76d8c7..834c293f212b 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -424,6 +424,14 @@ static LOGICAL_FILTER_EXPRESSION_SIMPLIFY: LazyLock = LazyLoc ) }); +static REWRITE_SOURCE_FOR_BATCH: LazyLock = LazyLock::new(|| { + OptimizationStage::new( + "Rewrite Source For Batch", + vec![SourceToKafkaScanRule::create()], + ApplyOrder::TopDown, + ) +}); + impl LogicalOptimizer { pub fn predicate_pushdown( plan: PlanRef, @@ -661,6 +669,7 @@ impl LogicalOptimizer { // Convert the dag back to the tree, because we don't support DAG plan for batch. plan = plan.optimize_by_rules(&DAG_TO_TREE); + plan = plan.optimize_by_rules(&REWRITE_SOURCE_FOR_BATCH); plan = plan.optimize_by_rules(&GROUPING_SETS); plan = plan.optimize_by_rules(&REWRITE_LIKE_EXPR); plan = plan.optimize_by_rules(&SET_OPERATION_MERGE); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index c498f43b9520..f68f2f3bc9e0 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -974,6 +974,7 @@ fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> boo fn is_source(plan: &PlanRef) -> bool { plan.node_type() == PlanNodeType::BatchSource + || plan.node_type() == PlanNodeType::BatchKafkaScan } fn is_insert(plan: &PlanRef) -> bool { @@ -1005,6 +1006,7 @@ fn require_additional_exchange_on_root_in_local_mode(plan: PlanRef) -> bool { fn is_source(plan: &PlanRef) -> bool { plan.node_type() == PlanNodeType::BatchSource + || plan.node_type() == PlanNodeType::BatchKafkaScan } fn is_insert(plan: &PlanRef) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/batch_kafka_scan.rs b/src/frontend/src/optimizer/plan_node/batch_kafka_scan.rs new file mode 100644 index 000000000000..423fe2a6771e --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/batch_kafka_scan.rs @@ -0,0 +1,140 @@ +// 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::ops::Bound; +use std::ops::Bound::{Excluded, Included, Unbounded}; +use std::rc::Rc; + +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_pb::batch_plan::plan_node::NodeBody; +use risingwave_pb::batch_plan::SourceNode; + +use super::batch::prelude::*; +use super::utils::{childless_record, column_names_pretty, Distill}; +use super::{ + generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, +}; +use crate::catalog::source_catalog::SourceCatalog; +use crate::error::Result; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::property::{Distribution, Order}; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct BatchKafkaScan { + pub base: PlanBase, + pub core: generic::Source, + + /// Kafka timestamp range. + kafka_timestamp_range: (Bound, Bound), +} + +impl BatchKafkaScan { + pub fn new(core: generic::Source, kafka_timestamp_range: (Bound, Bound)) -> Self { + let base = PlanBase::new_batch_with_core( + &core, + // Use `Single` by default, will be updated later with `clone_with_dist`. + Distribution::Single, + Order::any(), + ); + + Self { + base, + core, + kafka_timestamp_range, + } + } + + pub fn column_names(&self) -> Vec<&str> { + self.schema().names_str() + } + + pub fn source_catalog(&self) -> Option> { + self.core.catalog.clone() + } + + pub fn kafka_timestamp_range_value(&self) -> (Option, Option) { + let (lower_bound, upper_bound) = &self.kafka_timestamp_range; + let lower_bound = match lower_bound { + Included(t) => Some(*t), + Excluded(t) => Some(*t - 1), + Unbounded => None, + }; + + let upper_bound = match upper_bound { + Included(t) => Some(*t), + Excluded(t) => Some(*t + 1), + Unbounded => None, + }; + (lower_bound, upper_bound) + } + + pub fn clone_with_dist(&self) -> Self { + let base = self + .base + .clone_with_new_distribution(Distribution::SomeShard); + Self { + base, + core: self.core.clone(), + kafka_timestamp_range: self.kafka_timestamp_range, + } + } +} + +impl_plan_tree_node_for_leaf! { BatchKafkaScan } + +impl Distill for BatchKafkaScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let src = Pretty::from(self.source_catalog().unwrap().name.clone()); + let fields = vec![ + ("source", src), + ("columns", column_names_pretty(self.schema())), + ("filter", Pretty::debug(&self.kafka_timestamp_range_value())), + ]; + childless_record("BatchKafkaScan", fields) + } +} + +impl ToLocalBatch for BatchKafkaScan { + fn to_local(&self) -> Result { + Ok(self.clone_with_dist().into()) + } +} + +impl ToDistributedBatch for BatchKafkaScan { + fn to_distributed(&self) -> Result { + Ok(self.clone_with_dist().into()) + } +} + +impl ToBatchPb for BatchKafkaScan { + fn to_batch_prost_body(&self) -> NodeBody { + let source_catalog = self.source_catalog().unwrap(); + NodeBody::Source(SourceNode { + source_id: source_catalog.id, + info: Some(source_catalog.info.clone()), + columns: self + .core + .column_catalog + .iter() + .map(|c| c.to_protobuf()) + .collect(), + with_properties: source_catalog.with_properties.clone().into_iter().collect(), + split: vec![], + }) + } +} + +impl ExprRewritable for BatchKafkaScan {} + +impl ExprVisitable for BatchKafkaScan {} diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index a0ece261dcea..3836940be745 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -56,10 +56,6 @@ impl BatchSource { self.core.catalog.clone() } - pub fn kafka_timestamp_range_value(&self) -> (Option, Option) { - self.core.kafka_timestamp_range_value() - } - pub fn as_of(&self) -> Option { self.core.as_of.clone() } @@ -83,7 +79,6 @@ impl Distill for BatchSource { let mut fields = vec![ ("source", src), ("columns", column_names_pretty(self.schema())), - ("filter", Pretty::debug(&self.kafka_timestamp_range_value())), ]; if let Some(as_of) = &self.core.as_of { fields.push(("as_of", Pretty::debug(as_of))); diff --git a/src/frontend/src/optimizer/plan_node/generic/source.rs b/src/frontend/src/optimizer/plan_node/generic/source.rs index 0c1caa620a4b..40bd376dd46c 100644 --- a/src/frontend/src/optimizer/plan_node/generic/source.rs +++ b/src/frontend/src/optimizer/plan_node/generic/source.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::Bound; -use std::ops::Bound::{Excluded, Included, Unbounded}; use std::rc::Rc; use educe::Educe; @@ -65,9 +63,6 @@ pub struct Source { #[educe(Hash(ignore))] pub ctx: OptimizerContextRef, - /// Kafka timestamp range, currently we only support kafka, so we just leave it like this. - pub(crate) kafka_timestamp_range: (Bound, Bound), - pub as_of: Option, } @@ -113,6 +108,12 @@ impl Source { .is_some_and(|catalog| catalog.with_properties.is_iceberg_connector()) } + pub fn is_kafka_connector(&self) -> bool { + self.catalog + .as_ref() + .is_some_and(|catalog| catalog.with_properties.is_kafka_connector()) + } + /// Currently, only iceberg source supports time travel. pub fn support_time_travel(&self) -> bool { self.is_iceberg_connector() @@ -136,22 +137,6 @@ impl Source { (self, original_row_id_index) } - pub fn kafka_timestamp_range_value(&self) -> (Option, Option) { - let (lower_bound, upper_bound) = &self.kafka_timestamp_range; - let lower_bound = match lower_bound { - Included(t) => Some(*t), - Excluded(t) => Some(*t - 1), - Unbounded => None, - }; - - let upper_bound = match upper_bound { - Included(t) => Some(*t), - Excluded(t) => Some(*t + 1), - Unbounded => None, - }; - (lower_bound, upper_bound) - } - pub fn infer_internal_table_catalog(require_dist_key: bool) -> TableCatalog { // note that source's internal table is to store partition_id -> offset mapping and its // schema is irrelevant to input schema diff --git a/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs b/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs new file mode 100644 index 000000000000..18ae05a135ff --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs @@ -0,0 +1,346 @@ +// 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::cmp::{max, min}; +use std::ops::Bound; +use std::ops::Bound::{Excluded, Included, Unbounded}; +use std::rc::Rc; + +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{Schema, KAFKA_TIMESTAMP_COLUMN_NAME}; +use risingwave_connector::source::DataType; + +use super::generic::GenericPlanRef; +use super::utils::{childless_record, Distill}; +use super::{ + generic, BatchProject, ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, + PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; +use crate::catalog::source_catalog::SourceCatalog; +use crate::error::Result; +use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor}; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::utils::column_names_pretty; +use crate::optimizer::plan_node::{ + BatchKafkaScan, ColumnPruningContext, LogicalSource, PredicatePushdownContext, + RewriteStreamContext, ToStreamContext, +}; +use crate::utils::{ColIndexMapping, Condition}; + +/// `LogicalKafkaScan` is only used by batch queries. At the beginning of the batch query optimization, `LogicalSource` with a kafka property would be converted into a `LogicalKafkaScan`. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalKafkaScan { + pub base: PlanBase, + pub core: generic::Source, + + /// Expressions to output. This field presents and will be turned to a `Project` when + /// converting to a physical plan, only if there are generated columns. + output_exprs: Option>, + + /// Kafka timestamp range. + kafka_timestamp_range: (Bound, Bound), +} + +impl LogicalKafkaScan { + pub fn new(logical_source: &LogicalSource) -> Self { + assert!(logical_source.core.is_kafka_connector()); + + let base = logical_source.base.clone_with_new_plan_id(); + let core = logical_source.core.clone(); + let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); + + LogicalKafkaScan { + base, + core, + output_exprs: logical_source.output_exprs.clone(), + kafka_timestamp_range, + } + } + + pub fn source_catalog(&self) -> Option> { + self.core.catalog.clone() + } + + fn clone_with_kafka_timestamp_range(&self, range: (Bound, Bound)) -> Self { + Self { + base: self.base.clone(), + core: self.core.clone(), + output_exprs: self.output_exprs.clone(), + kafka_timestamp_range: range, + } + } +} + +impl_plan_tree_node_for_leaf! {LogicalKafkaScan} +impl Distill for LogicalKafkaScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let fields = if let Some(catalog) = self.source_catalog() { + let src = Pretty::from(catalog.name.clone()); + let time = Pretty::debug(&self.kafka_timestamp_range); + vec![ + ("source", src), + ("columns", column_names_pretty(self.schema())), + ("time_range", time), + ] + } else { + vec![] + }; + childless_record("LogicalKafkaScan", fields) + } +} + +impl ColPrunable for LogicalKafkaScan { + fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len()); + LogicalProject::with_mapping(self.clone().into(), mapping).into() + } +} + +impl ExprRewritable for LogicalKafkaScan { + fn has_rewritable_expr(&self) -> bool { + self.output_exprs.is_some() + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut output_exprs = self.output_exprs.clone(); + + for expr in output_exprs.iter_mut().flatten() { + *expr = r.rewrite_expr(expr.clone()); + } + + Self { + output_exprs, + ..self.clone() + } + .into() + } +} + +impl ExprVisitable for LogicalKafkaScan { + fn visit_exprs(&self, v: &mut dyn ExprVisitor) { + self.output_exprs + .iter() + .flatten() + .for_each(|e| v.visit_expr(e)); + } +} + +/// A util function to extract kafka offset timestamp range. +/// +/// Currently we only support limiting kafka offset timestamp range using literals, e.g. we only +/// support expressions like `_rw_kafka_timestamp <= '2022-10-11 1:00:00+00:00'`. +/// +/// # Parameters +/// +/// * `expr`: Expression to be consumed. +/// * `range`: Original timestamp range, if `expr` can be recognized, we will update `range`. +/// * `schema`: Input schema. +/// +/// # Return Value +/// +/// If `expr` can be recognized and consumed by this function, then we return `None`. +/// Otherwise `expr` is returned. +fn expr_to_kafka_timestamp_range( + expr: ExprImpl, + range: &mut (Bound, Bound), + schema: &Schema, +) -> Option { + let merge_upper_bound = |first, second| -> Bound { + match (first, second) { + (first, Unbounded) => first, + (Unbounded, second) => second, + (Included(f1), Included(f2)) => Included(min(f1, f2)), + (Included(f1), Excluded(f2)) => { + if f1 < f2 { + Included(f1) + } else { + Excluded(f2) + } + } + (Excluded(f1), Included(f2)) => { + if f2 < f1 { + Included(f2) + } else { + Excluded(f1) + } + } + (Excluded(f1), Excluded(f2)) => Excluded(min(f1, f2)), + } + }; + + let merge_lower_bound = |first, second| -> Bound { + match (first, second) { + (first, Unbounded) => first, + (Unbounded, second) => second, + (Included(f1), Included(f2)) => Included(max(f1, f2)), + (Included(f1), Excluded(f2)) => { + if f1 > f2 { + Included(f1) + } else { + Excluded(f2) + } + } + (Excluded(f1), Included(f2)) => { + if f2 > f1 { + Included(f2) + } else { + Excluded(f1) + } + } + (Excluded(f1), Excluded(f2)) => Excluded(max(f1, f2)), + } + }; + + let extract_timestampz_literal = |expr: &ExprImpl| -> Result> { + match expr { + ExprImpl::FunctionCall(function_call) if function_call.inputs().len() == 2 => { + match (&function_call.inputs()[0], &function_call.inputs()[1]) { + (ExprImpl::InputRef(input_ref), literal) + if let Some(datum) = literal.try_fold_const().transpose()? + && schema.fields[input_ref.index].name + == KAFKA_TIMESTAMP_COLUMN_NAME + && literal.return_type() == DataType::Timestamptz => + { + Ok(Some(( + datum.unwrap().into_timestamptz().timestamp_millis(), + false, + ))) + } + (literal, ExprImpl::InputRef(input_ref)) + if let Some(datum) = literal.try_fold_const().transpose()? + && schema.fields[input_ref.index].name + == KAFKA_TIMESTAMP_COLUMN_NAME + && literal.return_type() == DataType::Timestamptz => + { + Ok(Some(( + datum.unwrap().into_timestamptz().timestamp_millis(), + true, + ))) + } + _ => Ok(None), + } + } + _ => Ok(None), + } + }; + + match &expr { + ExprImpl::FunctionCall(function_call) => { + if let Ok(Some((timestampz_literal, reverse))) = extract_timestampz_literal(&expr) { + match function_call.func_type() { + ExprType::GreaterThan => { + if reverse { + range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); + } else { + range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); + } + + None + } + ExprType::GreaterThanOrEqual => { + if reverse { + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + } else { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + } + None + } + ExprType::Equal => { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + None + } + ExprType::LessThan => { + if reverse { + range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); + } else { + range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); + } + None + } + ExprType::LessThanOrEqual => { + if reverse { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + } else { + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + } + None + } + _ => Some(expr), + } + } else { + Some(expr) + } + } + _ => Some(expr), + } +} + +impl PredicatePushdown for LogicalKafkaScan { + fn predicate_pushdown( + &self, + predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + let mut range = self.kafka_timestamp_range; + + let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); + for expr in predicate.conjunctions { + if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, self.base.schema()) { + // Not recognized, so push back + new_conjunctions.push(e); + } + } + + let new_source = self.clone_with_kafka_timestamp_range(range).into(); + + if new_conjunctions.is_empty() { + new_source + } else { + LogicalFilter::create( + new_source, + Condition { + conjunctions: new_conjunctions, + }, + ) + } + } +} + +impl ToBatch for LogicalKafkaScan { + fn to_batch(&self) -> Result { + let mut plan: PlanRef = + BatchKafkaScan::new(self.core.clone(), self.kafka_timestamp_range).into(); + + if let Some(exprs) = &self.output_exprs { + let logical_project = generic::Project::new(exprs.to_vec(), plan); + plan = BatchProject::new(logical_project).into(); + } + + Ok(plan) + } +} + +impl ToStream for LogicalKafkaScan { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + unreachable!() + } + + fn logical_rewrite_for_stream( + &self, + _ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + unreachable!() + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index fd4aaba5b7d2..8a8fd37bd66b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -12,17 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cmp::{max, min}; -use std::ops::Bound; -use std::ops::Bound::{Excluded, Included, Unbounded}; use std::rc::Rc; use fixedbitset::FixedBitSet; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::bail; -use risingwave_common::catalog::{ - ColumnCatalog, ColumnDesc, Field, Schema, KAFKA_TIMESTAMP_COLUMN_NAME, -}; +use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, Field}; use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR; use risingwave_connector::source::{DataType, UPSTREAM_SOURCE_KEY}; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; @@ -39,7 +34,7 @@ use super::{ }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; -use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, InputRef}; +use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::stream_fs_fetch::StreamFsFetch; @@ -60,10 +55,10 @@ pub struct LogicalSource { /// Expressions to output. This field presents and will be turned to a `Project` when /// converting to a physical plan, only if there are generated columns. - output_exprs: Option>, + pub(crate) output_exprs: Option>, /// When there are generated columns, the `StreamRowIdGen`'s `row_id_index` is different from /// the one in `core`. So we store the one in `output_exprs` here. - output_row_id_index: Option, + pub(crate) output_row_id_index: Option, } impl LogicalSource { @@ -75,15 +70,12 @@ impl LogicalSource { ctx: OptimizerContextRef, as_of: Option, ) -> Result { - let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); - let core = generic::Source { catalog: source_catalog, column_catalog, row_id_index, kind, ctx, - kafka_timestamp_range, as_of, }; @@ -249,17 +241,6 @@ impl LogicalSource { pub fn source_catalog(&self) -> Option> { self.core.catalog.clone() } - - fn clone_with_kafka_timestamp_range(&self, range: (Bound, Bound)) -> Self { - let mut core = self.core.clone(); - core.kafka_timestamp_range = range; - Self { - base: self.base.clone(), - core, - output_exprs: self.output_exprs.clone(), - output_row_id_index: self.output_row_id_index, - } - } } impl_plan_tree_node_for_leaf! {LogicalSource} @@ -267,11 +248,9 @@ impl Distill for LogicalSource { fn distill<'a>(&self) -> XmlNode<'a> { let fields = if let Some(catalog) = self.source_catalog() { let src = Pretty::from(catalog.name.clone()); - let time = Pretty::debug(&self.core.kafka_timestamp_range); let mut fields = vec![ ("source", src), ("columns", column_names_pretty(self.schema())), - ("time_range", time), ]; if let Some(as_of) = &self.core.as_of { fields.push(("as_of", Pretty::debug(as_of))); @@ -320,190 +299,22 @@ impl ExprVisitable for LogicalSource { } } -/// A util function to extract kafka offset timestamp range. -/// -/// Currently we only support limiting kafka offset timestamp range using literals, e.g. we only -/// support expressions like `_rw_kafka_timestamp <= '2022-10-11 1:00:00+00:00'`. -/// -/// # Parameters -/// -/// * `expr`: Expression to be consumed. -/// * `range`: Original timestamp range, if `expr` can be recognized, we will update `range`. -/// * `schema`: Input schema. -/// -/// # Return Value -/// -/// If `expr` can be recognized and consumed by this function, then we return `None`. -/// Otherwise `expr` is returned. -fn expr_to_kafka_timestamp_range( - expr: ExprImpl, - range: &mut (Bound, Bound), - schema: &Schema, -) -> Option { - let merge_upper_bound = |first, second| -> Bound { - match (first, second) { - (first, Unbounded) => first, - (Unbounded, second) => second, - (Included(f1), Included(f2)) => Included(min(f1, f2)), - (Included(f1), Excluded(f2)) => { - if f1 < f2 { - Included(f1) - } else { - Excluded(f2) - } - } - (Excluded(f1), Included(f2)) => { - if f2 < f1 { - Included(f2) - } else { - Excluded(f1) - } - } - (Excluded(f1), Excluded(f2)) => Excluded(min(f1, f2)), - } - }; - - let merge_lower_bound = |first, second| -> Bound { - match (first, second) { - (first, Unbounded) => first, - (Unbounded, second) => second, - (Included(f1), Included(f2)) => Included(max(f1, f2)), - (Included(f1), Excluded(f2)) => { - if f1 > f2 { - Included(f1) - } else { - Excluded(f2) - } - } - (Excluded(f1), Included(f2)) => { - if f2 > f1 { - Included(f2) - } else { - Excluded(f1) - } - } - (Excluded(f1), Excluded(f2)) => Excluded(max(f1, f2)), - } - }; - - let extract_timestampz_literal = |expr: &ExprImpl| -> Result> { - match expr { - ExprImpl::FunctionCall(function_call) if function_call.inputs().len() == 2 => { - match (&function_call.inputs()[0], &function_call.inputs()[1]) { - (ExprImpl::InputRef(input_ref), literal) - if let Some(datum) = literal.try_fold_const().transpose()? - && schema.fields[input_ref.index].name - == KAFKA_TIMESTAMP_COLUMN_NAME - && literal.return_type() == DataType::Timestamptz => - { - Ok(Some(( - datum.unwrap().into_timestamptz().timestamp_millis(), - false, - ))) - } - (literal, ExprImpl::InputRef(input_ref)) - if let Some(datum) = literal.try_fold_const().transpose()? - && schema.fields[input_ref.index].name - == KAFKA_TIMESTAMP_COLUMN_NAME - && literal.return_type() == DataType::Timestamptz => - { - Ok(Some(( - datum.unwrap().into_timestamptz().timestamp_millis(), - true, - ))) - } - _ => Ok(None), - } - } - _ => Ok(None), - } - }; - - match &expr { - ExprImpl::FunctionCall(function_call) => { - if let Ok(Some((timestampz_literal, reverse))) = extract_timestampz_literal(&expr) { - match function_call.func_type() { - ExprType::GreaterThan => { - if reverse { - range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); - } else { - range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); - } - - None - } - ExprType::GreaterThanOrEqual => { - if reverse { - range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); - } else { - range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); - } - None - } - ExprType::Equal => { - range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); - range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); - None - } - ExprType::LessThan => { - if reverse { - range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); - } else { - range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); - } - None - } - ExprType::LessThanOrEqual => { - if reverse { - range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); - } else { - range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); - } - None - } - _ => Some(expr), - } - } else { - Some(expr) - } - } - _ => Some(expr), - } -} - impl PredicatePushdown for LogicalSource { fn predicate_pushdown( &self, predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { - let mut range = self.core.kafka_timestamp_range; - - let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); - for expr in predicate.conjunctions { - if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, self.base.schema()) { - // Not recognized, so push back - new_conjunctions.push(e); - } - } - - let new_source = self.clone_with_kafka_timestamp_range(range).into(); - - if new_conjunctions.is_empty() { - new_source - } else { - LogicalFilter::create( - new_source, - Condition { - conjunctions: new_conjunctions, - }, - ) - } + LogicalFilter::create(self.clone().into(), predicate) } } impl ToBatch for LogicalSource { fn to_batch(&self) -> Result { + assert!( + !self.core.is_kafka_connector(), + "LogicalSource with a kafka property should be converted to LogicalKafkaScan" + ); let mut plan: PlanRef = BatchSource::new(self.core.clone()).into(); if let Some(exprs) = &self.output_exprs { diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index ff749781f926..c2c46e0766ef 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -856,6 +856,7 @@ mod logical_hop_window; mod logical_insert; mod logical_intersect; mod logical_join; +mod logical_kafka_scan; mod logical_limit; mod logical_max_one_row; mod logical_multi_join; @@ -903,6 +904,7 @@ mod stream_topn; mod stream_values; mod stream_watermark_filter; +mod batch_kafka_scan; mod derive; mod stream_cdc_table_scan; mod stream_share; @@ -919,6 +921,7 @@ pub use batch_hash_agg::BatchHashAgg; pub use batch_hash_join::BatchHashJoin; pub use batch_hop_window::BatchHopWindow; pub use batch_insert::BatchInsert; +pub use batch_kafka_scan::BatchKafkaScan; pub use batch_limit::BatchLimit; pub use batch_lookup_join::BatchLookupJoin; pub use batch_max_one_row::BatchMaxOneRow; @@ -949,6 +952,7 @@ pub use logical_hop_window::LogicalHopWindow; pub use logical_insert::LogicalInsert; pub use logical_intersect::LogicalIntersect; pub use logical_join::LogicalJoin; +pub use logical_kafka_scan::LogicalKafkaScan; pub use logical_limit::LogicalLimit; pub use logical_max_one_row::LogicalMaxOneRow; pub use logical_multi_join::{LogicalMultiJoin, LogicalMultiJoinBuilder}; @@ -1053,6 +1057,7 @@ macro_rules! for_all_plan_nodes { , { Logical, Intersect } , { Logical, Except } , { Logical, MaxOneRow } + , { Logical, KafkaScan } , { Batch, SimpleAgg } , { Batch, HashAgg } , { Batch, SortAgg } @@ -1080,6 +1085,7 @@ macro_rules! for_all_plan_nodes { , { Batch, Source } , { Batch, OverWindow } , { Batch, MaxOneRow } + , { Batch, KafkaScan } , { Stream, Project } , { Stream, Filter } , { Stream, TableScan } @@ -1151,6 +1157,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, Intersect } , { Logical, Except } , { Logical, MaxOneRow } + , { Logical, KafkaScan } } }; } @@ -1187,6 +1194,7 @@ macro_rules! for_batch_plan_nodes { , { Batch, Source } , { Batch, OverWindow } , { Batch, MaxOneRow } + , { Batch, KafkaScan } } }; } diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index dff3f986ce22..e52e0ca7a8b3 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -157,9 +157,11 @@ pub use apply_hop_window_transpose_rule::*; mod agg_call_merge_rule; pub use agg_call_merge_rule::*; mod pull_up_correlated_predicate_agg_rule; +mod source_to_kafka_scan_rule; mod values_extract_project_rule; pub use batch::batch_push_limit_to_scan_rule::*; pub use pull_up_correlated_predicate_agg_rule::*; +pub use source_to_kafka_scan_rule::*; pub use values_extract_project_rule::*; #[macro_export] @@ -233,6 +235,7 @@ macro_rules! for_all_rules { , { ValuesExtractProjectRule } , { BatchPushLimitToScanRule } , { PullUpCorrelatedPredicateAggRule } + , { SourceToKafkaScanRule } } }; } diff --git a/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs b/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs new file mode 100644 index 000000000000..244278bdc33c --- /dev/null +++ b/src/frontend/src/optimizer/rule/source_to_kafka_scan_rule.rs @@ -0,0 +1,35 @@ +// 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 super::{BoxedRule, Rule}; +use crate::optimizer::plan_node::{LogicalKafkaScan, LogicalSource}; +use crate::optimizer::PlanRef; + +pub struct SourceToKafkaScanRule {} +impl Rule for SourceToKafkaScanRule { + fn apply(&self, plan: PlanRef) -> Option { + let source: &LogicalSource = plan.as_logical_source()?; + if source.core.is_kafka_connector() { + Some(LogicalKafkaScan::new(source).into()) + } else { + None + } + } +} + +impl SourceToKafkaScanRule { + pub fn create() -> BoxedRule { + Box::new(SourceToKafkaScanRule {}) + } +} diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index 9a5e00fbfd70..32dac7200619 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -987,7 +987,7 @@ impl StageRunner { node_body: Some(NodeBody::RowSeqScan(scan_node)), } } - PlanNodeType::BatchSource => { + PlanNodeType::BatchSource | PlanNodeType::BatchKafkaScan => { let node_body = execution_plan_node.node.clone(); let NodeBody::Source(mut source_node) = node_body else { unreachable!(); diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index ede32d203335..c6340189743f 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -472,7 +472,7 @@ impl LocalQueryExecution { node_body: Some(node_body), }) } - PlanNodeType::BatchSource => { + PlanNodeType::BatchSource | PlanNodeType::BatchKafkaScan => { let mut node_body = execution_plan_node.node.clone(); match &mut node_body { NodeBody::Source(ref mut source_node) => { diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index d95f4d627926..a578fd30d253 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -54,7 +54,7 @@ use crate::catalog::catalog_service::CatalogReader; use crate::catalog::TableId; use crate::error::RwError; use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; -use crate::optimizer::plan_node::{BatchSource, PlanNodeId, PlanNodeType}; +use crate::optimizer::plan_node::{BatchKafkaScan, BatchSource, PlanNodeId, PlanNodeType}; use crate::optimizer::property::Distribution; use crate::optimizer::PlanRef; use crate::scheduler::SchedulerResult; @@ -1003,7 +1003,23 @@ impl BatchPlanFragmenter { return Ok(None); } - if let Some(source_node) = node.as_batch_source() { + if let Some(batch_kafka_node) = node.as_batch_kafka_scan() { + let batch_kafka_scan: &BatchKafkaScan = batch_kafka_node; + let source_catalog = batch_kafka_scan.source_catalog(); + if let Some(source_catalog) = source_catalog { + let property = ConnectorProperties::extract( + source_catalog.with_properties.clone().into_iter().collect(), + false, + )?; + let timestamp_bound = batch_kafka_scan.kafka_timestamp_range_value(); + return Ok(Some(SourceScanInfo::new(SourceFetchInfo { + connector: property, + timebound: timestamp_bound, + as_of: None, + }))); + } + } else if let Some(source_node) = node.as_batch_source() { + // TODO: use specific batch operator instead of batch source. let source_node: &BatchSource = source_node; let source_catalog = source_node.source_catalog(); if let Some(source_catalog) = source_catalog { @@ -1011,11 +1027,10 @@ impl BatchPlanFragmenter { source_catalog.with_properties.clone().into_iter().collect(), false, )?; - let timestamp_bound = source_node.kafka_timestamp_range_value(); let as_of = source_node.as_of(); return Ok(Some(SourceScanInfo::new(SourceFetchInfo { connector: property, - timebound: timestamp_bound, + timebound: (None, None), as_of, }))); } From 3445741da6136c4c0e99876794fe9f4e33889487 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Thu, 18 Apr 2024 18:14:41 +0800 Subject: [PATCH 61/64] refactor(source): remove `Default` impl from `SourceContext` and `SourceEnumeratorContext` (#16379) Signed-off-by: Richard Chien --- src/batch/src/executor/source.rs | 2 +- src/bench/sink_bench/main.rs | 16 ++++--- src/compute/tests/integration_tests.rs | 5 +- src/connector/benches/json_parser.rs | 2 +- src/connector/benches/nexmark_integration.rs | 5 +- src/connector/benches/parser.rs | 4 +- src/connector/src/lib.rs | 1 + src/connector/src/parser/avro/parser.rs | 4 +- src/connector/src/parser/bytes_parser.rs | 3 +- .../src/parser/canal/simd_json_parser.rs | 6 +-- src/connector/src/parser/csv_parser.rs | 4 +- .../src/parser/debezium/avro_parser.rs | 11 +++-- .../src/parser/debezium/debezium_parser.rs | 13 ++--- .../src/parser/debezium/mongo_json_parser.rs | 6 ++- .../src/parser/debezium/simd_json_parser.rs | 33 ++++++------- src/connector/src/parser/json_parser.rs | 17 +++---- .../src/parser/maxwell/simd_json_parser.rs | 4 +- src/connector/src/parser/plain_parser.rs | 4 +- src/connector/src/sink/kafka.rs | 2 +- src/connector/src/source/base.rs | 47 ++++++++++++++----- .../src/source/datagen/source/reader.rs | 7 +-- .../src/source/filesystem/s3/enumerator.rs | 2 +- .../src/source/filesystem/s3/source/reader.rs | 8 ++-- .../src/source/kinesis/source/reader.rs | 7 +-- .../src/source/nexmark/source/reader.rs | 10 ++-- src/frontend/src/scheduler/plan_fragmenter.rs | 4 +- src/meta/service/src/cloud_service.rs | 2 +- src/meta/src/rpc/ddl_controller.rs | 2 +- .../src/executor/backfill/cdc/cdc_backfill.rs | 2 +- .../src/executor/source/fetch_executor.rs | 2 +- .../src/executor/source/fs_source_executor.rs | 2 +- .../source/source_backfill_executor.rs | 2 +- .../src/executor/source/source_executor.rs | 16 +++++-- .../src/from_proto/source/trad_source.rs | 5 +- 34 files changed, 156 insertions(+), 104 deletions(-) diff --git a/src/batch/src/executor/source.rs b/src/batch/src/executor/source.rs index 9a348b25a402..cbc69444af7f 100644 --- a/src/batch/src/executor/source.rs +++ b/src/batch/src/executor/source.rs @@ -171,10 +171,10 @@ impl SourceExecutor { u32::MAX, self.source_id, u32::MAX, + "NA".to_owned(), // source name was not passed in batch plan self.metrics, self.source_ctrl_opts.clone(), ConnectorProperties::default(), - "NA".to_owned(), // source name was not passed in batch plan )); let stream = self .source diff --git a/src/bench/sink_bench/main.rs b/src/bench/sink_bench/main.rs index 577a8d21d613..91ebe6bd44d0 100644 --- a/src/bench/sink_bench/main.rs +++ b/src/bench/sink_bench/main.rs @@ -46,7 +46,9 @@ use risingwave_connector::sink::{ use risingwave_connector::source::datagen::{ DatagenProperties, DatagenSplitEnumerator, DatagenSplitReader, }; -use risingwave_connector::source::{Column, DataType, SplitEnumerator, SplitReader}; +use risingwave_connector::source::{ + Column, DataType, SourceContext, SourceEnumeratorContext, SplitEnumerator, SplitReader, +}; use risingwave_pb::connector_service::SinkPayloadFormat; use risingwave_stream::executor::test_utils::prelude::ColumnDesc; use risingwave_stream::executor::{Barrier, Message, MessageStreamItem, StreamExecutorError}; @@ -200,10 +202,12 @@ impl MockDatagenSource { rows_per_second, fields: HashMap::default(), }; - let mut datagen_enumerator = - DatagenSplitEnumerator::new(properties.clone(), Default::default()) - .await - .unwrap(); + let mut datagen_enumerator = DatagenSplitEnumerator::new( + properties.clone(), + SourceEnumeratorContext::dummy().into(), + ) + .await + .unwrap(); let parser_config = ParserConfig { specific: SpecificParserConfig { key_encoding_config: None, @@ -220,7 +224,7 @@ impl MockDatagenSource { properties.clone(), vec![splits], parser_config.clone(), - Default::default(), + SourceContext::dummy().into(), Some(source_schema.clone()), ) .await diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index e25027b61573..49f54c18a5ff 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -173,7 +173,10 @@ async fn test_table_materialize() -> StreamResult<()> { Arc::new(StreamingMetrics::unused()), barrier_rx, system_params_manager.get_params(), - SourceCtrlOpts::default(), + SourceCtrlOpts { + chunk_size: 1024, + rate_limit: None, + }, ) .boxed(), ); diff --git a/src/connector/benches/json_parser.rs b/src/connector/benches/json_parser.rs index ffbcbb149244..90c78044d622 100644 --- a/src/connector/benches/json_parser.rs +++ b/src/connector/benches/json_parser.rs @@ -132,7 +132,7 @@ fn bench_json_parser(c: &mut Criterion) { .build() .unwrap(); let records = generate_json_rows(); - let ctx = Arc::new(SourceContext::default()); + let ctx = Arc::new(SourceContext::dummy()); c.bench_function("json_parser", |b| { b.to_async(&rt).iter_batched( || records.clone(), diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs index e6388ed4b0d2..1c05147eeafb 100644 --- a/src/connector/benches/nexmark_integration.rs +++ b/src/connector/benches/nexmark_integration.rs @@ -26,7 +26,8 @@ use risingwave_connector::parser::{ ByteStreamSourceParser, JsonParser, SourceParserIntoStreamExt, SpecificParserConfig, }; use risingwave_connector::source::{ - BoxChunkSourceStream, BoxSourceStream, SourceColumnDesc, SourceMessage, SourceMeta, + BoxChunkSourceStream, BoxSourceStream, SourceColumnDesc, SourceContext, SourceMessage, + SourceMeta, }; use tracing::Level; use tracing_subscriber::prelude::*; @@ -87,7 +88,7 @@ fn make_parser() -> impl ByteStreamSourceParser { let props = SpecificParserConfig::DEFAULT_PLAIN_JSON; - JsonParser::new(props, columns, Default::default()).unwrap() + JsonParser::new(props, columns, SourceContext::dummy().into()).unwrap() } fn make_stream_iter() -> impl Iterator { diff --git a/src/connector/benches/parser.rs b/src/connector/benches/parser.rs index f0527e119086..21ce72dd1b2b 100644 --- a/src/connector/benches/parser.rs +++ b/src/connector/benches/parser.rs @@ -20,7 +20,7 @@ use risingwave_connector::parser::{ EncodingProperties, JsonParser, JsonProperties, ProtocolProperties, SourceStreamChunkBuilder, SpecificParserConfig, }; -use risingwave_connector::source::SourceColumnDesc; +use risingwave_connector::source::{SourceColumnDesc, SourceContext}; use serde_json::json; use tokio::runtime::Runtime; @@ -81,7 +81,7 @@ fn create_parser( }), protocol_config: ProtocolProperties::Plain, }; - let parser = JsonParser::new(props, desc.clone(), Default::default()).unwrap(); + let parser = JsonParser::new(props, desc.clone(), SourceContext::dummy().into()).unwrap(); let input = gen_input(mode, chunk_size, chunk_num); (parser, desc, input) } diff --git a/src/connector/src/lib.rs b/src/connector/src/lib.rs index 1153a133eb1d..f28f3cdce5e7 100644 --- a/src/connector/src/lib.rs +++ b/src/connector/src/lib.rs @@ -32,6 +32,7 @@ #![feature(iterator_try_collect)] #![feature(try_blocks)] #![feature(error_generic_member_access)] +#![feature(negative_impls)] #![feature(register_tool)] #![register_tool(rw)] #![recursion_limit = "256"] diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 6125e87d069b..2e9a2ef6c18a 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -201,7 +201,7 @@ mod test { use crate::parser::{ AccessBuilderImpl, EncodingType, SourceStreamChunkBuilder, SpecificParserConfig, }; - use crate::source::SourceColumnDesc; + use crate::source::{SourceColumnDesc, SourceContext}; fn test_data_path(file_name: &str) -> String { let curr_dir = env::current_dir().unwrap().into_os_string(); @@ -279,7 +279,7 @@ mod test { EncodingType::Value, )?), rw_columns: Vec::default(), - source_ctx: Default::default(), + source_ctx: SourceContext::dummy().into(), transaction_meta_builder: None, }) } diff --git a/src/connector/src/parser/bytes_parser.rs b/src/connector/src/parser/bytes_parser.rs index 4f353ce2c60e..255c3ef829c7 100644 --- a/src/connector/src/parser/bytes_parser.rs +++ b/src/connector/src/parser/bytes_parser.rs @@ -54,6 +54,7 @@ mod tests { BytesProperties, EncodingProperties, ProtocolProperties, SourceColumnDesc, SourceStreamChunkBuilder, SpecificParserConfig, }; + use crate::source::SourceContext; fn get_payload() -> Vec> { vec![br#"t"#.to_vec(), br#"random"#.to_vec()] @@ -66,7 +67,7 @@ mod tests { encoding_config: EncodingProperties::Bytes(BytesProperties { column_name: None }), protocol_config: ProtocolProperties::Plain, }; - let mut parser = PlainParser::new(props, descs.clone(), Default::default()) + let mut parser = PlainParser::new(props, descs.clone(), SourceContext::dummy().into()) .await .unwrap(); diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index 75e6656fd7a7..9bc236392eb4 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -158,7 +158,7 @@ mod tests { ]; let parser = CanalJsonParser::new( descs.clone(), - Default::default(), + SourceContext::dummy().into(), &JsonProperties::default(), ) .unwrap(); @@ -229,7 +229,7 @@ mod tests { let parser = CanalJsonParser::new( descs.clone(), - Default::default(), + SourceContext::dummy().into(), &JsonProperties::default(), ) .unwrap(); @@ -283,7 +283,7 @@ mod tests { let parser = CanalJsonParser::new( descs.clone(), - Default::default(), + SourceContext::dummy().into(), &JsonProperties::default(), ) .unwrap(); diff --git a/src/connector/src/parser/csv_parser.rs b/src/connector/src/parser/csv_parser.rs index 8a8bb211da32..eaf09ab83989 100644 --- a/src/connector/src/parser/csv_parser.rs +++ b/src/connector/src/parser/csv_parser.rs @@ -192,7 +192,7 @@ mod tests { delimiter: b',', has_header: false, }, - Default::default(), + SourceContext::dummy().into(), ) .unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 4); @@ -299,7 +299,7 @@ mod tests { delimiter: b',', has_header: true, }, - Default::default(), + SourceContext::dummy().into(), ) .unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 4); diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index ca1574af3d6b..29d9139d221c 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -154,7 +154,7 @@ mod tests { use crate::parser::{ DebeziumAvroParserConfig, DebeziumParser, SourceStreamChunkBuilder, SpecificParserConfig, }; - use crate::source::SourceColumnDesc; + use crate::source::{SourceColumnDesc, SourceContext}; const DEBEZIUM_AVRO_DATA: &[u8] = b"\x00\x00\x00\x00\x06\x00\x02\xd2\x0f\x0a\x53\x61\x6c\x6c\x79\x0c\x54\x68\x6f\x6d\x61\x73\x2a\x73\x61\x6c\x6c\x79\x2e\x74\x68\x6f\x6d\x61\x73\x40\x61\x63\x6d\x65\x2e\x63\x6f\x6d\x16\x32\x2e\x31\x2e\x32\x2e\x46\x69\x6e\x61\x6c\x0a\x6d\x79\x73\x71\x6c\x12\x64\x62\x73\x65\x72\x76\x65\x72\x31\xc0\xb4\xe8\xb7\xc9\x61\x00\x30\x66\x69\x72\x73\x74\x5f\x69\x6e\x5f\x64\x61\x74\x61\x5f\x63\x6f\x6c\x6c\x65\x63\x74\x69\x6f\x6e\x12\x69\x6e\x76\x65\x6e\x74\x6f\x72\x79\x00\x02\x12\x63\x75\x73\x74\x6f\x6d\x65\x72\x73\x00\x00\x20\x6d\x79\x73\x71\x6c\x2d\x62\x69\x6e\x2e\x30\x30\x30\x30\x30\x33\x8c\x06\x00\x00\x00\x02\x72\x02\x92\xc3\xe8\xb7\xc9\x61\x00"; @@ -367,9 +367,12 @@ mod tests { .map(CatColumnDesc::from) .map(|c| SourceColumnDesc::from(&c)) .collect_vec(); - let parser = - DebeziumParser::new(parser_config, columns.clone(), Arc::new(Default::default())) - .await?; + let parser = DebeziumParser::new( + parser_config, + columns.clone(), + SourceContext::dummy().into(), + ) + .await?; let [(op, row)]: [_; 1] = parse_one(parser, columns, DEBEZIUM_AVRO_DATA.to_vec()) .await .try_into() diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index 7fb497c399c6..52d1e4e4a15a 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -24,9 +24,8 @@ use crate::parser::unified::debezium::DebeziumChangeEvent; use crate::parser::unified::json::TimestamptzHandling; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; use crate::parser::{ - AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, JsonProperties, - ParseResult, ParserFormat, ProtocolProperties, SourceStreamChunkRowWriter, - SpecificParserConfig, + AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, ParseResult, + ParserFormat, ProtocolProperties, SourceStreamChunkRowWriter, SpecificParserConfig, }; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; @@ -112,6 +111,8 @@ impl DebeziumParser { } pub async fn new_for_test(rw_columns: Vec) -> ConnectorResult { + use crate::parser::JsonProperties; + let props = SpecificParserConfig { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { @@ -120,7 +121,7 @@ impl DebeziumParser { }), protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()), }; - Self::new(props, rw_columns, Default::default()).await + Self::new(props, rw_columns, SourceContext::dummy().into()).await } pub async fn parse_inner( @@ -199,7 +200,7 @@ mod tests { use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; use super::*; - use crate::parser::{SourceStreamChunkBuilder, TransactionControl}; + use crate::parser::{JsonProperties, SourceStreamChunkBuilder, TransactionControl}; use crate::source::{ConnectorProperties, DataType}; #[tokio::test] @@ -228,7 +229,7 @@ mod tests { }; let source_ctx = SourceContext { connector_props: ConnectorProperties::PostgresCdc(Box::default()), - ..Default::default() + ..SourceContext::dummy() }; let mut parser = DebeziumParser::new(props, columns.clone(), Arc::new(source_ctx)) .await diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index ad0bc9e7f147..2de46a9aa546 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -182,7 +182,8 @@ mod tests { SourceColumnDesc::simple("_id", DataType::Varchar, ColumnId::from(0)), SourceColumnDesc::simple("payload", DataType::Jsonb, ColumnId::from(1)), ]; - let mut parser = DebeziumMongoJsonParser::new(columns.clone(), Default::default()).unwrap(); + let mut parser = + DebeziumMongoJsonParser::new(columns.clone(), SourceContext::dummy().into()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 3); let writer = builder.row_writer(); parser @@ -218,7 +219,8 @@ mod tests { ]; for data in input { let mut parser = - DebeziumMongoJsonParser::new(columns.clone(), Default::default()).unwrap(); + DebeziumMongoJsonParser::new(columns.clone(), SourceContext::dummy().into()) + .unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 3); diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index 72c1688d95e7..63cb939d8123 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -115,7 +115,7 @@ mod tests { DebeziumParser, DebeziumProps, EncodingProperties, JsonProperties, ProtocolProperties, SourceColumnDesc, SourceStreamChunkBuilder, SpecificParserConfig, }; - use crate::source::SourceContextRef; + use crate::source::SourceContext; fn assert_json_eq(parse_result: &Option, json_str: &str) { if let Some(ScalarImpl::Jsonb(json_val)) = parse_result { @@ -130,10 +130,7 @@ mod tests { } } - async fn build_parser( - rw_columns: Vec, - source_ctx: SourceContextRef, - ) -> DebeziumParser { + async fn build_parser(rw_columns: Vec) -> DebeziumParser { let props = SpecificParserConfig { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { @@ -142,7 +139,7 @@ mod tests { }), protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()), }; - DebeziumParser::new(props, rw_columns, source_ctx) + DebeziumParser::new(props, rw_columns, SourceContext::dummy().into()) .await .unwrap() } @@ -197,7 +194,7 @@ mod tests { let columns = get_test1_columns(); for data in input { - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(_op, row)]: [_; 1] = parse_one(parser, columns.clone(), data) .await .try_into() @@ -228,7 +225,7 @@ mod tests { let columns = get_test1_columns(); for data in input { - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns.clone(), data) .await .try_into() @@ -259,7 +256,7 @@ mod tests { for data in input { let columns = get_test1_columns(); - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns.clone(), data) .await .try_into() @@ -297,7 +294,7 @@ mod tests { let columns = get_test1_columns(); for data in input { - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns.clone(), data) .await .try_into() @@ -356,7 +353,7 @@ mod tests { let columns = get_test2_columns(); - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(_op, row)]: [_; 1] = parse_one(parser, columns, data.to_vec()) .await @@ -391,7 +388,7 @@ mod tests { let data = br#"{"payload":{"before":null,"after":{"O_KEY":111,"O_BOOL":1,"O_TINY":-1,"O_INT":-1111,"O_REAL":-11.11,"O_DOUBLE":-111.11111,"O_DECIMAL":-111.11,"O_CHAR":"yes please","O_DATE":"1000-01-01","O_TIME":0,"O_DATETIME":0,"O_TIMESTAMP":"1970-01-01T00:00:01Z","O_JSON":"{\"k1\": \"v1\", \"k2\": 11}"},"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678088861000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":789,"row":0,"thread":4,"query":null},"op":"c","ts_ms":1678088861249,"transaction":null}}"#; let columns = get_test2_columns(); - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns, data.to_vec()) .await .try_into() @@ -426,7 +423,7 @@ mod tests { let data = br#"{"payload":{"before":{"O_KEY":111,"O_BOOL":0,"O_TINY":3,"O_INT":3333,"O_REAL":33.33,"O_DOUBLE":333.33333,"O_DECIMAL":333.33,"O_CHAR":"no thanks","O_DATE":"9999-12-31","O_TIME":86399000000,"O_DATETIME":99999999999000,"O_TIMESTAMP":"2038-01-09T03:14:07Z","O_JSON":"{\"k1\":\"v1_updated\",\"k2\":33}"},"after":null,"source":{"version":"1.9.7.Final","connector":"mysql","name":"RW_CDC_test.orders","ts_ms":1678090653000,"snapshot":"false","db":"test","sequence":null,"table":"orders","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1643,"row":0,"thread":4,"query":null},"op":"d","ts_ms":1678090653611,"transaction":null}}"#; let columns = get_test2_columns(); - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns, data.to_vec()) .await .try_into() @@ -463,7 +460,7 @@ mod tests { let columns = get_test2_columns(); - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns, data.to_vec()) .await .try_into() @@ -506,7 +503,7 @@ mod tests { SourceColumnDesc::simple("O_REAL", DataType::Float32, ColumnId::from(4)), SourceColumnDesc::simple("O_DOUBLE", DataType::Float64, ColumnId::from(5)), ]; - let mut parser = build_parser(columns.clone(), Default::default()).await; + let mut parser = build_parser(columns.clone()).await; let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 2); @@ -647,7 +644,7 @@ mod tests { // this test covers an insert event on the table above let data = br#"{"payload":{"before":null,"after":{"o_key":0,"o_time_0":40271000000,"o_time_6":40271000010,"o_timez_0":"11:11:11Z","o_timez_6":"11:11:11.00001Z","o_timestamp_0":1321009871000,"o_timestamp_6":1321009871123456,"o_timestampz_0":"2011-11-11T03:11:11Z","o_timestampz_6":"2011-11-11T03:11:11.123456Z","o_interval":"P1Y2M3DT4H5M6.78S","o_date":"1999-09-09"},"source":{"version":"1.9.7.Final","connector":"postgresql","name":"RW_CDC_localhost.test.orders","ts_ms":1684733351963,"snapshot":"last","db":"test","sequence":"[null,\"26505352\"]","schema":"public","table":"orders","txId":729,"lsn":26505352,"xmin":null},"op":"r","ts_ms":1684733352110,"transaction":null}}"#; let columns = get_temporal_test_columns(); - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns, data.to_vec()) .await .try_into() @@ -702,7 +699,7 @@ mod tests { // this test covers an insert event on the table above let data = br#"{"payload":{"before":null,"after":{"o_key":0,"o_smallint":32767,"o_integer":2147483647,"o_bigint":9223372036854775807,"o_real":9.999,"o_double":9.999999,"o_numeric":123456.789,"o_numeric_6_3":123.456,"o_money":123.12},"source":{"version":"1.9.7.Final","connector":"postgresql","name":"RW_CDC_localhost.test.orders","ts_ms":1684404343201,"snapshot":"last","db":"test","sequence":"[null,\"26519216\"]","schema":"public","table":"orders","txId":729,"lsn":26519216,"xmin":null},"op":"r","ts_ms":1684404343349,"transaction":null}}"#; let columns = get_numeric_test_columns(); - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns, data.to_vec()) .await .try_into() @@ -741,7 +738,7 @@ mod tests { // this test covers an insert event on the table above let data = br#"{"payload":{"before":null,"after":{"o_key":1,"o_boolean":false,"o_bit":true,"o_bytea":"ASNFZ4mrze8=","o_json":"{\"k1\": \"v1\", \"k2\": 11}","o_xml":"","o_uuid":"60f14fe2-f857-404a-b586-3b5375b3259f","o_point":{"x":1.0,"y":2.0,"wkb":"AQEAAAAAAAAAAADwPwAAAAAAAABA","srid":null},"o_enum":"polar","o_char":"h","o_varchar":"ha","o_character":"h","o_character_varying":"hahaha"},"source":{"version":"1.9.7.Final","connector":"postgresql","name":"RW_CDC_localhost.test.orders","ts_ms":1684743927178,"snapshot":"last","db":"test","sequence":"[null,\"26524528\"]","schema":"public","table":"orders","txId":730,"lsn":26524528,"xmin":null},"op":"r","ts_ms":1684743927343,"transaction":null}}"#; let columns = get_other_types_test_columns(); - let parser = build_parser(columns.clone(), Default::default()).await; + let parser = build_parser(columns.clone()).await; let [(op, row)]: [_; 1] = parse_one(parser, columns, data.to_vec()) .await .try_into() diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 05e689f07be7..f010b8e6b7df 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -106,10 +106,11 @@ impl JsonParser { }) } + #[cfg(test)] pub fn new_for_test(rw_columns: Vec) -> ConnectorResult { Ok(Self { rw_columns, - source_ctx: Default::default(), + source_ctx: SourceContext::dummy().into(), payload_start_idx: 0, }) } @@ -218,7 +219,7 @@ mod tests { EncodingProperties, JsonProperties, ProtocolProperties, SourceColumnDesc, SourceStreamChunkBuilder, SpecificParserConfig, }; - use crate::source::SourceColumnType; + use crate::source::{SourceColumnType, SourceContext}; fn get_payload() -> Vec> { vec![ @@ -251,7 +252,7 @@ mod tests { let parser = JsonParser::new( SpecificParserConfig::DEFAULT_PLAIN_JSON, descs.clone(), - Default::default(), + SourceContext::dummy().into(), ) .unwrap(); @@ -361,7 +362,7 @@ mod tests { let parser = JsonParser::new( SpecificParserConfig::DEFAULT_PLAIN_JSON, descs.clone(), - Default::default(), + SourceContext::dummy().into(), ) .unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 3); @@ -432,7 +433,7 @@ mod tests { let parser = JsonParser::new( SpecificParserConfig::DEFAULT_PLAIN_JSON, descs.clone(), - Default::default(), + SourceContext::dummy().into(), ) .unwrap(); let payload = br#" @@ -504,7 +505,7 @@ mod tests { let parser = JsonParser::new( SpecificParserConfig::DEFAULT_PLAIN_JSON, descs.clone(), - Default::default(), + SourceContext::dummy().into(), ) .unwrap(); let payload = br#" @@ -550,7 +551,7 @@ mod tests { let parser = JsonParser::new( SpecificParserConfig::DEFAULT_PLAIN_JSON, descs.clone(), - Default::default(), + SourceContext::dummy().into(), ) .unwrap(); let payload = br#" @@ -614,7 +615,7 @@ mod tests { }), protocol_config: ProtocolProperties::Upsert, }; - let mut parser = UpsertParser::new(props, descs.clone(), Default::default()) + let mut parser = UpsertParser::new(props, descs.clone(), SourceContext::dummy().into()) .await .unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 4); diff --git a/src/connector/src/parser/maxwell/simd_json_parser.rs b/src/connector/src/parser/maxwell/simd_json_parser.rs index d018ef73b050..5db6cdd52e90 100644 --- a/src/connector/src/parser/maxwell/simd_json_parser.rs +++ b/src/connector/src/parser/maxwell/simd_json_parser.rs @@ -23,6 +23,8 @@ mod tests { EncodingProperties, JsonProperties, ProtocolProperties, SourceColumnDesc, SourceStreamChunkBuilder, SpecificParserConfig, }; + use crate::source::SourceContext; + #[tokio::test] async fn test_json_parser() { let descs = vec![ @@ -40,7 +42,7 @@ mod tests { }), protocol_config: ProtocolProperties::Maxwell, }; - let mut parser = MaxwellParser::new(props, descs.clone(), Default::default()) + let mut parser = MaxwellParser::new(props, descs.clone(), SourceContext::dummy().into()) .await .unwrap(); diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index b50f0d57645f..c52636690593 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -195,7 +195,7 @@ mod tests { let source_ctx = SourceContext { connector_props: ConnectorProperties::PostgresCdc(Box::default()), - ..Default::default() + ..SourceContext::dummy() }; let source_ctx = Arc::new(source_ctx); // format plain encode json parser @@ -348,7 +348,7 @@ mod tests { // format plain encode json parser let source_ctx = SourceContext { connector_props: ConnectorProperties::MysqlCdc(Box::default()), - ..Default::default() + ..SourceContext::dummy() }; let mut parser = PlainParser::new( SpecificParserConfig::DEFAULT_PLAIN_JSON, diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index c6598f163a97..8d56441e5348 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -363,7 +363,7 @@ impl Sink for KafkaSink { // use enumerator to validate broker reachability and existence of topic let check = KafkaSplitEnumerator::new( KafkaProperties::from(self.config.clone()), - Arc::new(SourceEnumeratorContext::default()), + Arc::new(SourceEnumeratorContext::dummy()), ) .await?; if !check.check_reachability().await { diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 420194fe4a94..6f594c09366d 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -138,27 +138,33 @@ pub struct SourceCtrlOpts { pub rate_limit: Option, } -impl Default for SourceCtrlOpts { - fn default() -> Self { - Self { - chunk_size: MAX_CHUNK_SIZE, - rate_limit: None, - } - } -} +// The options in `SourceCtrlOpts` are so important that we don't want to impl `Default` for it, +// so that we can prevent any unintentional use of the default value. +impl !Default for SourceCtrlOpts {} -#[derive(Debug, Default)] +#[derive(Debug)] pub struct SourceEnumeratorContext { pub info: SourceEnumeratorInfo, pub metrics: Arc, } -#[derive(Clone, Debug, Default)] +impl SourceEnumeratorContext { + /// Create a dummy `SourceEnumeratorContext` for testing purpose, or for the situation + /// where the real context doesn't matter. + pub fn dummy() -> SourceEnumeratorContext { + SourceEnumeratorContext { + info: SourceEnumeratorInfo { source_id: 0 }, + metrics: Arc::new(EnumeratorMetrics::default()), + } + } +} + +#[derive(Clone, Debug)] pub struct SourceEnumeratorInfo { pub source_id: u32, } -#[derive(Debug, Default)] +#[derive(Debug)] pub struct SourceContext { pub actor_id: u32, pub source_id: TableId, @@ -175,10 +181,10 @@ impl SourceContext { actor_id: u32, source_id: TableId, fragment_id: u32, + source_name: String, metrics: Arc, source_ctrl_opts: SourceCtrlOpts, connector_props: ConnectorProperties, - source_name: String, ) -> Self { Self { actor_id, @@ -190,6 +196,23 @@ impl SourceContext { connector_props, } } + + /// Create a dummy `SourceContext` for testing purpose, or for the situation + /// where the real context doesn't matter. + pub fn dummy() -> Self { + Self::new( + 0, + TableId::new(0), + 0, + "dummy".to_string(), + Arc::new(SourceMetrics::default()), + SourceCtrlOpts { + chunk_size: MAX_CHUNK_SIZE, + rate_limit: None, + }, + ConnectorProperties::default(), + ) + } } #[derive(Clone, Copy, Debug, PartialEq, Eq, Default)] diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index 0836fef581a3..87f798d59f38 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -337,6 +337,7 @@ mod tests { use super::*; use crate::parser::SpecificParserConfig; + use crate::source::SourceContext; #[tokio::test] async fn test_generator() -> Result<()> { @@ -403,7 +404,7 @@ mod tests { }, ..Default::default() }, - Default::default(), + SourceContext::dummy().into(), Some(mock_datum), ) .await? @@ -465,7 +466,7 @@ mod tests { properties.clone(), state, parser_config.clone(), - Default::default(), + SourceContext::dummy().into(), Some(mock_datum.clone()), ) .await? @@ -482,7 +483,7 @@ mod tests { properties, state, parser_config, - Default::default(), + SourceContext::dummy().into(), Some(mock_datum), ) .await? diff --git a/src/connector/src/source/filesystem/s3/enumerator.rs b/src/connector/src/source/filesystem/s3/enumerator.rs index ae671ad2265f..7491cac0df7c 100644 --- a/src/connector/src/source/filesystem/s3/enumerator.rs +++ b/src/connector/src/source/filesystem/s3/enumerator.rs @@ -140,7 +140,7 @@ mod tests { endpoint_url: None, }; let mut enumerator = - S3SplitEnumerator::new(props.into(), SourceEnumeratorContext::default().into()) + S3SplitEnumerator::new(props.into(), SourceEnumeratorContext::dummy().into()) .await .unwrap(); let splits = enumerator.list_splits().await.unwrap(); diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index ff5dc9c9c148..129b708a6152 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -266,7 +266,9 @@ mod tests { }; use crate::source::filesystem::s3::S3PropertiesCommon; use crate::source::filesystem::S3SplitEnumerator; - use crate::source::{SourceColumnDesc, SourceEnumeratorContext, SplitEnumerator}; + use crate::source::{ + SourceColumnDesc, SourceContext, SourceEnumeratorContext, SplitEnumerator, + }; #[tokio::test] #[ignore] @@ -281,7 +283,7 @@ mod tests { } .into(); let mut enumerator = - S3SplitEnumerator::new(props.clone(), SourceEnumeratorContext::default().into()) + S3SplitEnumerator::new(props.clone(), SourceEnumeratorContext::dummy().into()) .await .unwrap(); let splits = enumerator.list_splits().await.unwrap(); @@ -307,7 +309,7 @@ mod tests { }, }; - let reader = S3FileReader::new(props, splits, config, Default::default(), None) + let reader = S3FileReader::new(props, splits, config, SourceContext::dummy().into(), None) .await .unwrap(); diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 959940bd8441..c9026428d1df 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -307,6 +307,7 @@ mod tests { use super::*; use crate::connector_common::KinesisCommon; use crate::source::kinesis::split::KinesisSplit; + use crate::source::SourceContext; #[tokio::test] async fn test_reject_redundant_seq_props() { @@ -335,7 +336,7 @@ mod tests { end_position: KinesisOffset::None, }], Default::default(), - Default::default(), + SourceContext::dummy().into(), None, ) .await; @@ -371,7 +372,7 @@ mod tests { end_position: KinesisOffset::None, }], Default::default(), - Default::default(), + SourceContext::dummy().into(), None, ) .await? @@ -389,7 +390,7 @@ mod tests { end_position: KinesisOffset::None, }], Default::default(), - Default::default(), + SourceContext::dummy().into(), None, ) .await? diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index 7b402cbc1368..6441baa154ae 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -211,7 +211,7 @@ impl NexmarkSplitReader { mod tests { use super::*; use crate::source::nexmark::{NexmarkProperties, NexmarkSplitEnumerator}; - use crate::source::{SourceEnumeratorContext, SplitEnumerator}; + use crate::source::{SourceContext, SourceEnumeratorContext, SplitEnumerator}; #[tokio::test] async fn test_nexmark_split_reader() -> crate::error::ConnectorResult<()> { @@ -224,7 +224,7 @@ mod tests { }; let mut enumerator = - NexmarkSplitEnumerator::new(props.clone(), SourceEnumeratorContext::default().into()) + NexmarkSplitEnumerator::new(props.clone(), SourceEnumeratorContext::dummy().into()) .await?; let list_splits_resp: Vec<_> = enumerator.list_splits().await?.into_iter().collect(); @@ -236,7 +236,7 @@ mod tests { props.clone(), state, Default::default(), - Default::default(), + SourceContext::dummy().into(), None, ) .await? @@ -261,7 +261,7 @@ mod tests { }; let mut enumerator = - NexmarkSplitEnumerator::new(props.clone(), SourceEnumeratorContext::default().into()) + NexmarkSplitEnumerator::new(props.clone(), SourceEnumeratorContext::dummy().into()) .await?; let list_splits_resp: Vec<_> = enumerator.list_splits().await?.into_iter().collect(); @@ -271,7 +271,7 @@ mod tests { props.clone(), state, Default::default(), - Default::default(), + SourceContext::dummy().into(), None, ) .await? diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index a578fd30d253..9fc9316d0281 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -294,7 +294,7 @@ impl SourceScanInfo { match fetch_info.connector { ConnectorProperties::Kafka(prop) => { let mut kafka_enumerator = - KafkaSplitEnumerator::new(*prop, SourceEnumeratorContext::default().into()) + KafkaSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into()) .await?; let split_info = kafka_enumerator .list_splits_batch(fetch_info.timebound.0, fetch_info.timebound.1) @@ -329,7 +329,7 @@ impl SourceScanInfo { } ConnectorProperties::Iceberg(prop) => { let iceberg_enumerator = - IcebergSplitEnumerator::new(*prop, SourceEnumeratorContext::default().into()) + IcebergSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into()) .await?; let time_travel_info = match fetch_info.as_of { diff --git a/src/meta/service/src/cloud_service.rs b/src/meta/service/src/cloud_service.rs index 9c213bd7cb9e..b77b751b281e 100644 --- a/src/meta/service/src/cloud_service.rs +++ b/src/meta/service/src/cloud_service.rs @@ -159,7 +159,7 @@ impl CloudService for CloudServiceImpl { async fn new_enumerator( props: P, ) -> ConnectorResult { - P::SplitEnumerator::new(props, SourceEnumeratorContext::default().into()).await + P::SplitEnumerator::new(props, SourceEnumeratorContext::dummy().into()).await } dispatch_source_prop!(props.unwrap(), props, { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 1c551f46b6fc..c0aa62750fac 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -845,7 +845,7 @@ impl DdlController { async fn new_enumerator_for_validate( source_props: P, ) -> Result { - P::SplitEnumerator::new(source_props, SourceEnumeratorContext::default().into()).await + P::SplitEnumerator::new(source_props, SourceEnumeratorContext::dummy().into()).await } for actor in &stream_scan_fragment.actors { diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 234eceea9fc9..96a53ace0b25 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -626,7 +626,7 @@ pub async fn transform_upstream(upstream: BoxedMessageStream, schema: &Schema) { let mut parser = DebeziumParser::new( props, get_rw_columns(schema), - Arc::new(SourceContext::default()), + Arc::new(SourceContext::dummy()), ) .await .map_err(StreamExecutorError::connector_error)?; diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 1d6e409c63a2..c555fb036d94 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -172,10 +172,10 @@ impl FsFetchExecutor { self.actor_ctx.id, source_id, self.actor_ctx.fragment_id, + source_name.to_owned(), source_desc.metrics.clone(), self.source_ctrl_opts.clone(), source_desc.source.config.clone(), - source_name.to_owned(), ) } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 13b876c05c2b..36b778395023 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -100,10 +100,10 @@ impl FsSourceExecutor { self.actor_ctx.id, self.stream_source_core.source_id, self.actor_ctx.fragment_id, + self.stream_source_core.source_name.clone(), source_desc.metrics.clone(), self.source_ctrl_opts.clone(), source_desc.source.config.clone(), - self.stream_source_core.source_name.clone(), ); let stream = source_desc .source diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index be752b4598a3..7b3c594cab5a 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -210,10 +210,10 @@ impl SourceBackfillExecutorInner { self.actor_ctx.id, self.stream_source_core.source_id, self.actor_ctx.fragment_id, + self.stream_source_core.source_name.clone(), source_desc.metrics.clone(), self.source_ctrl_opts.clone(), source_desc.source.config.clone(), - self.stream_source_core.source_name.clone(), ); let stream = source_desc .source diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index d86a85ec5bde..cf54166afd39 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -160,14 +160,14 @@ impl SourceExecutor { self.actor_ctx.id, self.stream_source_core.as_ref().unwrap().source_id, self.actor_ctx.fragment_id, - source_desc.metrics.clone(), - self.source_ctrl_opts.clone(), - source_desc.source.config.clone(), self.stream_source_core .as_ref() .unwrap() .source_name .clone(), + source_desc.metrics.clone(), + self.source_ctrl_opts.clone(), + source_desc.source.config.clone(), ); let stream = source_desc .source @@ -839,7 +839,10 @@ mod tests { Arc::new(StreamingMetrics::unused()), barrier_rx, system_params_manager.get_params(), - SourceCtrlOpts::default(), + SourceCtrlOpts { + chunk_size: 1024, + rate_limit: None, + }, ); let mut executor = executor.boxed().execute(); @@ -927,7 +930,10 @@ mod tests { Arc::new(StreamingMetrics::unused()), barrier_rx, system_params_manager.get_params(), - SourceCtrlOpts::default(), + SourceCtrlOpts { + chunk_size: 1024, + rate_limit: None, + }, ); let mut handler = executor.boxed().execute(); diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index ca057ced81f6..3610c8a95caf 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -269,7 +269,10 @@ impl ExecutorBuilder for SourceExecutorBuilder { barrier_receiver, system_params, // we don't expect any data in, so no need to set chunk_sizes - SourceCtrlOpts::default(), + SourceCtrlOpts { + chunk_size: 0, + rate_limit: None, + }, ); Ok((params.info, exec).into()) } From b73643ac71767913eb5c5ab73989db437192ca50 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 18 Apr 2024 18:24:17 +0800 Subject: [PATCH 62/64] feat: deprecate legacy s3 connector (#16337) --- ci/workflows/main-cron.yml | 88 ---------- e2e_test/s3/json_file.py | 122 -------------- e2e_test/s3/run.py | 128 --------------- e2e_test/s3/run_csv.py | 154 ------------------ .../kinesis-s3-source/create_source.sql | 2 +- src/frontend/src/error.rs | 2 + src/frontend/src/handler/create_source.rs | 9 +- 7 files changed, 11 insertions(+), 494 deletions(-) delete mode 100644 e2e_test/s3/json_file.py delete mode 100644 e2e_test/s3/run.py delete mode 100644 e2e_test/s3/run_csv.py diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 7cb23f76ce25..bf09d5fe6634 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -472,72 +472,6 @@ steps: timeout_in_minutes: 10 retry: *auto-retry - - label: "S3 source check on AWS" - key: "s3-source-check-aws" - command: "ci/scripts/s3-source-test.sh -p ci-release -s run.py" - if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-s3-source-tests" - || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ - depends_on: build - plugins: - - seek-oss/aws-sm#v2.3.1: - env: - S3_SOURCE_TEST_CONF: ci_s3_source_test_aws - - docker-compose#v5.1.0: - run: rw-build-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - environment: - - S3_SOURCE_TEST_CONF - - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 20 - retry: *auto-retry - - - label: "S3 source check on AWS (json parser)" - key: "s3-source-check-aws-json-parser" - command: "ci/scripts/s3-source-test.sh -p ci-release -s json_file.py" - if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-s3-source-tests" - || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ - depends_on: build - plugins: - - seek-oss/aws-sm#v2.3.1: - env: - S3_SOURCE_TEST_CONF: ci_s3_source_test_aws - - docker-compose#v5.1.0: - run: rw-build-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - environment: - - S3_SOURCE_TEST_CONF - - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 25 - retry: *auto-retry - - - label: "S3 source check on AWS (csv parser)" - key: "s3-source-check-aws-csv-parser" - command: "ci/scripts/s3-source-test.sh -p ci-release -s run_csv.py" - if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-s3-source-tests" - || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ - depends_on: build - plugins: - - seek-oss/aws-sm#v2.3.1: - env: - S3_SOURCE_TEST_CONF: ci_s3_source_test_aws - - docker-compose#v5.1.0: - run: rw-build-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - environment: - - S3_SOURCE_TEST_CONF - - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 25 - retry: *auto-retry - - label: "S3_v2 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" @@ -621,28 +555,6 @@ steps: timeout_in_minutes: 25 retry: *auto-retry - - label: "S3 source on OpenDAL fs engine" - key: "s3-source-test-for-opendal-fs-engine" - command: "ci/scripts/s3-source-test-for-opendal-fs-engine.sh -p ci-release -s run.py" - if: | - !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-s3-source-tests" - || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ - depends_on: build - plugins: - - seek-oss/aws-sm#v2.3.1: - env: - S3_SOURCE_TEST_CONF: ci_s3_source_test_aws - - docker-compose#v5.1.0: - run: rw-build-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - environment: - - S3_SOURCE_TEST_CONF - - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 20 - retry: *auto-retry - # TODO(Kexiang): Enable this test after we have a GCS_SOURCE_TEST_CONF. # - label: "GCS source on OpenDAL fs engine" # key: "s3-source-test-for-opendal-fs-engine" diff --git a/e2e_test/s3/json_file.py b/e2e_test/s3/json_file.py deleted file mode 100644 index 76ec86fae0dc..000000000000 --- a/e2e_test/s3/json_file.py +++ /dev/null @@ -1,122 +0,0 @@ -import os -import string -import json -import string -from time import sleep -from minio import Minio -import psycopg2 -import random - - -def do_test(client, config, N, prefix): - conn = psycopg2.connect( - host="localhost", - port="4566", - user="root", - database="dev" - ) - - # Open a cursor to execute SQL statements - cur = conn.cursor() - - # Execute a SELECT statement - cur.execute(f'''CREATE TABLE s3_test_jsonfile( - id int, - name TEXT, - sex int, - mark int, - ) - include file as file_col - WITH ( - connector = 's3', - match_pattern = '{prefix}*.json', - s3.region_name = '{config['S3_REGION']}', - s3.bucket_name = '{config['S3_BUCKET']}', - s3.credentials.access = '{config['S3_ACCESS_KEY']}', - s3.credentials.secret = '{config['S3_SECRET_KEY']}', - s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' - ) FORMAT PLAIN ENCODE JSON;''') - - for i in range(N): - try: - client.fput_object( - config["S3_BUCKET"], - f"{run_id}_data_{i}.json", - f"data_{i}.json" - ) - print(f"Uploaded {run_id}_data_{i}.json to S3") - os.remove(f"data_{i}.json") - except Exception as e: - print(f"Error uploading data_{i}.json: {e}") - - total_row = int(N) - while True: - sleep(60) - cur.execute('select count(*) from s3_test_jsonfile') - result = cur.fetchone() - if result[0] == total_row: - break - print( - f"Now got {result[0]} rows in table, {total_row} expected, wait 60s") - - cur.execute( - 'select count(*), sum(id), sum(sex), sum(mark) from s3_test_jsonfile') - result = cur.fetchone() - print(result) - - assert result[0] == total_row - assert result[1] == int(((N - 1) * N / 2)) - assert result[2] == int(N / 2) - assert result[3] == 0 - - cur.execute('select file_col from s3_test_jsonfile') - result = cur.fetchone() - file_col = result[0] - print(file_col) - - cur.execute('drop table s3_test_jsonfile') - - cur.close() - conn.close() - - -if __name__ == "__main__": - config = json.loads(os.environ["S3_SOURCE_TEST_CONF"]) - run_id = str(random.randint(1000, 9999)) - N = 1080 - - for i in range(N): - with open(f"data_{i}.json", "w") as f: - f.write(json.dumps( - { - "id": i, - "name": str(i), - "sex": i % 2, - "mark": -1 if i % 2 else 1, - } - )) - os.fsync(f.fileno()) - - client = Minio( - config["S3_ENDPOINT"], - access_key=config["S3_ACCESS_KEY"], - secret_key=config["S3_SECRET_KEY"], - secure=True - ) - return_code = 0 - try: - do_test(client, config, N, run_id) - except Exception as e: - print("Test failed", e) - return_code = 1 - - # Clean up - for i in range(N): - try: - client.remove_object( - config["S3_BUCKET"], f"{run_id}_data_{i}.json") - print(f"Removed {run_id}_data_{i}.json from S3") - except Exception as e: - print(f"Error removing data_{i}.json: {e}") - - exit(return_code) diff --git a/e2e_test/s3/run.py b/e2e_test/s3/run.py deleted file mode 100644 index 58e3c5765c0e..000000000000 --- a/e2e_test/s3/run.py +++ /dev/null @@ -1,128 +0,0 @@ -import os -import string -import json -import string -from time import sleep -from minio import Minio -import psycopg2 -import random - -def do_test(config, N, n, prefix): - conn = psycopg2.connect( - host="localhost", - port="4566", - user="root", - database="dev" - ) - - # Open a cursor to execute SQL statements - cur = conn.cursor() - - # Execute a SELECT statement - cur.execute(f'''CREATE TABLE s3_test( - id int, - name TEXT, - sex int, - mark int, - ) WITH ( - connector = 's3', - match_pattern = '{prefix}*.ndjson', - s3.region_name = '{config['S3_REGION']}', - s3.bucket_name = '{config['S3_BUCKET']}', - s3.credentials.access = '{config['S3_ACCESS_KEY']}', - s3.credentials.secret = '{config['S3_SECRET_KEY']}', - s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' - ) FORMAT PLAIN ENCODE JSON;''') - - total_row = int(N * n) - sleep(120) - while True: - sleep(60) - cur.execute('select count(*) from s3_test') - result = cur.fetchone() - if result[0] == total_row: - break - print( - f"Now got {result[0]} rows in table, {total_row} expected, wait 60s") - - cur.execute('select count(*), sum(id), sum(sex), sum(mark) from s3_test') - result = cur.fetchone() - - print(result) - - assert result[0] == total_row - assert result[1] == int(((N - 1) * N / 2) * n) - assert result[2] == int(N*n / 2) - assert result[3] == 0 - - cur.execute('drop table s3_test') - - cur.close() - conn.close() - - -if __name__ == "__main__": - config = json.loads(os.environ["S3_SOURCE_TEST_CONF"]) - run_id = str(random.randint(1000, 9999)) - N = 10000 - - items = [ - { - "id": j, - "name": str(j), - "sex": j % 2, - "mark": -1 if j % 2 else 1, - } - for j in range(N) - ] - - data = "\n".join([json.dumps(item) for item in items]) + "\n" - n = 0 - with open("data_0.ndjson", "w") as f: - for _ in range(1000): - n += 1 - f.write(data) - os.fsync(f.fileno()) - - for i in range(1, 20): - with open(f"data_{i}.ndjson", "w") as f: - n += 1 - f.write(data) - os.fsync(f.fileno()) - - client = Minio( - config["S3_ENDPOINT"], - access_key=config["S3_ACCESS_KEY"], - secret_key=config["S3_SECRET_KEY"], - secure=True - ) - - for i in range(20): - try: - client.fput_object( - config["S3_BUCKET"], - f"{run_id}_data_{i}.ndjson", - f"data_{i}.ndjson" - - ) - print(f"Uploaded {run_id}_data_{i}.ndjson to S3") - os.remove(f"data_{i}.ndjson") - except Exception as e: - print(f"Error uploading data_{i}.ndjson: {e}") - - return_code = 0 - try: - do_test(config, N, n, run_id) - except Exception as e: - print("Test failed", e) - return_code = 1 - - # Clean up - for i in range(20): - try: - client.remove_object(config["S3_BUCKET"], f"{run_id}_data_{i}.ndjson") - print(f"Removed {run_id}_data_{i}.ndjson from S3") - except Exception as e: - print(f"Error removing data_{i}.ndjson: {e}") - - exit(return_code) diff --git a/e2e_test/s3/run_csv.py b/e2e_test/s3/run_csv.py deleted file mode 100644 index a6c0dc37bc4c..000000000000 --- a/e2e_test/s3/run_csv.py +++ /dev/null @@ -1,154 +0,0 @@ -import os -import json -import string -from time import sleep -from minio import Minio -import psycopg2 -import random - - -def do_test(config, N, n, prefix): - conn = psycopg2.connect( - host="localhost", - port="4566", - user="root", - database="dev" - ) - - # Open a cursor to execute SQL statements - cur = conn.cursor() - cur.execute(f'''CREATE TABLE s3_test_csv_without_headers( - a int, - b int, - c int, - ) WITH ( - connector = 's3', - match_pattern = '{prefix}_data_without_headers.csv', - s3.region_name = '{config['S3_REGION']}', - s3.bucket_name = '{config['S3_BUCKET']}', - s3.credentials.access = '{config['S3_ACCESS_KEY']}', - s3.credentials.secret = '{config['S3_SECRET_KEY']}', - s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' - ) FORMAT PLAIN ENCODE CSV (delimiter = ',', without_header = true);''') - - cur.execute(f'''CREATE TABLE s3_test_csv_with_headers( - a int, - b int, - c int, - ) WITH ( - connector = 's3', - match_pattern = '{prefix}_data_with_headers.csv', - s3.region_name = '{config['S3_REGION']}', - s3.bucket_name = '{config['S3_BUCKET']}', - s3.credentials.access = '{config['S3_ACCESS_KEY']}', - s3.credentials.secret = '{config['S3_SECRET_KEY']}', - s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' - ) FORMAT PLAIN ENCODE CSV (delimiter = ',');''') - - total_row = int(N * n) - sleep(60) - while True: - sleep(60) - cur.execute('select count(*) from s3_test_csv_with_headers') - result_with_headers = cur.fetchone() - cur.execute('select count(*) from s3_test_csv_without_headers') - result_without_headers = cur.fetchone() - if result_with_headers[0] == total_row and result_without_headers[0] == total_row: - break - print( - f"Now got {result_with_headers[0]} rows in table, {total_row} expected, wait 60s") - - cur.execute( - 'select count(*), sum(a), sum(b), sum(c) from s3_test_csv_with_headers') - result_with_headers = cur.fetchone() - - cur.execute( - 'select count(*), sum(a), sum(b), sum(c) from s3_test_csv_without_headers') - s3_test_csv_without_headers = cur.fetchone() - - print(result_with_headers, s3_test_csv_without_headers, - int(((N - 1) * N / 2) * n), int(N*n / 2)) - - assert s3_test_csv_without_headers[0] == total_row - assert s3_test_csv_without_headers[1] == int(((N - 1) * N / 2) * n) - assert s3_test_csv_without_headers[2] == int(N*n / 2) - assert s3_test_csv_without_headers[3] == 0 - - assert result_with_headers[0] == total_row - assert result_with_headers[1] == 0 - assert result_with_headers[2] == int(N*n / 2) - assert result_with_headers[3] == int(((N - 1) * N / 2) * n) - - cur.execute('drop table s3_test_csv_with_headers') - cur.execute('drop table s3_test_csv_without_headers') - - cur.close() - conn.close() - - -if __name__ == "__main__": - config = json.loads(os.environ["S3_SOURCE_TEST_CONF"]) - run_id = str(random.randint(1000, 9999)) - N = 10000 - # do_test(config, N, 0, run_id) - items = [",".join([str(j), str(j % 2), str(-1 if j % 2 else 1)]) - for j in range(N) - ] - - data = "\n".join(items) + "\n" - n = 10 - with open("data_without_headers.csv", "w") as f: - for _ in range(10): - f.write(data) - os.fsync(f.fileno()) - - with open("data_with_headers.csv", "w") as f: - f.write("c,b,a\n") - for _ in range(10): - f.write(data) - os.fsync(f.fileno()) - - client = Minio( - config["S3_ENDPOINT"], - access_key=config["S3_ACCESS_KEY"], - secret_key=config["S3_SECRET_KEY"], - secure=True - ) - - try: - client.fput_object( - config["S3_BUCKET"], - f"{run_id}_data_without_headers.csv", - f"data_without_headers.csv" - - ) - client.fput_object( - config["S3_BUCKET"], - f"{run_id}_data_with_headers.csv", - f"data_with_headers.csv" - ) - print( - f"Uploaded {run_id}_data_with_headers.csv & {run_id}_data_with_headers.csv to S3") - os.remove(f"data_with_headers.csv") - os.remove(f"data_without_headers.csv") - except Exception as e: - print(f"Error uploading test files") - - return_code = 0 - try: - do_test(config, N, n, run_id) - except Exception as e: - print("Test failed", e) - return_code = 1 - - # Clean up - for i in range(20): - try: - client.remove_object( - config["S3_BUCKET"], f"{run_id}_data_with_headers.csv") - client.remove_object( - config["S3_BUCKET"], f"{run_id}_data_without_headers.csv") - except Exception as e: - print(f"Error removing testing files {e}") - - exit(return_code) diff --git a/integration_tests/kinesis-s3-source/create_source.sql b/integration_tests/kinesis-s3-source/create_source.sql index 902eccdf1950..c4650f22d822 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', + connector = 's3_v2', s3.region_name = 'us-east-1', s3.bucket_name = 'ad-click', s3.credentials.access = 'test', diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index 27a6418a7b0c..e2ec1b35fe50 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -164,6 +164,8 @@ pub enum ErrorCode { #[backtrace] SessionConfigError, ), + #[error("{0} has been deprecated, please use {1} instead.")] + Deprecated(String, String), } /// The result type for the frontend crate. diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 364c7aafed12..0830cdb5392d 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -68,7 +68,7 @@ use thiserror_ext::AsReport; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::source_catalog::SourceCatalog; -use crate::error::ErrorCode::{self, InvalidInputSyntax, NotSupported, ProtocolError}; +use crate::error::ErrorCode::{self, Deprecated, InvalidInputSyntax, NotSupported, ProtocolError}; use crate::error::{Result, RwError}; use crate::expr::Expr; use crate::handler::create_table::{ @@ -1077,6 +1077,13 @@ 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 0c464ab2a1d513955c1cb2aaa3a38388529b53f8 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 18 Apr 2024 19:28:52 +0800 Subject: [PATCH 63/64] fix(streaming): really find the root cause of recovery (#16376) --- src/stream/src/task/barrier_manager.rs | 33 ++++++++++++++++---------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 0f3a90ad81f4..9d006a426d6b 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -62,7 +62,7 @@ use risingwave_storage::store::SyncResult; use crate::executor::exchange::permit::Receiver; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{Actor, Barrier, DispatchExecutor, Mutation}; +use crate::executor::{Actor, Barrier, DispatchExecutor, Mutation, StreamExecutorError}; use crate::task::barrier_manager::progress::BackfillState; /// If enabled, all actors will be grouped in the same tracing span within one epoch. @@ -694,7 +694,8 @@ impl LocalBarrierWorker { /// will notice actor's exit while collecting. async fn notify_failure(&mut self, actor_id: ActorId, err: StreamError) { self.add_failure(actor_id, err.clone()); - let root_err = self.try_find_root_failure(err).await; + let root_err = self.try_find_root_failure().await; + let failed_epochs = self.state.epochs_await_on_actor(actor_id).collect_vec(); if !failed_epochs.is_empty() { self.control_stream_handle.reset_stream_with_err( @@ -709,7 +710,6 @@ impl LocalBarrierWorker { } fn add_failure(&mut self, actor_id: ActorId, err: StreamError) { - let err = err.into_unexpected_exit(actor_id); if let Some(prev_err) = self.failure_actors.insert(actor_id, err) { warn!( actor_id, @@ -719,7 +719,7 @@ impl LocalBarrierWorker { } } - async fn try_find_root_failure(&mut self, default_err: StreamError) -> StreamError { + async fn try_find_root_failure(&mut self) -> StreamError { if let Some(root_failure) = &self.root_failure { return root_failure.clone(); } @@ -731,7 +731,10 @@ impl LocalBarrierWorker { }) .await; self.root_failure = try_find_root_actor_failure(self.failure_actors.values()); - self.root_failure.clone().unwrap_or(default_err) + + self.root_failure + .clone() + .expect("failure actors should not be empty") } } @@ -817,7 +820,9 @@ impl LocalBarrierManager { /// When a actor exit unexpectedly, it should report this event using this function, so meta /// will notice actor's exit while collecting. pub fn notify_failure(&self, actor_id: ActorId, err: StreamError) { - let _ = self.actor_failure_sender.send((actor_id, err)); + let _ = self + .actor_failure_sender + .send((actor_id, err.into_unexpected_exit(actor_id))); } /// When a `RemoteInput` get a barrier, it should wait and read the barrier mutation from the barrier manager. @@ -839,26 +844,30 @@ impl LocalBarrierManager { pub fn try_find_root_actor_failure<'a>( actor_errors: impl IntoIterator, ) -> Option { - use crate::executor::StreamExecutorError; - let stream_executor_error_score = |e: &StreamExecutorError| { + fn stream_executor_error_score(e: &StreamExecutorError) -> i32 { use crate::executor::error::ErrorKind; match e.inner() { ErrorKind::ChannelClosed(_) => 0, ErrorKind::Internal(_) => 1, _ => 999, } - }; - let stream_error_score = |e: &&StreamError| { + } + + fn stream_error_score(e: &StreamError) -> i32 { use crate::error::ErrorKind; match e.inner() { + // `UnexpectedExit` wraps the original error. Score on the inner error. + ErrorKind::UnexpectedExit { source, .. } => stream_error_score(source), + ErrorKind::Internal(_) => 1000, ErrorKind::Executor(ee) => 2000 + stream_executor_error_score(ee), _ => 3000, } - }; + } + actor_errors .into_iter() - .max_by_key(stream_error_score) + .max_by_key(|&e| stream_error_score(e)) .cloned() } From 3daa16077321a486d762b055da96d43ae0327ac4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Giovanny=20Guti=C3=A9rrez?= Date: Thu, 18 Apr 2024 07:04:06 -0500 Subject: [PATCH 64/64] feat(udf): Add deno as UDF language (#16263) Co-authored-by: Runji Wang --- Cargo.lock | 1940 +++++++++++++++-- Cargo.toml | 10 + Makefile.toml | 6 + ci/scripts/build.sh | 1 + ci/scripts/run-e2e-test.sh | 2 + docker/Dockerfile | 2 +- docker/Dockerfile.hdfs | 2 +- e2e_test/udf/deno_udf.slt | 233 ++ e2e_test/udf/mock_server.py | 43 + e2e_test/udf/requirements.txt | 2 + e2e_test/udf/sse/bundled.table.js | 186 ++ proto/catalog.proto | 4 + proto/expr.proto | 6 + src/cmd_all/Cargo.toml | 1 + src/connector/src/parser/avro/parser.rs | 2 +- src/connector/src/with_options_test.rs | 6 +- src/expr/core/Cargo.toml | 3 + src/expr/core/src/expr/expr_udf.rs | 66 +- .../core/src/table_function/user_defined.rs | 58 +- src/frontend/src/catalog/function_catalog.rs | 4 + src/frontend/src/expr/table_function.rs | 2 + .../src/expr/user_defined_function.rs | 4 + src/frontend/src/handler/create_function.rs | 74 +- .../src/handler/create_sql_function.rs | 2 + src/meta/model_v2/migration/src/lib.rs | 2 + .../src/m20240418_142249_function_runtime.rs | 61 + src/meta/model_v2/src/function.rs | 4 + src/meta/src/controller/mod.rs | 2 + src/sqlparser/src/ast/mod.rs | 77 + src/sqlparser/src/keywords.rs | 3 + src/sqlparser/src/parser.rs | 42 + src/sqlparser/tests/sqlparser_postgres.rs | 25 + 32 files changed, 2659 insertions(+), 216 deletions(-) create mode 100644 e2e_test/udf/deno_udf.slt create mode 100644 e2e_test/udf/mock_server.py create mode 100644 e2e_test/udf/requirements.txt create mode 100644 e2e_test/udf/sse/bundled.table.js create mode 100644 src/meta/model_v2/migration/src/m20240418_142249_function_runtime.rs diff --git a/Cargo.lock b/Cargo.lock index 1f6f31de2cf7..edae74d715f7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2,6 +2,16 @@ # It is not intended for manual editing. version = 3 +[[package]] +name = "Inflector" +version = "0.11.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe438c63458706e03479442743baae6c88256498e6431708f6dfc520a26515d3" +dependencies = [ + "lazy_static", + "regex", +] + [[package]] name = "addr2line" version = "0.21.0" @@ -23,17 +33,50 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" +[[package]] +name = "aead" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d122413f284cf2d62fb1b7db97e02edb8cda96d769b16e443a4f6195e35662b0" +dependencies = [ + "crypto-common", + "generic-array", +] + [[package]] name = "aes" -version = "0.8.4" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b169f7a6d4742236a0a00c541b845991d0ac43e546831af1249753ab4c3aa3a0" +checksum = "ac1f845298e95f983ff1944b728ae08b8cebab80d684f0a832ed0fc74dfa27e2" dependencies = [ "cfg-if", "cipher", "cpufeatures", ] +[[package]] +name = "aes-gcm" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "831010a0f742e1209b3bcea8fab6a8e149051ba6099432c8cb2cc117dec3ead1" +dependencies = [ + "aead", + "aes", + "cipher", + "ctr", + "ghash", + "subtle", +] + +[[package]] +name = "aes-kw" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69fa2b352dcefb5f7f3a5fb840e02665d311d878955380515e4fd50095dd3d8c" +dependencies = [ + "aes", +] + [[package]] name = "ahash" version = "0.7.7" @@ -651,6 +694,56 @@ dependencies = [ "rquickjs", ] +[[package]] +name = "arrow-udf-js-deno" +version = "0.0.1" +source = "git+https://github.com/risingwavelabs/arrow-udf.git?rev=23fe0dd#23fe0dd41616f4646f9139e22a335518e6cc9a47" +dependencies = [ + "anyhow", + "arrow-array 50.0.0", + "arrow-buffer 50.0.0", + "arrow-data 50.0.0", + "arrow-schema 50.0.0", + "arrow-udf-js-deno-runtime", + "async-trait", + "deno_core", + "futures", + "futures-util", + "libc", + "serde", + "serde_json", + "tokio", + "v8", +] + +[[package]] +name = "arrow-udf-js-deno-runtime" +version = "0.0.1" +source = "git+https://github.com/risingwavelabs/arrow-udf.git?rev=23fe0dd#23fe0dd41616f4646f9139e22a335518e6cc9a47" +dependencies = [ + "anyhow", + "deno_ast", + "deno_console", + "deno_core", + "deno_crypto", + "deno_fetch", + "deno_http", + "deno_io", + "deno_net", + "deno_tls", + "deno_url", + "deno_web", + "deno_webidl", + "deno_websocket", + "hyper 0.14.27", + "libc", + "serde", + "serde_json", + "signal-hook-registry", + "tokio", + "v8", +] + [[package]] name = "arrow-udf-python" version = "0.1.0" @@ -691,6 +784,18 @@ version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9b34d609dfbaf33d6889b2b7106d3ca345eacad44200913df5ba02bfd31d2ba9" +[[package]] +name = "ast_node" +version = "0.9.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3e3e06ec6ac7d893a0db7127d91063ad7d9da8988f8a1a256f03729e6eec026" +dependencies = [ + "proc-macro2", + "quote", + "swc_macros_common", + "syn 2.0.57", +] + [[package]] name = "async-attributes" version = "1.1.2" @@ -718,6 +823,7 @@ version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bc2d0cfb2a7388d34f590e76686704c494ed7aaceed62ee1ba35cbf363abc2a5" dependencies = [ + "brotli", "bzip2", "flate2", "futures-core", @@ -1280,7 +1386,7 @@ dependencies = [ "http 0.2.9", "http-body 0.4.5", "hyper 0.14.27", - "hyper-rustls", + "hyper-rustls 0.24.2", "once_cell", "pin-project-lite", "pin-utils", @@ -1310,7 +1416,7 @@ version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5d4bb944488536cd2fef43212d829bc7e9a8bfc4afa079d21170441e7be8d2d0" dependencies = [ - "base64-simd", + "base64-simd 0.8.0", "bytes", "bytes-utils", "futures-core", @@ -1358,7 +1464,7 @@ dependencies = [ "aws-smithy-runtime-api", "aws-smithy-types", "http 0.2.9", - "rustc_version", + "rustc_version 0.4.0", "tracing", ] @@ -1403,7 +1509,7 @@ dependencies = [ "http 1.0.0", "http-body 1.0.0", "http-body-util", - "hyper 1.2.0", + "hyper 1.1.0", "hyper-util", "itoa", "matchit", @@ -1546,13 +1652,22 @@ version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9475866fec1451be56a3c2400fd081ff546538961565ccb5b7142cbd22bc7a51" +[[package]] +name = "base64-simd" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "781dd20c3aff0bd194fe7d2a977dd92f21c173891f3a03b677359e5fa457e5d5" +dependencies = [ + "simd-abstraction", +] + [[package]] name = "base64-simd" version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "339abbe78e73178762e23bea9dfd08e697eb3f3301cd4be981c0f78ba5859195" dependencies = [ - "outref", + "outref 0.5.1", "vsimd", ] @@ -1591,6 +1706,15 @@ version = "0.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f40afb3abbf90895dda3ddbc6d8734d24215130a22d646067690f5e318f81bc" +[[package]] +name = "better_scoped_tls" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "794edcc9b3fb07bb4aecaa11f093fd45663b4feadb782d68303a2268bc2701de" +dependencies = [ + "scoped-tls", +] + [[package]] name = "bigdecimal" version = "0.3.1" @@ -1663,7 +1787,7 @@ dependencies = [ "rustc-hash", "shlex", "syn 2.0.57", - "which", + "which 4.4.2", ] [[package]] @@ -1937,6 +2061,12 @@ dependencies = [ "pkg-config", ] +[[package]] +name = "cache_control" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bf2a5fb3207c12b5d208ebc145f967fea5cac41a021c37417ccc31ba40f39ee" + [[package]] name = "camino" version = "1.1.6" @@ -2034,7 +2164,7 @@ checksum = "4acbb09d9ee8e23699b9634375c72795d095bf268439da88562cf9b501f181fa" dependencies = [ "camino", "cargo-platform", - "semver", + "semver 1.0.18", "serde", "serde_json", ] @@ -2465,6 +2595,12 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" +[[package]] +name = "convert_case" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6245d59a3e82a7fc217c5828a6692dbc6dfb63a0c8c90495621f7b9d79704a0e" + [[package]] name = "convert_case" version = "0.6.0" @@ -2474,6 +2610,12 @@ dependencies = [ "unicode-segmentation", ] +[[package]] +name = "cooked-waker" +version = "5.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "147be55d677052dabc6b22252d5dd0fd4c29c8c27aa4f2fbef0f94aa003b406f" + [[package]] name = "core-foundation" version = "0.9.3" @@ -2653,7 +2795,7 @@ version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d8f48d60e5b4d2c53d5c2b1d8a58c849a70ae5e5509b08a48d047e3b65714a74" dependencies = [ - "rustc_version", + "rustc_version 0.4.0", ] [[package]] @@ -2858,6 +3000,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" dependencies = [ "generic-array", + "rand_core", "typenum", ] @@ -2882,11 +3025,20 @@ dependencies = [ "memchr", ] +[[package]] +name = "ctr" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0369ee1ad671834580515889b80f2ea915f23b8be8d0daa4bbaf2ac5c7590835" +dependencies = [ + "cipher", +] + [[package]] name = "curve25519-dalek" -version = "4.1.0" +version = "4.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "622178105f911d937a42cdb140730ba4a3ed2becd8ae6ce39c7d28b5d75d4588" +checksum = "e89b8c6a2e4b1f45971ad09761aafb85514a84744b67a95e32c3cc1352d1f65c" dependencies = [ "cfg-if", "cpufeatures", @@ -2894,7 +3046,7 @@ dependencies = [ "digest", "fiat-crypto", "platforms", - "rustc_version", + "rustc_version 0.4.0", "subtle", "zeroize", ] @@ -3344,6 +3496,7 @@ version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" dependencies = [ + "serde", "uuid", ] @@ -3352,7 +3505,7 @@ name = "delta_btree_map" version = "1.7.0-alpha" dependencies = [ "educe 0.5.7", - "enum-as-inner", + "enum-as-inner 0.6.0", ] [[package]] @@ -3417,112 +3570,469 @@ dependencies = [ ] [[package]] -name = "der" -version = "0.6.1" +name = "deno_ast" +version = "0.34.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1a467a65c5e759bce6e65eaf91cc29f466cdc57cb65777bd646872a8a1fd4de" +checksum = "58d986a1df3f1538ffa04162b5c5f00b856121391b860dc003bde2a6a741e878" dependencies = [ - "const-oid", - "zeroize", + "anyhow", + "base64 0.21.7", + "deno_media_type", + "deno_terminal", + "dprint-swc-ext", + "once_cell", + "percent-encoding", + "serde", + "swc_atoms", + "swc_common", + "swc_config", + "swc_config_macro", + "swc_ecma_ast", + "swc_ecma_codegen", + "swc_ecma_codegen_macros", + "swc_ecma_loader", + "swc_ecma_parser", + "swc_ecma_transforms_base", + "swc_ecma_transforms_classes", + "swc_ecma_transforms_macros", + "swc_ecma_transforms_proposal", + "swc_ecma_transforms_react", + "swc_ecma_transforms_typescript", + "swc_ecma_utils", + "swc_ecma_visit", + "swc_eq_ignore_macros", + "swc_macros_common", + "swc_visit", + "swc_visit_macros", + "text_lines", + "unicode-width", + "url", ] [[package]] -name = "der" -version = "0.7.8" +name = "deno_console" +version = "0.144.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fffa369a668c8af7dbf8b5e56c9f744fbd399949ed171606040001947de40b1c" +checksum = "372a00d0d0c42c3f5a451037e0ce327fa284144112c3109d32b50b40a2ef9532" dependencies = [ - "const-oid", - "pem-rfc7468", - "zeroize", + "deno_core", ] [[package]] -name = "deranged" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f32d04922c60427da6f9fef14d042d9edddef64cb9d4ce0d64d0685fbeb1fd3" +name = "deno_core" +version = "0.272.0" +source = "git+https://github.com/bakjos/deno_core?rev=9b241c6#9b241c6aa536fd64add55b34b9532efc2ced9b7c" dependencies = [ - "powerfmt", + "anyhow", + "bincode 1.3.3", + "bit-set", + "bit-vec", + "bytes", + "cooked-waker", + "deno_core_icudata", + "deno_ops", + "deno_unsync", + "futures", + "libc", + "log", + "memoffset", + "parking_lot 0.12.1", + "pin-project", "serde", + "serde_json", + "serde_v8", + "smallvec", + "sourcemap 7.1.1", + "static_assertions", + "tokio", + "url", + "v8", ] [[package]] -name = "derivative" -version = "2.2.0" +name = "deno_core_icudata" +version = "0.0.73" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" -dependencies = [ - "proc-macro2", - "quote", - "syn 1.0.109", -] +checksum = "a13951ea98c0a4c372f162d669193b4c9d991512de9f2381dd161027f34b26b1" [[package]] -name = "derive_builder" -version = "0.12.0" +name = "deno_crypto" +version = "0.158.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d67778784b508018359cbc8696edb3db78160bab2c2a28ba7f56ef6932997f8" +checksum = "b26b7ac283934ee9c1ebb0b2328e4edda1487dd147d444c886993a96edf74b79" dependencies = [ - "derive_builder_macro 0.12.0", + "aes", + "aes-gcm", + "aes-kw", + "base64 0.21.7", + "cbc", + "const-oid", + "ctr", + "curve25519-dalek", + "deno_core", + "deno_web", + "elliptic-curve 0.13.8", + "num-traits", + "once_cell", + "p256 0.13.2", + "p384", + "p521", + "rand", + "ring 0.17.5", + "rsa", + "serde", + "serde_bytes", + "sha1", + "sha2", + "signature 2.2.0", + "spki 0.7.2", + "tokio", + "uuid", + "x25519-dalek", ] [[package]] -name = "derive_builder" -version = "0.20.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0350b5cb0331628a5916d6c5c0b72e97393b8b6b03b47a9284f4e7f5a405ffd7" +name = "deno_fetch" +version = "0.168.0" +source = "git+https://github.com/bakjos/deno?rev=787a232#787a232c51f1ea390a2aff2b622016217d854328" dependencies = [ - "derive_builder_macro 0.20.0", + "bytes", + "data-url", + "deno_core", + "deno_tls", + "dyn-clone", + "http 1.0.0", + "pin-project", + "reqwest 0.12.2", + "serde", + "serde_json", + "tokio", + "tokio-util", ] [[package]] -name = "derive_builder_core" -version = "0.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c11bdc11a0c47bc7d37d582b5285da6849c96681023680b906673c5707af7b0f" +name = "deno_http" +version = "0.141.0" +source = "git+https://github.com/bakjos/deno?rev=787a232#787a232c51f1ea390a2aff2b622016217d854328" dependencies = [ - "darling 0.14.4", - "proc-macro2", - "quote", - "syn 1.0.109", + "async-compression", + "async-trait", + "base64 0.21.7", + "brotli", + "bytes", + "cache_control", + "deno_core", + "deno_net", + "deno_websocket", + "flate2", + "http 1.0.0", + "httparse", + "hyper 0.14.27", + "hyper 1.1.0", + "hyper-util", + "itertools 0.10.5", + "memmem", + "mime", + "once_cell", + "percent-encoding", + "phf", + "pin-project", + "ring 0.17.5", + "scopeguard", + "serde", + "smallvec", + "thiserror", + "tokio", + "tokio-util", ] [[package]] -name = "derive_builder_core" -version = "0.20.0" +name = "deno_io" +version = "0.54.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d48cda787f839151732d396ac69e3473923d54312c070ee21e9effcaa8ca0b1d" +checksum = "3200f90cc9d232fd864cfcf25c535ab114fe824d9a38161e89418676360915e4" dependencies = [ - "darling 0.20.8", - "proc-macro2", - "quote", - "syn 2.0.57", + "async-trait", + "deno_core", + "filetime", + "fs3", + "once_cell", + "os_pipe", + "rand", + "tokio", + "winapi", ] [[package]] -name = "derive_builder_macro" -version = "0.12.0" +name = "deno_media_type" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ebcda35c7a396850a55ffeac740804b40ffec779b98fffbb1738f4033f0ee79e" +checksum = "edf9879493856d1622be70f396b0b0d3e519538dd6501b7c609ecbaa7e2194d2" dependencies = [ - "derive_builder_core 0.12.0", - "syn 1.0.109", + "data-url", + "serde", + "url", ] [[package]] -name = "derive_builder_macro" -version = "0.20.0" +name = "deno_native_certs" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "206868b8242f27cecce124c19fd88157fbd0dd334df2587f36417bafbc85097b" +checksum = "f4785d0bdc13819b665b71e4fb7e119d859568471e4c245ec5610857e70c9345" dependencies = [ - "derive_builder_core 0.20.0", - "syn 2.0.57", + "dlopen2", + "dlopen2_derive", + "once_cell", + "rustls-native-certs 0.6.3", + "rustls-pemfile 1.0.4", ] [[package]] -name = "derive_utils" -version = "0.14.1" +name = "deno_net" +version = "0.136.0" +source = "git+https://github.com/bakjos/deno?rev=787a232#787a232c51f1ea390a2aff2b622016217d854328" +dependencies = [ + "deno_core", + "deno_tls", + "enum-as-inner 0.5.1", + "log", + "pin-project", + "rustls-tokio-stream", + "serde", + "socket2 0.5.6", + "tokio", + "trust-dns-proto 0.22.0", + "trust-dns-resolver 0.22.0", +] + +[[package]] +name = "deno_ops" +version = "0.148.0" +source = "git+https://github.com/bakjos/deno_core?rev=9b241c6#9b241c6aa536fd64add55b34b9532efc2ced9b7c" +dependencies = [ + "proc-macro-rules", + "proc-macro2", + "quote", + "strum 0.25.0", + "strum_macros 0.25.3", + "syn 2.0.57", + "thiserror", +] + +[[package]] +name = "deno_terminal" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e6337d4e7f375f8b986409a76fbeecfa4bd8a1343e63355729ae4befa058eaf" +dependencies = [ + "once_cell", + "termcolor", +] + +[[package]] +name = "deno_tls" +version = "0.131.0" +source = "git+https://github.com/bakjos/deno?rev=787a232#787a232c51f1ea390a2aff2b622016217d854328" +dependencies = [ + "deno_core", + "deno_native_certs", + "once_cell", + "rustls 0.22.3", + "rustls-pemfile 1.0.4", + "rustls-tokio-stream", + "rustls-webpki 0.101.7", + "serde", + "webpki-roots 0.25.2", +] + +[[package]] +name = "deno_unsync" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30dff7e03584dbae188dae96a0f1876740054809b2ad0cf7c9fc5d361f20e739" +dependencies = [ + "tokio", +] + +[[package]] +name = "deno_url" +version = "0.144.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e172f44cffc771f0050d1befb12523d0420b6a44983d0c74d572609f4d3e02e3" +dependencies = [ + "deno_core", + "serde", + "urlpattern", +] + +[[package]] +name = "deno_web" +version = "0.175.0" +source = "git+https://github.com/bakjos/deno?rev=787a232#787a232c51f1ea390a2aff2b622016217d854328" +dependencies = [ + "async-trait", + "base64-simd 0.8.0", + "bytes", + "deno_core", + "encoding_rs", + "flate2", + "futures", + "serde", + "tokio", + "uuid", + "windows-sys 0.48.0", +] + +[[package]] +name = "deno_webidl" +version = "0.144.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c0484ffd8b436e926538fc0db4bc588832800b6681e554cebb3ae21f346c84e" +dependencies = [ + "deno_core", +] + +[[package]] +name = "deno_websocket" +version = "0.149.0" +source = "git+https://github.com/bakjos/deno?rev=787a232#787a232c51f1ea390a2aff2b622016217d854328" +dependencies = [ + "bytes", + "deno_core", + "deno_net", + "deno_tls", + "fastwebsockets", + "h2 0.4.4", + "http 1.0.0", + "http-body-util", + "hyper 1.1.0", + "hyper-util", + "once_cell", + "rustls-tokio-stream", + "serde", + "tokio", +] + +[[package]] +name = "der" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1a467a65c5e759bce6e65eaf91cc29f466cdc57cb65777bd646872a8a1fd4de" +dependencies = [ + "const-oid", + "zeroize", +] + +[[package]] +name = "der" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fffa369a668c8af7dbf8b5e56c9f744fbd399949ed171606040001947de40b1c" +dependencies = [ + "const-oid", + "pem-rfc7468", + "zeroize", +] + +[[package]] +name = "deranged" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f32d04922c60427da6f9fef14d042d9edddef64cb9d4ce0d64d0685fbeb1fd3" +dependencies = [ + "powerfmt", + "serde", +] + +[[package]] +name = "derivative" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "derive_builder" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8d67778784b508018359cbc8696edb3db78160bab2c2a28ba7f56ef6932997f8" +dependencies = [ + "derive_builder_macro 0.12.0", +] + +[[package]] +name = "derive_builder" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0350b5cb0331628a5916d6c5c0b72e97393b8b6b03b47a9284f4e7f5a405ffd7" +dependencies = [ + "derive_builder_macro 0.20.0", +] + +[[package]] +name = "derive_builder_core" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c11bdc11a0c47bc7d37d582b5285da6849c96681023680b906673c5707af7b0f" +dependencies = [ + "darling 0.14.4", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "derive_builder_core" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d48cda787f839151732d396ac69e3473923d54312c070ee21e9effcaa8ca0b1d" +dependencies = [ + "darling 0.20.8", + "proc-macro2", + "quote", + "syn 2.0.57", +] + +[[package]] +name = "derive_builder_macro" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebcda35c7a396850a55ffeac740804b40ffec779b98fffbb1738f4033f0ee79e" +dependencies = [ + "derive_builder_core 0.12.0", + "syn 1.0.109", +] + +[[package]] +name = "derive_builder_macro" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "206868b8242f27cecce124c19fd88157fbd0dd334df2587f36417bafbc85097b" +dependencies = [ + "derive_builder_core 0.20.0", + "syn 2.0.57", +] + +[[package]] +name = "derive_more" +version = "0.99.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fb810d30a7c1953f91334de7244731fc3f3c10d7fe163338a35b9f640960321" +dependencies = [ + "convert_case 0.4.0", + "proc-macro2", + "quote", + "rustc_version 0.4.0", + "syn 1.0.109", +] + +[[package]] +name = "derive_utils" +version = "0.14.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "61bb5a1014ce6dfc2a378578509abe775a5aa06bff584a547555d9efdb81b926" dependencies = [ @@ -3626,6 +4136,29 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "86e3bdc80eee6e16b2b6b0f87fbc98c04bee3455e35174c0de1a125d0688c632" +[[package]] +name = "dlopen2" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bc2c7ed06fd72a8513ded8d0d2f6fd2655a85d6885c48cae8625d80faf28c03" +dependencies = [ + "dlopen2_derive", + "libc", + "once_cell", + "winapi", +] + +[[package]] +name = "dlopen2_derive" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b99bf03862d7f545ebc28ddd33a665b50865f4dfd84031a393823879bd4c54" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.57", +] + [[package]] name = "dlv-list" version = "0.5.2" @@ -3659,6 +4192,21 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ea835d29036a4087793836fa931b08837ad5e957da9e23886b29586fb9b6650" +[[package]] +name = "dprint-swc-ext" +version = "0.15.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5bad772f9e49af3a613fcddf1671d1e2e877e0a6d94f2b7162bfea4ac8140bee" +dependencies = [ + "num-bigint", + "rustc-hash", + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "swc_ecma_parser", + "text_lines", +] + [[package]] name = "duct" version = "0.13.6" @@ -3717,15 +4265,15 @@ dependencies = [ [[package]] name = "ecdsa" -version = "0.16.8" +version = "0.16.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4b1e0c257a9e9f25f90ff76d7a68360ed497ee519c8e428d1825ef0000799d4" +checksum = "ee27f32b5c5292967d2d4a9d7f1e0b0aed2c15daded5a60300e4abb9d8020bca" dependencies = [ "der 0.7.8", "digest", - "elliptic-curve 0.13.6", + "elliptic-curve 0.13.8", "rfc6979 0.4.0", - "signature 2.0.0", + "signature 2.2.0", "spki 0.7.2", ] @@ -3736,20 +4284,21 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "60f6d271ca33075c88028be6f04d502853d63a5ece419d269c15315d4fc1cf1d" dependencies = [ "pkcs8 0.10.2", - "signature 2.0.0", + "signature 2.2.0", ] [[package]] name = "ed25519-dalek" -version = "2.0.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7277392b266383ef8396db7fdeb1e77b6c52fed775f5df15bb24f35b72156980" +checksum = "4a3daa8e81a3963a60642bcc1f90a670680bd4a77535faa384e9d1c79d620871" dependencies = [ "curve25519-dalek", "ed25519", "serde", "sha2", - "signature 2.0.0", + "signature 2.2.0", + "subtle", "zeroize", ] @@ -3808,9 +4357,9 @@ dependencies = [ [[package]] name = "elliptic-curve" -version = "0.13.6" +version = "0.13.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d97ca172ae9dc9f9b779a6e3a65d308f2af74e5b8c921299075bdb4a0370e914" +checksum = "b5e6043086bf7973472e0c7dff2142ea0b680d30e18d9cc40f267efbf222bd47" dependencies = [ "base16ct 0.2.0", "crypto-bigint 0.5.5", @@ -3842,6 +4391,18 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "enum-as-inner" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "enum-as-inner" version = "0.6.0" @@ -3869,7 +4430,7 @@ version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a0ce3a36047ede676eb0d2721d065beed8410cf4f113f489604d2971331cb378" dependencies = [ - "convert_case", + "convert_case 0.6.0", "quote", "syn 1.0.109", ] @@ -4103,6 +4664,25 @@ version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" +[[package]] +name = "fastwebsockets" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f63dd7b57f9b33b1741fa631c9522eb35d43e96dcca4a6a91d5e4ca7c93acdc1" +dependencies = [ + "base64 0.21.7", + "http-body-util", + "hyper 1.1.0", + "hyper-util", + "pin-project", + "rand", + "sha1", + "simdutf8", + "thiserror", + "tokio", + "utf-8", +] + [[package]] name = "fd-lock" version = "4.0.1" @@ -4150,16 +4730,28 @@ dependencies = [ ] [[package]] -name = "findshlibs" -version = "0.10.2" +name = "filetime" +version = "0.2.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40b9e59cd0f7e0806cca4be089683ecb6434e602038df21fe6bf6711b2f07f64" +checksum = "1ee447700ac8aa0b2f2bd7bc4462ad686ba06baa6727ac149a2d6277f0d240fd" dependencies = [ - "cc", - "lazy_static", + "cfg-if", "libc", - "winapi", -] + "redox_syscall 0.4.1", + "windows-sys 0.52.0", +] + +[[package]] +name = "findshlibs" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40b9e59cd0f7e0806cca4be089683ecb6434e602038df21fe6bf6711b2f07f64" +dependencies = [ + "cc", + "lazy_static", + "libc", + "winapi", +] [[package]] name = "finl_unicode" @@ -4212,7 +4804,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4dac53e22462d78c16d64a1cd22371b54cc3fe94aa15e7886a2fa6e5d1ab8640" dependencies = [ "bitflags 1.3.2", - "rustc_version", + "rustc_version 0.4.0", ] [[package]] @@ -4429,6 +5021,17 @@ version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c2141d6d6c8512188a7891b4b01590a45f6dac67afb4f255c4124dbb86d4eaa" +[[package]] +name = "from_variant" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a0b11eeb173ce52f84ebd943d42e58813a2ebb78a6a3ff0a243b71c5199cd7b" +dependencies = [ + "proc-macro2", + "swc_macros_common", + "syn 2.0.57", +] + [[package]] name = "frunk" version = "0.4.2" @@ -4511,12 +5114,33 @@ dependencies = [ "winapi", ] +[[package]] +name = "fs3" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb17cf6ed704f72485332f6ab65257460c4f9f3083934cf402bf9f5b3b600a90" +dependencies = [ + "libc", + "rustc_version 0.2.3", + "winapi", +] + [[package]] name = "fs_extra" version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42703706b716c37f96a77aea830392ad231f44c9e9a67872fa5548707e11b11c" +[[package]] +name = "fslock" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04412b8935272e3a9bae6f48c7bfff74c2911f60525404edfdd28e49884c3bfb" +dependencies = [ + "libc", + "winapi", +] + [[package]] name = "function_name" version = "0.3.0" @@ -4724,7 +5348,7 @@ dependencies = [ "async-trait", "dyn-clone", "hyper 0.14.27", - "hyper-rustls", + "hyper-rustls 0.24.2", "log", "reqwest 0.11.20", "serde", @@ -4790,6 +5414,16 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "ghash" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0d8a4362ccb29cb0b265253fb0a2728f592895ee6854fd9bc13f2ffda266ff1" +dependencies = [ + "opaque-debug", + "polyval", +] + [[package]] name = "gimli" version = "0.28.0" @@ -4815,7 +5449,7 @@ dependencies = [ "tonic 0.10.2", "tower", "tracing", - "trust-dns-resolver", + "trust-dns-resolver 0.23.2", ] [[package]] @@ -4963,6 +5597,15 @@ dependencies = [ "subtle", ] +[[package]] +name = "gzip-header" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95cc527b92e6029a62960ad99aa8a6660faa4555fe5f731aab13aa6a921795a2" +dependencies = [ + "crc32fast", +] + [[package]] name = "h2" version = "0.3.26" @@ -5158,6 +5801,20 @@ dependencies = [ "windows 0.52.0", ] +[[package]] +name = "hstr" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0f5356d62012374578cd3a5c013d6586de3efbca3b53379fc1edfbb95c9db14" +dependencies = [ + "hashbrown 0.14.3", + "new_debug_unreachable", + "once_cell", + "phf", + "rustc-hash", + "triomphe", +] + [[package]] name = "http" version = "0.2.9" @@ -5264,9 +5921,9 @@ dependencies = [ [[package]] name = "hyper" -version = "1.2.0" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "186548d73ac615b32a73aafe38fb4f56c0d340e110e5a200bcadbaf2e199263a" +checksum = "fb5aa53871fc917b1a9ed87b683a5d86db645e23acb32c2e0785a353e522fb75" dependencies = [ "bytes", "futures-channel", @@ -5278,7 +5935,6 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "smallvec", "tokio", "want", ] @@ -5297,7 +5953,24 @@ dependencies = [ "rustls-native-certs 0.6.3", "tokio", "tokio-rustls 0.24.1", - "webpki-roots", + "webpki-roots 0.25.2", +] + +[[package]] +name = "hyper-rustls" +version = "0.26.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0bea761b46ae2b24eb4aef630d8d1c398157b6fc29e6350ecf090a0b70c952c" +dependencies = [ + "futures-util", + "http 1.0.0", + "hyper 1.1.0", + "hyper-util", + "rustls 0.22.3", + "rustls-pki-types", + "tokio", + "tokio-rustls 0.25.0", + "tower-service", ] [[package]] @@ -5333,7 +6006,7 @@ checksum = "70206fc6890eaca9fde8a0bf71caa2ddfc9fe045ac9e5c70df101a7dbde866e0" dependencies = [ "bytes", "http-body-util", - "hyper 1.2.0", + "hyper 1.1.0", "hyper-util", "native-tls", "tokio", @@ -5352,7 +6025,7 @@ dependencies = [ "futures-util", "http 1.0.0", "http-body 1.0.0", - "hyper 1.2.0", + "hyper 1.1.0", "pin-project-lite", "socket2 0.5.6", "tokio", @@ -5456,6 +6129,17 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" +[[package]] +name = "idna" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "418a0a6fab821475f634efe3ccc45c013f742efe03d853e8d3355d5cb850ecf8" +dependencies = [ + "matches", + "unicode-bidi", + "unicode-normalization", +] + [[package]] name = "idna" version = "0.4.0" @@ -5476,6 +6160,12 @@ dependencies = [ "unicode-normalization", ] +[[package]] +name = "if_chain" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb56e1aa765b4b4f3aadfab769793b7087bb03a4ea4920644a6d238e2df5b9ed" + [[package]] name = "indexmap" version = "1.9.3" @@ -5639,6 +6329,18 @@ version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" +[[package]] +name = "is-macro" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59a85abdc13717906baccb5a1e435556ce0df215f242892f721dff62bf25288f" +dependencies = [ + "Inflector", + "proc-macro2", + "quote", + "syn 2.0.57", +] + [[package]] name = "is-terminal" version = "0.4.9" @@ -6435,6 +7137,12 @@ dependencies = [ "libc", ] +[[package]] +name = "memmem" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a64a92489e2744ce060c349162be1c5f33c6969234104dbd99ddb5feb08b8c15" + [[package]] name = "memoffset" version = "0.9.0" @@ -6468,9 +7176,9 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +checksum = "9d811f3e15f28568be3407c8e7fdb6514c1cda3cb30683f15b6a1a1dc4ea14a7" dependencies = [ "adler", ] @@ -6535,7 +7243,7 @@ dependencies = [ "once_cell", "parking_lot 0.12.1", "quanta", - "rustc_version", + "rustc_version 0.4.0", "skeptic", "smallvec", "tagptr", @@ -6687,6 +7395,12 @@ dependencies = [ "tempfile", ] +[[package]] +name = "new_debug_unreachable" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "650eef8c711430f1a879fdd01d4745a7deea475becfb90269c06775983bbf086" + [[package]] name = "newline-converter" version = "0.3.0" @@ -6834,6 +7548,8 @@ dependencies = [ "autocfg", "num-integer", "num-traits", + "rand", + "serde", ] [[package]] @@ -7025,9 +7741,9 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" +checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" [[package]] name = "oorandom" @@ -7035,6 +7751,12 @@ version = "11.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" +[[package]] +name = "opaque-debug" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08d65885ee38876c4f86fa503fb49d7b507c2b62552df7c70b2fce627e06381" + [[package]] name = "opendal" version = "0.45.1" @@ -7269,12 +7991,12 @@ dependencies = [ [[package]] name = "os_pipe" -version = "1.1.4" +version = "1.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ae859aa07428ca9a929b936690f8b12dc5f11dd8c6992a18ca93919f28bc177" +checksum = "57119c3b893986491ec9aa85056780d3a0f3cf4da7cc09dd3650dbd6c6738fb9" dependencies = [ "libc", - "windows-sys 0.48.0", + "windows-sys 0.52.0", ] [[package]] @@ -7321,6 +8043,12 @@ dependencies = [ "syn 2.0.57", ] +[[package]] +name = "outref" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f222829ae9293e33a9f5e9f440c6760a3d450a64affe1846486b140db81c1f4" + [[package]] name = "outref" version = "0.5.1" @@ -7356,8 +8084,8 @@ version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c9863ad85fa8f4460f9c48cb909d38a0d689dba1f6f6988a5e3e0d31071bcd4b" dependencies = [ - "ecdsa 0.16.8", - "elliptic-curve 0.13.6", + "ecdsa 0.16.9", + "elliptic-curve 0.13.8", "primeorder", "sha2", ] @@ -7368,9 +8096,23 @@ version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70786f51bcc69f6a4c0360e063a4cac5419ef7c5cd5b3c99ad70f3be5ba79209" dependencies = [ - "ecdsa 0.16.8", - "elliptic-curve 0.13.6", + "ecdsa 0.16.9", + "elliptic-curve 0.13.8", + "primeorder", + "sha2", +] + +[[package]] +name = "p521" +version = "0.13.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fc9e2161f1f215afdfce23677034ae137bbd45016a880c2eb3ba8eb95f085b2" +dependencies = [ + "base16ct 0.2.0", + "ecdsa 0.16.9", + "elliptic-curve 0.13.8", "primeorder", + "rand_core", "sha2", ] @@ -7563,6 +8305,12 @@ dependencies = [ "once_cell", ] +[[package]] +name = "pathdiff" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8835116a5c179084a830efb3adc117ab007512b535bc1a21c991d3b32a6b44dd" + [[package]] name = "pbjson" version = "0.6.0" @@ -7685,6 +8433,7 @@ version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" dependencies = [ + "phf_macros", "phf_shared", ] @@ -7708,6 +8457,19 @@ dependencies = [ "rand", ] +[[package]] +name = "phf_macros" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3444646e286606587e49f3bcf1679b8cef1dc2c5ecc29ddacaffc305180d464b" +dependencies = [ + "phf_generator", + "phf_shared", + "proc-macro2", + "quote", + "syn 2.0.57", +] + [[package]] name = "phf_shared" version = "0.11.2" @@ -7838,6 +8600,17 @@ dependencies = [ "plotters-backend", ] +[[package]] +name = "pmutil" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52a40bc70c2c58040d2d8b167ba9a5ff59fc9dab7ad44771cfde3dcfde7a09c6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.57", +] + [[package]] name = "polling" version = "2.8.0" @@ -7854,6 +8627,18 @@ dependencies = [ "windows-sys 0.48.0", ] +[[package]] +name = "polyval" +version = "0.6.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d1fe60d06143b2430aa532c94cfe9e29783047f06c0d7fd359a9a51b729fa25" +dependencies = [ + "cfg-if", + "cpufeatures", + "opaque-debug", + "universal-hash", +] + [[package]] name = "portable-atomic" version = "1.4.3" @@ -8036,11 +8821,11 @@ dependencies = [ [[package]] name = "primeorder" -version = "0.13.2" +version = "0.13.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c2fcef82c0ec6eefcc179b978446c399b3cdf73c392c35604e399eee6df1ee3" +checksum = "353e1ca18966c16d9deb1c69278edbc5f194139612772bd9537af60ac231e1e6" dependencies = [ - "elliptic-curve 0.13.6", + "elliptic-curve 0.13.8", ] [[package]] @@ -8127,6 +8912,29 @@ version = "0.5.20+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" +[[package]] +name = "proc-macro-rules" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07c277e4e643ef00c1233393c673f655e3672cf7eb3ba08a00bdd0ea59139b5f" +dependencies = [ + "proc-macro-rules-macros", + "proc-macro2", + "syn 2.0.57", +] + +[[package]] +name = "proc-macro-rules-macros" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "207fffb0fe655d1d47f6af98cc2793405e85929bdbc420d685554ff07be27ac7" +dependencies = [ + "once_cell", + "proc-macro2", + "quote", + "syn 2.0.57", +] + [[package]] name = "proc-macro2" version = "1.0.79" @@ -8195,7 +9003,7 @@ version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e7849dd7319357e220886589ea0a0760ff138cde2eedfbbfd2708caee0b15dfc" dependencies = [ - "enum-as-inner", + "enum-as-inner 0.6.0", "mime", "reqwest 0.11.20", "serde", @@ -8242,7 +9050,7 @@ dependencies = [ "regex", "syn 1.0.109", "tempfile", - "which", + "which 4.4.2", ] [[package]] @@ -8264,7 +9072,7 @@ dependencies = [ "regex", "syn 2.0.57", "tempfile", - "which", + "which 4.4.2", ] [[package]] @@ -8865,7 +9673,7 @@ dependencies = [ "http 0.2.9", "http-body 0.4.5", "hyper 0.14.27", - "hyper-rustls", + "hyper-rustls 0.24.2", "hyper-tls 0.5.0", "ipnet", "js-sys", @@ -8891,7 +9699,7 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams 0.3.0", "web-sys", - "webpki-roots", + "webpki-roots 0.25.2", "winreg", ] @@ -8901,6 +9709,7 @@ version = "0.12.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d66674f2b6fb864665eea7a3c1ac4e3dfacd2fda83cf6f935a612e01b0e3338" dependencies = [ + "async-compression", "base64 0.21.7", "bytes", "encoding_rs", @@ -8911,7 +9720,8 @@ dependencies = [ "http 1.0.0", "http-body 1.0.0", "http-body-util", - "hyper 1.2.0", + "hyper 1.1.0", + "hyper-rustls 0.26.0", "hyper-tls 0.6.0", "hyper-util", "ipnet", @@ -8922,7 +9732,9 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", + "rustls 0.22.3", "rustls-pemfile 1.0.4", + "rustls-pki-types", "serde", "serde_json", "serde_urlencoded", @@ -8930,6 +9742,8 @@ dependencies = [ "system-configuration", "tokio", "tokio-native-tls", + "tokio-rustls 0.25.0", + "tokio-socks", "tokio-util", "tower-service", "url", @@ -8937,6 +9751,7 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams 0.4.0", "web-sys", + "webpki-roots 0.26.1", "winreg", ] @@ -9268,7 +10083,7 @@ dependencies = [ "easy-ext", "educe 0.5.7", "either", - "enum-as-inner", + "enum-as-inner 0.6.0", "enumflags2", "ethnum", "expect-test", @@ -9574,7 +10389,7 @@ dependencies = [ "deltalake", "duration-str", "easy-ext", - "enum-as-inner", + "enum-as-inner 0.6.0", "expect-test", "futures", "futures-async-stream", @@ -9760,6 +10575,7 @@ dependencies = [ "arrow-array 50.0.0", "arrow-schema 50.0.0", "arrow-udf-js", + "arrow-udf-js-deno", "arrow-udf-python", "arrow-udf-wasm", "async-trait", @@ -9772,8 +10588,9 @@ dependencies = [ "easy-ext", "educe 0.5.7", "either", - "enum-as-inner", + "enum-as-inner 0.6.0", "expect-test", + "futures", "futures-async-stream", "futures-util", "itertools 0.12.1", @@ -9874,7 +10691,7 @@ dependencies = [ "easy-ext", "educe 0.5.7", "either", - "enum-as-inner", + "enum-as-inner 0.6.0", "expect-test", "fancy-regex", "fixedbitset 0.5.0", @@ -10096,7 +10913,7 @@ dependencies = [ "crepe", "easy-ext", "either", - "enum-as-inner", + "enum-as-inner 0.6.0", "expect-test", "fail", "function_name", @@ -10272,7 +11089,7 @@ dependencies = [ "fail", "futures", "hyper 0.14.27", - "hyper-rustls", + "hyper-rustls 0.24.2", "hyper-tls 0.5.0", "itertools 0.12.1", "madsim", @@ -10294,7 +11111,7 @@ dependencies = [ name = "risingwave_pb" version = "1.7.0-alpha" dependencies = [ - "enum-as-inner", + "enum-as-inner 0.6.0", "fs-err", "madsim-tonic", "madsim-tonic-build", @@ -10554,7 +11371,7 @@ dependencies = [ "dashmap", "dyn-clone", "either", - "enum-as-inner", + "enum-as-inner 0.6.0", "expect-test", "fail", "fiemap", @@ -10622,7 +11439,7 @@ dependencies = [ "delta_btree_map", "educe 0.5.7", "either", - "enum-as-inner", + "enum-as-inner 0.6.0", "expect-test", "fail", "foyer", @@ -10795,22 +11612,20 @@ dependencies = [ [[package]] name = "rsa" -version = "0.9.2" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ab43bb47d23c1a631b4b680199a45255dce26fa9ab2fa902581f624ff13e6a8" +checksum = "af6c4b23d99685a1408194da11270ef8e9809aff951cc70ec9b17350b087e474" dependencies = [ - "byteorder", "const-oid", "digest", "num-bigint-dig", "num-integer", - "num-iter", "num-traits", "pkcs1", "pkcs8 0.10.2", "rand_core", "sha2", - "signature 2.0.0", + "signature 2.2.0", "spki 0.7.2", "subtle", "zeroize", @@ -10910,13 +11725,22 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" +[[package]] +name = "rustc_version" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "138e3e0acb6c9fb258b19b67cb8abd63c00679d2851805ea151465464fe9030a" +dependencies = [ + "semver 0.9.0", +] + [[package]] name = "rustc_version" version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" dependencies = [ - "semver", + "semver 1.0.18", ] [[package]] @@ -11053,6 +11877,18 @@ version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ecd36cc4259e3e4514335c4a138c6b43171a8d61d8f5c9348f9fc7529416f247" +[[package]] +name = "rustls-tokio-stream" +version = "0.2.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "696a389edb0b54b9bb888c8318404d9a0c0b9091fb03ca3d9c64731511db03f6" +dependencies = [ + "futures", + "rustls 0.22.3", + "socket2 0.5.6", + "tokio", +] + [[package]] name = "rustls-webpki" version = "0.101.7" @@ -11150,6 +11986,12 @@ version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" +[[package]] +name = "ryu-js" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad97d4ce1560a5e27cec89519dc8300d1aa6035b099821261c651486a19e44d5" + [[package]] name = "salsa20" version = "0.10.2" @@ -11207,6 +12049,12 @@ dependencies = [ "hashbrown 0.13.2", ] +[[package]] +name = "scoped-tls" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" + [[package]] name = "scopeguard" version = "1.2.0" @@ -11474,6 +12322,15 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e388332cd64eb80cd595a00941baf513caffae8dce9cfd0467fc9c66397dade6" +[[package]] +name = "semver" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d7eb9ef2c18661902cc47e535f9bc51b78acd254da71d375c2f6720d9a40403" +dependencies = [ + "semver-parser", +] + [[package]] name = "semver" version = "1.0.18" @@ -11483,6 +12340,12 @@ dependencies = [ "serde", ] +[[package]] +name = "semver-parser" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "388a1df253eca08550bef6c72392cfe7c30914bf41df5269b68cbd6ff8f570a3" + [[package]] name = "seq-macro" version = "0.3.5" @@ -11631,6 +12494,19 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_v8" +version = "0.181.0" +source = "git+https://github.com/bakjos/deno_core?rev=9b241c6#9b241c6aa536fd64add55b34b9532efc2ced9b7c" +dependencies = [ + "bytes", + "num-bigint", + "serde", + "smallvec", + "thiserror", + "v8", +] + [[package]] name = "serde_with" version = "3.7.0" @@ -11699,11 +12575,22 @@ dependencies = [ "syn 2.0.57", ] +[[package]] +name = "sha-1" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "028f48d513f9678cda28f6e4064755b3fbb2af6acd672f2c209b62323f7aea0f" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest", +] + [[package]] name = "sha1" -version = "0.10.5" +version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f04293dc80c3993519f2d7f6f511707ee7094fe0c6d3406feb330cdb3540eba3" +checksum = "e3bf829a2d51ab4a5ddf1352d8470c140cadc8301b2ae1789db023f01cedd6ba" dependencies = [ "cfg-if", "cpufeatures", @@ -11718,9 +12605,9 @@ checksum = "ae1a47186c03a32177042e55dbc5fd5aee900b8e0069a8d70fba96a9375cd012" [[package]] name = "sha2" -version = "0.10.7" +version = "0.10.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "479fb9d862239e610720565ca91403019f2f00410f1864c5aa7479b950a76ed8" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" dependencies = [ "cfg-if", "cpufeatures", @@ -11814,7 +12701,7 @@ checksum = "c1e303f8205714074f6068773f0e29527e0453937fe837c9717d066635b65f31" dependencies = [ "pkcs8 0.10.2", "rand_core", - "signature 2.0.0", + "signature 2.2.0", "zeroize", ] @@ -11830,16 +12717,25 @@ dependencies = [ [[package]] name = "signature" -version = "2.0.0" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fe458c98333f9c8152221191a77e2a44e8325d0193484af2e9421a53019e57d" +checksum = "77549399552de45a898a580c1b41d445bf730df867cc44e6c0233bbc4b8329de" dependencies = [ "digest", "rand_core", ] [[package]] -name = "simd-json" +name = "simd-abstraction" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cadb29c57caadc51ff8346233b5cec1d240b68ce55cf1afc764818791876987" +dependencies = [ + "outref 0.1.0", +] + +[[package]] +name = "simd-json" version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d123f285a3635e423ec2ef5b67e0168dcf86c0d62fffbcea88fcd1c926e47413" @@ -11954,6 +12850,17 @@ dependencies = [ "serde", ] +[[package]] +name = "smartstring" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fb72c633efbaa2dd666986505016c32c3044395ceaf881518399d2f4127ee29" +dependencies = [ + "autocfg", + "static_assertions", + "version_check", +] + [[package]] name = "snafu" version = "0.7.5" @@ -12002,6 +12909,39 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "sourcemap" +version = "6.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4cbf65ca7dc576cf50e21f8d0712d96d4fcfd797389744b7b222a85cdf5bd90" +dependencies = [ + "data-encoding", + "debugid", + "if_chain", + "rustc_version 0.2.3", + "serde", + "serde_json", + "unicode-id", + "url", +] + +[[package]] +name = "sourcemap" +version = "7.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7768edd06c02535e0d50653968f46e1e0d3aa54742190d35dd9466f59de9c71" +dependencies = [ + "base64-simd 0.7.0", + "data-encoding", + "debugid", + "if_chain", + "rustc_version 0.2.3", + "serde", + "serde_json", + "unicode-id-start", + "url", +] + [[package]] name = "speedate" version = "0.14.0" @@ -12368,6 +13308,18 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" +[[package]] +name = "string_enum" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b650ea2087d32854a0f20b837fc56ec987a1cb4f758c9757e1171ee9812da63" +dependencies = [ + "proc-macro2", + "quote", + "swc_macros_common", + "syn 2.0.57", +] + [[package]] name = "stringprep" version = "0.1.4" @@ -12380,110 +13332,461 @@ dependencies = [ ] [[package]] -name = "strsim" -version = "0.10.0" +name = "strsim" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" + +[[package]] +name = "strsim" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ee073c9e4cd00e28217186dbe12796d692868f432bf2e97ee73bed0c56dfa01" + +[[package]] +name = "structmeta" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e1575d8d40908d70f6fd05537266b90ae71b15dbbe7a8b7dffa2b759306d329" +dependencies = [ + "proc-macro2", + "quote", + "structmeta-derive", + "syn 2.0.57", +] + +[[package]] +name = "structmeta-derive" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.57", +] + +[[package]] +name = "strum" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" +dependencies = [ + "strum_macros 0.25.3", +] + +[[package]] +name = "strum" +version = "0.26.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "723b93e8addf9aa965ebe2d11da6d7540fa2283fcea14b3371ff055f7ba13f5f" +dependencies = [ + "strum_macros 0.26.1", +] + +[[package]] +name = "strum_macros" +version = "0.25.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.57", +] + +[[package]] +name = "strum_macros" +version = "0.26.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a3417fc93d76740d974a01654a09777cb500428cc874ca9f45edfe0c4d4cd18" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.57", +] + +[[package]] +name = "subprocess" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c2e86926081dda636c546d8c5e641661049d7562a68f5488be4a1f7f66f6086" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "subst" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca1318e5d6716d6541696727c88d9b8dfc8cfe6afd6908e186546fd4af7f5b98" +dependencies = [ + "memchr", + "unicode-width", +] + +[[package]] +name = "subtle" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" + +[[package]] +name = "swc_atoms" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d538eaaa6f085161d088a04cf0a3a5a52c5a7f2b3bd9b83f73f058b0ed357c0" +dependencies = [ + "hstr", + "once_cell", + "rustc-hash", + "serde", +] + +[[package]] +name = "swc_cached" +version = "0.3.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "630c761c74ac8021490b78578cc2223aa4a568241e26505c27bf0e4fd4ad8ec2" +dependencies = [ + "ahash 0.8.6", + "anyhow", + "dashmap", + "once_cell", + "regex", + "serde", +] + +[[package]] +name = "swc_common" +version = "0.33.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c85e8b15d0fb87691e27c8f3cf953748db3ccd2a39e165d6d5275a48fb0d29e3" +dependencies = [ + "ast_node", + "better_scoped_tls", + "cfg-if", + "either", + "from_variant", + "new_debug_unreachable", + "num-bigint", + "once_cell", + "rustc-hash", + "serde", + "siphasher", + "sourcemap 6.4.1", + "swc_atoms", + "swc_eq_ignore_macros", + "swc_visit", + "tracing", + "unicode-width", + "url", +] + +[[package]] +name = "swc_config" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce837c5eae1cb200a310940de989fd9b3d12ed62d7752bc69b39ef8aa775ec04" +dependencies = [ + "anyhow", + "indexmap 2.0.0", + "serde", + "serde_json", + "swc_cached", + "swc_config_macro", +] + +[[package]] +name = "swc_config_macro" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b2574f75082322a27d990116cd2a24de52945fc94172b24ca0b3e9e2a6ceb6b" +dependencies = [ + "proc-macro2", + "quote", + "swc_macros_common", + "syn 2.0.57", +] + +[[package]] +name = "swc_ecma_ast" +version = "0.112.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "36226eb87bfd2f5620bde04f149a4b869ab34e78496d60cb0d8eb9da765d0732" +dependencies = [ + "bitflags 2.5.0", + "is-macro", + "num-bigint", + "phf", + "scoped-tls", + "serde", + "string_enum", + "swc_atoms", + "swc_common", + "unicode-id", +] + +[[package]] +name = "swc_ecma_codegen" +version = "0.148.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ba8669ab28bb5d1e65c1e8690257c026745ac368e0101c2c6544d4a03afc95e" +dependencies = [ + "memchr", + "num-bigint", + "once_cell", + "rustc-hash", + "serde", + "sourcemap 6.4.1", + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "swc_ecma_codegen_macros", + "tracing", +] + +[[package]] +name = "swc_ecma_codegen_macros" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "394b8239424b339a12012ceb18726ed0244fce6bf6345053cb9320b2791dcaa5" +dependencies = [ + "proc-macro2", + "quote", + "swc_macros_common", + "syn 2.0.57", +] + +[[package]] +name = "swc_ecma_loader" +version = "0.45.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0058cf970880f5382effe43eb2b727a73ba09ae41922fa140c2c3fa6ca9b2d1" +dependencies = [ + "anyhow", + "pathdiff", + "serde", + "swc_atoms", + "swc_common", + "tracing", +] + +[[package]] +name = "swc_ecma_parser" +version = "0.143.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "20823cac99a9adbd4c03fb5e126aaccbf92446afedad99252a0e1fc76e2ffc43" +dependencies = [ + "either", + "new_debug_unreachable", + "num-bigint", + "num-traits", + "phf", + "serde", + "smallvec", + "smartstring", + "stacker", + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "tracing", + "typed-arena", +] + +[[package]] +name = "swc_ecma_transforms_base" +version = "0.137.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "66539401f619730b26d380a120b91b499f80cbdd9bb15d00aa73bc3a4d4cc394" +dependencies = [ + "better_scoped_tls", + "bitflags 2.5.0", + "indexmap 2.0.0", + "once_cell", + "phf", + "rustc-hash", + "serde", + "smallvec", + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "swc_ecma_parser", + "swc_ecma_utils", + "swc_ecma_visit", + "tracing", +] + +[[package]] +name = "swc_ecma_transforms_classes" +version = "0.126.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebf9048e687b746d2bbe6149601c3eedd819fef08d7657e5fddcef99b22febba" +dependencies = [ + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "swc_ecma_transforms_base", + "swc_ecma_utils", + "swc_ecma_visit", +] + +[[package]] +name = "swc_ecma_transforms_macros" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" +checksum = "17e309b88f337da54ef7fe4c5b99c2c522927071f797ee6c9fb8b6bf2d100481" +dependencies = [ + "proc-macro2", + "quote", + "swc_macros_common", + "syn 2.0.57", +] [[package]] -name = "strsim" -version = "0.11.0" +name = "swc_ecma_transforms_proposal" +version = "0.171.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ee073c9e4cd00e28217186dbe12796d692868f432bf2e97ee73bed0c56dfa01" +checksum = "35f0a72ee781aa9208836046fd2c12e483f5515898858511b68863290cb97b45" +dependencies = [ + "either", + "rustc-hash", + "serde", + "smallvec", + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "swc_ecma_transforms_base", + "swc_ecma_transforms_classes", + "swc_ecma_transforms_macros", + "swc_ecma_utils", + "swc_ecma_visit", +] [[package]] -name = "structmeta" -version = "0.3.0" +name = "swc_ecma_transforms_react" +version = "0.183.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e1575d8d40908d70f6fd05537266b90ae71b15dbbe7a8b7dffa2b759306d329" +checksum = "f0ec75c1194365abe4d44d94e58f918ec853469ecd39733b381a089cfdcdee1a" dependencies = [ - "proc-macro2", - "quote", - "structmeta-derive", - "syn 2.0.57", + "base64 0.21.7", + "dashmap", + "indexmap 2.0.0", + "once_cell", + "serde", + "sha-1", + "string_enum", + "swc_atoms", + "swc_common", + "swc_config", + "swc_ecma_ast", + "swc_ecma_parser", + "swc_ecma_transforms_base", + "swc_ecma_transforms_macros", + "swc_ecma_utils", + "swc_ecma_visit", ] [[package]] -name = "structmeta-derive" -version = "0.3.0" +name = "swc_ecma_transforms_typescript" +version = "0.188.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" +checksum = "fec5e95a9c840eb13562884123eaa627cb6e05e0461c94a2ce69ae7e70313010" dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.57", + "ryu-js", + "serde", + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "swc_ecma_transforms_base", + "swc_ecma_transforms_react", + "swc_ecma_utils", + "swc_ecma_visit", ] [[package]] -name = "strum" -version = "0.25.0" +name = "swc_ecma_utils" +version = "0.127.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" +checksum = "14482e455df85486d68a51533a31645d511e56df93a35cadf0eabbe7abe96b98" dependencies = [ - "strum_macros 0.25.3", + "indexmap 2.0.0", + "num_cpus", + "once_cell", + "rustc-hash", + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "swc_ecma_visit", + "tracing", + "unicode-id", ] [[package]] -name = "strum" -version = "0.26.1" +name = "swc_ecma_visit" +version = "0.98.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "723b93e8addf9aa965ebe2d11da6d7540fa2283fcea14b3371ff055f7ba13f5f" +checksum = "df0127694c36d656ea9eab5c170cdd8ab398246ae2a335de26961c913a4aca47" dependencies = [ - "strum_macros 0.26.1", + "num-bigint", + "swc_atoms", + "swc_common", + "swc_ecma_ast", + "swc_visit", + "tracing", ] [[package]] -name = "strum_macros" -version = "0.25.3" +name = "swc_eq_ignore_macros" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" +checksum = "695a1d8b461033d32429b5befbf0ad4d7a2c4d6ba9cd5ba4e0645c615839e8e4" dependencies = [ - "heck 0.4.1", "proc-macro2", "quote", - "rustversion", "syn 2.0.57", ] [[package]] -name = "strum_macros" -version = "0.26.1" +name = "swc_macros_common" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a3417fc93d76740d974a01654a09777cb500428cc874ca9f45edfe0c4d4cd18" +checksum = "50176cfc1cbc8bb22f41c6fe9d1ec53fbe057001219b5954961b8ad0f336fce9" dependencies = [ - "heck 0.4.1", "proc-macro2", "quote", - "rustversion", "syn 2.0.57", ] [[package]] -name = "subprocess" -version = "0.2.9" +name = "swc_visit" +version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c2e86926081dda636c546d8c5e641661049d7562a68f5488be4a1f7f66f6086" +checksum = "358e246dedeb4ae8efacebcce1360dc2f9b6c0b4c1ad8b737cc60f5b6633691a" dependencies = [ - "libc", - "winapi", + "either", + "swc_visit_macros", ] [[package]] -name = "subst" -version = "0.3.0" +name = "swc_visit_macros" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca1318e5d6716d6541696727c88d9b8dfc8cfe6afd6908e186546fd4af7f5b98" +checksum = "fbbbb9d77d5112f90ed7ea00477135b16c4370c872b93a0b63b766e8710650ad" dependencies = [ - "memchr", - "unicode-width", + "Inflector", + "pmutil", + "proc-macro2", + "quote", + "swc_macros_common", + "syn 2.0.57", ] -[[package]] -name = "subtle" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" - [[package]] name = "symbolic-common" version = "12.4.0" @@ -12664,6 +13967,15 @@ version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" +[[package]] +name = "text_lines" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fd5828de7deaa782e1dd713006ae96b3bee32d3279b79eb67ecf8072c059bcf" +dependencies = [ + "serde", +] + [[package]] name = "thiserror" version = "1.0.58" @@ -12971,6 +14283,18 @@ dependencies = [ "tokio", ] +[[package]] +name = "tokio-socks" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51165dfa029d2a65969413a6cc96f354b86b464498702f174a4efa13608fd8c0" +dependencies = [ + "either", + "futures-util", + "thiserror", + "tokio", +] + [[package]] name = "tokio-stream" version = "0.1.14" @@ -13133,7 +14457,7 @@ dependencies = [ "tower-layer", "tower-service", "tracing", - "webpki-roots", + "webpki-roots 0.25.2", ] [[package]] @@ -13351,9 +14675,13 @@ dependencies = [ [[package]] name = "triomphe" -version = "0.1.9" +version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0eee8098afad3fb0c54a9007aab6804558410503ad676d4633f9c2559a00ac0f" +checksum = "859eb650cfee7434994602c3a68b25d77ad9e68c8a6cd491616ef86661382eb3" +dependencies = [ + "serde", + "stable_deref_trait", +] [[package]] name = "triple_accel" @@ -13361,6 +14689,32 @@ version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "622b09ce2fe2df4618636fb92176d205662f59803f39e70d1c333393082de96c" +[[package]] +name = "trust-dns-proto" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f7f83d1e4a0e4358ac54c5c3681e5d7da5efc5a7a632c90bb6d6669ddd9bc26" +dependencies = [ + "async-trait", + "cfg-if", + "data-encoding", + "enum-as-inner 0.5.1", + "futures-channel", + "futures-io", + "futures-util", + "idna 0.2.3", + "ipnet", + "lazy_static", + "rand", + "serde", + "smallvec", + "thiserror", + "tinyvec", + "tokio", + "tracing", + "url", +] + [[package]] name = "trust-dns-proto" version = "0.23.2" @@ -13370,7 +14724,7 @@ dependencies = [ "async-trait", "cfg-if", "data-encoding", - "enum-as-inner", + "enum-as-inner 0.6.0", "futures-channel", "futures-io", "futures-util", @@ -13386,6 +14740,27 @@ dependencies = [ "url", ] +[[package]] +name = "trust-dns-resolver" +version = "0.22.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aff21aa4dcefb0a1afbfac26deb0adc93888c7d295fb63ab273ef276ba2b7cfe" +dependencies = [ + "cfg-if", + "futures-util", + "ipconfig", + "lazy_static", + "lru-cache", + "parking_lot 0.12.1", + "resolv-conf", + "serde", + "smallvec", + "thiserror", + "tokio", + "tracing", + "trust-dns-proto 0.22.0", +] + [[package]] name = "trust-dns-resolver" version = "0.23.2" @@ -13404,7 +14779,7 @@ dependencies = [ "thiserror", "tokio", "tracing", - "trust-dns-proto", + "trust-dns-proto 0.23.2", ] [[package]] @@ -13435,6 +14810,12 @@ dependencies = [ "static_assertions", ] +[[package]] +name = "typed-arena" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6af6ae20167a9ece4bcb41af5b80f8a1f1df981f6391189ce00fd257af04126a" + [[package]] name = "typed-builder" version = "0.16.2" @@ -13490,6 +14871,47 @@ dependencies = [ "version_check", ] +[[package]] +name = "unic-char-property" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8c57a407d9b6fa02b4795eb81c5b6652060a15a7903ea981f3d723e6c0be221" +dependencies = [ + "unic-char-range", +] + +[[package]] +name = "unic-char-range" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0398022d5f700414f6b899e10b8348231abf9173fa93144cbc1a43b9793c1fbc" + +[[package]] +name = "unic-common" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "80d7ff825a6a654ee85a63e80f92f054f904f21e7d12da4e22f9834a4aaa35bc" + +[[package]] +name = "unic-ucd-ident" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e230a37c0381caa9219d67cf063aa3a375ffed5bf541a452db16e744bdab6987" +dependencies = [ + "unic-char-property", + "unic-char-range", + "unic-ucd-version", +] + +[[package]] +name = "unic-ucd-version" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96bd2f2237fe450fcd0a1d2f5f4e91711124f7857ba2e964247776ebeeb7b0c4" +dependencies = [ + "unic-common", +] + [[package]] name = "unicase" version = "2.7.0" @@ -13505,6 +14927,18 @@ version = "0.3.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460" +[[package]] +name = "unicode-id" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1b6def86329695390197b82c1e244a54a131ceb66c996f2088a3876e2ae083f" + +[[package]] +name = "unicode-id-start" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8f73150333cb58412db36f2aca8f2875b013049705cc77b94ded70a1ab1f5da" + [[package]] name = "unicode-ident" version = "1.0.11" @@ -13550,6 +14984,16 @@ version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c7de7d73e1754487cb58364ee906a499937a0dfabd86bcb980fa99ec8c8fa2ce" +[[package]] +name = "universal-hash" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fc1de2c688dc15305988b563c3854064043356019f97a4b46276fe734c4f07ea" +dependencies = [ + "crypto-common", + "subtle", +] + [[package]] name = "unsafe-libyaml" version = "0.2.10" @@ -13586,6 +15030,25 @@ version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "daf8dba3b7eb870caf1ddeed7bc9d2a049f3cfdfae7cb521b087cc33ae4c49da" +[[package]] +name = "urlpattern" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9bd5ff03aea02fa45b13a7980151fe45009af1980ba69f651ec367121a31609" +dependencies = [ + "derive_more", + "regex", + "serde", + "unic-ucd-ident", + "url", +] + +[[package]] +name = "utf-8" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09cc8ee72d2a9becf2f2febe0205bbed8fc6615b7cb429ad062dc7b7ddd036a9" + [[package]] name = "utf8parse" version = "0.2.1" @@ -13603,6 +15066,21 @@ dependencies = [ "serde", ] +[[package]] +name = "v8" +version = "0.89.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe2197fbef82c98f7953d13568a961d4e1c663793b5caf3c74455a13918cdf33" +dependencies = [ + "bitflags 2.5.0", + "fslock", + "gzip-header", + "home", + "miniz_oxide", + "once_cell", + "which 5.0.0", +] + [[package]] name = "valuable" version = "0.1.0" @@ -13844,7 +15322,7 @@ checksum = "84e5df6dba6c0d7fafc63a450f1738451ed7a0b52295d83e868218fa286bf708" dependencies = [ "bitflags 2.5.0", "indexmap 2.0.0", - "semver", + "semver 1.0.18", ] [[package]] @@ -13881,7 +15359,7 @@ dependencies = [ "paste", "rayon", "rustix 0.38.31", - "semver", + "semver 1.0.18", "serde", "serde_derive", "serde_json", @@ -14216,6 +15694,15 @@ version = "0.25.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "14247bb57be4f377dfb94c72830b8ce8fc6beac03cf4bf7b9732eadd414123fc" +[[package]] +name = "webpki-roots" +version = "0.26.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b3de34ae270483955a94f4b21bdaaeb83d508bb84a01435f393818edb0012009" +dependencies = [ + "rustls-pki-types", +] + [[package]] name = "which" version = "4.4.2" @@ -14228,6 +15715,19 @@ dependencies = [ "rustix 0.38.31", ] +[[package]] +name = "which" +version = "5.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9bf3ea8596f3a0dd5980b46430f2058dfe2c36a27ccfbb1845d6fbfcd9ba6e14" +dependencies = [ + "either", + "home", + "once_cell", + "rustix 0.38.31", + "windows-sys 0.48.0", +] + [[package]] name = "whoami" version = "1.5.0" @@ -14627,7 +16127,7 @@ dependencies = [ "id-arena", "indexmap 2.0.0", "log", - "semver", + "semver 1.0.18", "serde", "serde_derive", "serde_json", @@ -14682,6 +16182,18 @@ dependencies = [ "tap", ] +[[package]] +name = "x25519-dalek" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7e468321c81fb07fa7f4c636c3972b9100f0346e5b6a9f2bd0603a52f7ed277" +dependencies = [ + "curve25519-dalek", + "rand_core", + "serde", + "zeroize", +] + [[package]] name = "xmlparser" version = "0.13.5" @@ -14740,7 +16252,7 @@ dependencies = [ "futures", "http 0.2.9", "hyper 0.14.27", - "hyper-rustls", + "hyper-rustls 0.24.2", "itertools 0.10.5", "log", "percent-encoding", @@ -14786,6 +16298,20 @@ name = "zeroize" version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "525b4ec142c6b68a2d10f01f7bbf6755599ca3f81ea53b8431b7dd348f5fdb2d" +dependencies = [ + "zeroize_derive", +] + +[[package]] +name = "zeroize_derive" +version = "1.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.57", +] [[package]] name = "zstd" diff --git a/Cargo.toml b/Cargo.toml index 2bd253f9e4d2..165549c71993 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -140,6 +140,7 @@ arrow-select = "50" arrow-ord = "50" arrow-row = "50" arrow-udf-js = "0.1" +arrow-udf-js-deno = { git = "https://github.com/risingwavelabs/arrow-udf.git", rev = "23fe0dd" } arrow-udf-wasm = { version = "0.2.1", features = ["build"] } arrow-udf-python = { git = "https://github.com/risingwavelabs/arrow-udf.git", rev = "6c32f71" } arrow-array-deltalake = { package = "arrow-array", version = "48.0.1" } @@ -315,6 +316,15 @@ futures-timer = { git = "https://github.com/madsim-rs/futures-timer.git", rev = etcd-client = { git = "https://github.com/risingwavelabs/etcd-client.git", rev = "4e84d40" } # todo(wcy-fdu): remove this patch fork after opendal release a new version to apply azure workload identity change. reqsign = { git = "https://github.com/wcy-fdu/reqsign.git", rev = "002ee2a" } +# patch to remove preserve_order from serde_json +deno_core = { git = "https://github.com/bakjos/deno_core", rev = "9b241c6" } +# patch to user reqwest 0.12.2 +deno_fetch = { git = "https://github.com/bakjos/deno", rev = "787a232" } +deno_http = { git = "https://github.com/bakjos/deno", rev = "787a232" } +deno_net = { git = "https://github.com/bakjos/deno", rev = "787a232" } +deno_tls = { git = "https://github.com/bakjos/deno", rev = "787a232" } +deno_web = { git = "https://github.com/bakjos/deno", rev = "787a232" } +deno_websocket = { git = "https://github.com/bakjos/deno", rev = "787a232" } [workspace.metadata.dylint] libraries = [{ path = "./lints" }] diff --git a/Makefile.toml b/Makefile.toml index f95ed99e5ec4..5e9d65c00dd3 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -35,6 +35,7 @@ is_release = get_env ENABLE_RELEASE_PROFILE is_not_release = not ${is_release} is_dynamic_linking = get_env ENABLE_DYNAMIC_LINKING is_hummock_trace = get_env ENABLE_HUMMOCK_TRACE +is_deno_udf_enabled = get_env ENABLE_DENO_UDF is_python_udf_enabled = get_env ENABLE_PYTHON_UDF if ${is_sanitizer_enabled} @@ -58,6 +59,11 @@ else set_env RISINGWAVE_FEATURE_FLAGS "--features rw-static-link" end +if ${is_deno_udf_enabled} + flags = get_env RISINGWAVE_FEATURE_FLAGS + set_env RISINGWAVE_FEATURE_FLAGS "${flags} --features embedded-deno-udf" +end + if ${is_python_udf_enabled} flags = get_env RISINGWAVE_FEATURE_FLAGS set_env RISINGWAVE_FEATURE_FLAGS "${flags} --features embedded-python-udf" diff --git a/ci/scripts/build.sh b/ci/scripts/build.sh index 0f4d40d8ff65..f50c5a794ec8 100755 --- a/ci/scripts/build.sh +++ b/ci/scripts/build.sh @@ -54,6 +54,7 @@ cargo build \ -p risingwave_compaction_test \ -p risingwave_e2e_extended_mode_test \ "${RISINGWAVE_FEATURE_FLAGS[@]}" \ + --features embedded-deno-udf \ --features embedded-python-udf \ --profile "$profile" diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index 9c205e418222..da42fb5acac3 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -123,10 +123,12 @@ sqllogictest -p 4566 -d dev './e2e_test/udf/external_udf.slt' pkill java echo "--- e2e, $mode, embedded udf" +python3 -m pip install --break-system-packages flask waitress sqllogictest -p 4566 -d dev './e2e_test/udf/wasm_udf.slt' sqllogictest -p 4566 -d dev './e2e_test/udf/rust_udf.slt' sqllogictest -p 4566 -d dev './e2e_test/udf/js_udf.slt' sqllogictest -p 4566 -d dev './e2e_test/udf/python_udf.slt' +sqllogictest -p 4566 -d dev './e2e_test/udf/deno_udf.slt' echo "--- Kill cluster" cluster_stop diff --git a/docker/Dockerfile b/docker/Dockerfile index 8f6bbeb045a6..3d902bc2ffc6 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -75,7 +75,7 @@ WORKDIR /risingwave ENV ENABLE_BUILD_DASHBOARD=1 RUN cargo fetch && \ - cargo build -p risingwave_cmd_all --release --features "rw-static-link" --features embedded-python-udf && \ + cargo build -p risingwave_cmd_all --release --features "rw-static-link" --features embedded-deno-udf --features embedded-python-udf && \ mkdir -p /risingwave/bin && \ mv /risingwave/target/release/risingwave /risingwave/bin/ && \ mv /risingwave/target/release/risingwave.dwp /risingwave/bin/ && \ diff --git a/docker/Dockerfile.hdfs b/docker/Dockerfile.hdfs index 482f8758aac7..53a6da30fe6e 100644 --- a/docker/Dockerfile.hdfs +++ b/docker/Dockerfile.hdfs @@ -105,7 +105,7 @@ ENV JAVA_HOME ${JAVA_HOME_PATH} ENV LD_LIBRARY_PATH ${JAVA_HOME_PATH}/lib/server:${LD_LIBRARY_PATH} RUN cargo fetch && \ - cargo build -p risingwave_cmd_all --release -p risingwave_object_store --features hdfs-backend --features "rw-static-link" --features embedded-python-udf && \ + cargo build -p risingwave_cmd_all --release -p risingwave_object_store --features hdfs-backend --features "rw-static-link" --features embedded-deno-udf --features embedded-python-udf && \ mkdir -p /risingwave/bin && \ mv /risingwave/target/release/risingwave /risingwave/bin/ && \ mv /risingwave/target/release/risingwave.dwp /risingwave/bin/ && \ diff --git a/e2e_test/udf/deno_udf.slt b/e2e_test/udf/deno_udf.slt new file mode 100644 index 000000000000..35e16daea16a --- /dev/null +++ b/e2e_test/udf/deno_udf.slt @@ -0,0 +1,233 @@ +statement ok +CREATE FUNCTION gcd(a int, b int) RETURNS int LANGUAGE javascript RUNTIME deno AS $$ + if(a == null || b == null) { + return null; + } + while (b != 0) { + let t = b; + b = a % b; + a = t; + } + return a; +$$; + +query I +select gcd(25, 15); +---- +5 + +statement ok +drop function gcd; + +statement ok +create function decimal_add(a decimal, b decimal) returns decimal language javascript RUNTIME deno as $$ + return a.add(b); +$$; + +query R +select decimal_add(1.11, 2.22); +---- +3.33 + +statement ok +drop function decimal_add; + + +statement ok +create function to_string(a boolean, b smallint, c int, d bigint, e real, f float, g decimal, h varchar, i bytea, j jsonb) returns varchar language javascript RUNTIME deno as $$ + return a.toString() + b.toString() + c.toString() + d.toString() + e.toString() + f.toString() + g.toString() + h.toString() + i.toString() + JSON.stringify(j); +$$; + +query T +select to_string(false, 1::smallint, 2, 3, 4.5, 6.7, 8.9, 'abc', '\x010203', '{"key": 1}'); +---- +false1234.56.78.9abc1,2,3{"key":1} + +statement ok +drop function to_string; + +# show data types in javascript +statement ok +create function js_typeof(a boolean, b smallint, c int, d bigint, e real, f float, g decimal, h varchar, i bytea, j jsonb) returns jsonb language javascript RUNTIME deno as $$ + return { + boolean: typeof a, + smallint: typeof b, + int: typeof c, + bigint: typeof d, + real: typeof e, + float: typeof f, + decimal: typeof g, + varchar: typeof h, + bytea: typeof i, + jsonb: typeof j, + }; +$$; + +query T +select js_typeof(false, 1::smallint, 2, 3, 4.5, 6.7, 8.9, 'abc', '\x010203', '{"key": 1}'); +---- +{"bigint": "bigint", "boolean": "boolean", "bytea": "object", "decimal": "object", "float": "number", "int": "number", "jsonb": "object", "real": "number", "smallint": "number", "varchar": "string"} + +statement ok +drop function js_typeof; + +statement ok +create function return_all(a boolean, b smallint, c int, d bigint, e real, f float, g decimal, h varchar, i bytea, j jsonb, s struct) +returns struct> +language javascript runtime deno as $$ + return {a,b,c,d,e,f,g,h,i,j,s}; +$$; + +query T +select (return_all( + true, + 1 ::smallint, + 1, + 1, + 1, + 1, + 12345678901234567890.12345678, + 'string', + 'bytes', + '{"key":1}', + row(1, 2)::struct +)).*; +---- +t 1 1 1 1 1 12345678901234567890.12345678 string \x6279746573 {"key": 1} (1,2) + +statement ok +drop function return_all; + + +statement ok +create function series(n int) returns table (x int) language javascript RUNTIME deno as $$ + for(let i = 0; i < n; i++) { + yield i; + } +$$; + +query I +select series(5); +---- +0 +1 +2 +3 +4 + +statement ok +drop function series; + + +statement ok +create function split(s varchar) returns table (word varchar, length int) language javascript RUNTIME deno as $$ + for(let word of s.split(' ')) { + yield { word: word, length: word.length }; + } +$$; + +query IT +select * from split('rising wave'); +---- +rising 6 +wave 4 + +statement ok +drop function split; + + +statement ok +CREATE FUNCTION digest( t string ) RETURNS bytea LANGUAGE javascript RUNTIME deno AS $$ + const subtle = crypto.subtle; + const key = await subtle.generateKey({ + name: 'HMAC', + hash: 'SHA-256', + length: 256, + }, true, ['sign', 'verify']); + const enc = new TextEncoder(); + const message = enc.encode(t); + const result = await subtle.sign({ + name: 'HMAC', + }, key, message); + return result; +$$ ASYNC; + +query I +select bit_length(digest('Hello')); +---- +256 + +statement ok +drop function digest; + +statement ok +CREATE FUNCTION delay_response() + RETURNS TABLE (x int) LANGUAGE javascript RUNTIME deno AS $$ + const delayedResponses = { + delays: [50, 10, 15], + wait(delay) { + return new Promise((resolve) => { + setTimeout(resolve, delay); + }); + }, + async *[Symbol.asyncIterator]() { + for (const delay of this.delays) { + await this.wait(delay); + yield delay; + } + }, + }; + return delayedResponses; +$$ SYNC; + +query I +select * FROM delay_response(); +---- +50 +10 +15 + +statement ok +drop function delay_response; + +system ok +python3 e2e_test/udf/mock_server.py & + +# wait for server to start +sleep 1s + +statement ok +CREATE FUNCTION call_sse() RETURNS TABLE ( data struct>) LANGUAGE javascript RUNTIME deno USING LINK 'fs://e2e_test/udf/sse/bundled.table.js' SYNC; + +query I +select * FROM call_sse(); +---- +(Hi) +(Bonjour) +(Hola) +(Ciao) +(Zdravo) + +statement ok +drop function call_sse; + +statement ok +CREATE FUNCTION fetch_api() RETURNS TABLE ( data struct< idx int>) LANGUAGE javascript RUNTIME deno AS $$ + const response = await fetch('http://127.0.0.1:4200'); + const resp = await response.json(); + for (const r of resp.results) { + yield r; + } +$$ ASYNC GENERATOR; + +query I +select * FROM fetch_api(); +---- +1 +2 + +statement ok +drop function fetch_api; + +system ok +pkill -9 python3 diff --git a/e2e_test/udf/mock_server.py b/e2e_test/udf/mock_server.py new file mode 100644 index 000000000000..8f525f946ee4 --- /dev/null +++ b/e2e_test/udf/mock_server.py @@ -0,0 +1,43 @@ +import json + +from flask import Flask, Response, stream_with_context, jsonify + +app = Flask(__name__) +def format_sse(data: str | None, event=None) -> str: + if data: + msg = f'data: {data}\n\n' + else: + msg = '\n' + + if event is not None: + msg = f'event: {event}\n{msg}' + + return msg + +@app.route('/') +def home(): + return jsonify({"results": [{"idx": 1}, {"idx": 2}]}) + +@app.route('/graphql/stream', methods=['POST']) +def stream(): + print("sse stream called") + @stream_with_context + def eventStream(): + messages = ["Hi", "Bonjour", "Hola", "Ciao", "Zdravo"] + for msg in messages: + data = { + "data": { + "greetings": msg + } + } + yield format_sse(json.dumps(data), "next") + + yield format_sse(None, "complete") + return Response(eventStream(), mimetype="text/event-stream") + +if __name__ == '__main__': + from waitress import serve + from werkzeug.serving import WSGIRequestHandler + WSGIRequestHandler.protocol_version = "HTTP/1.1" + serve(app, host="127.0.0.1", port=4200) + print("Server stopped.") \ No newline at end of file diff --git a/e2e_test/udf/requirements.txt b/e2e_test/udf/requirements.txt new file mode 100644 index 000000000000..8642e2b1ec25 --- /dev/null +++ b/e2e_test/udf/requirements.txt @@ -0,0 +1,2 @@ +flask +waitress \ No newline at end of file diff --git a/e2e_test/udf/sse/bundled.table.js b/e2e_test/udf/sse/bundled.table.js new file mode 100644 index 000000000000..91c7b3dd2af8 --- /dev/null +++ b/e2e_test/udf/sse/bundled.table.js @@ -0,0 +1,186 @@ +//This code was generated using esbuild +var z=(()=>{var h=(e,t)=>()=>(t||e((t={exports:{}}).exports,t),t.exports);var Pn=h(jn=>{"use strict";Object.defineProperty(jn,"__esModule",{value:!0});jn.isObject=void 0;function Zd(e){return typeof e=="object"&&e!==null}jn.isObject=Zd});var dr=h(Re=>{"use strict";Object.defineProperty(Re,"__esModule",{value:!0});Re.isAsyncGenerator=Re.isAsyncIterable=Re.parseStreamData=Re.print=Re.validateStreamEvent=Re.TOKEN_QUERY_KEY=Re.TOKEN_HEADER_KEY=void 0;var ef=Pn();Re.TOKEN_HEADER_KEY="x-graphql-event-stream-token";Re.TOKEN_QUERY_KEY="token";function tf(e){if(e=e,e!=="next"&&e!=="complete")throw new Error(`Invalid stream event "${e}"`);return e}Re.validateStreamEvent=tf;function rf(e){let t=`event: ${e.event} +data:`;return e.data&&(t+=" ",t+=JSON.stringify(e.data)),t+=` + +`,t}Re.print=rf;function nf(e,t){if(t)try{t=JSON.parse(t)}catch{throw new Error("Invalid stream data")}if(e==="next"&&!t)throw new Error('Stream data must be an object for "next" events');return t||null}Re.parseStreamData=nf;function sf(e){return typeof Object(e)[Symbol.asyncIterator]=="function"}Re.isAsyncIterable=sf;function af(e){return(0,ef.isObject)(e)&&typeof Object(e)[Symbol.asyncIterator]=="function"&&typeof e.return=="function"&&typeof e.throw=="function"&&typeof e.next=="function"}Re.isAsyncGenerator=af});var Gu=h(fr=>{"use strict";Object.defineProperty(fr,"__esModule",{value:!0});fr.versionInfo=fr.version=void 0;var of="16.8.1";fr.version=of;var uf=Object.freeze({major:16,minor:8,patch:1,preReleaseTag:null});fr.versionInfo=uf});var Fe=h(rs=>{"use strict";Object.defineProperty(rs,"__esModule",{value:!0});rs.devAssert=cf;function cf(e,t){if(!!!e)throw new Error(t)}});var wn=h(ns=>{"use strict";Object.defineProperty(ns,"__esModule",{value:!0});ns.isPromise=lf;function lf(e){return typeof e?.then=="function"}});var pt=h(is=>{"use strict";Object.defineProperty(is,"__esModule",{value:!0});is.isObjectLike=pf;function pf(e){return typeof e=="object"&&e!==null}});var je=h(ss=>{"use strict";Object.defineProperty(ss,"__esModule",{value:!0});ss.invariant=df;function df(e,t){if(!!!e)throw new Error(t??"Unexpected invariant triggered.")}});var Fn=h(as=>{"use strict";Object.defineProperty(as,"__esModule",{value:!0});as.getLocation=yf;var ff=je(),mf=/\r\n|[\n\r]/g;function yf(e,t){let r=0,n=1;for(let i of e.body.matchAll(mf)){if(typeof i.index=="number"||(0,ff.invariant)(!1),i.index>=t)break;r=i.index+i[0].length,n+=1}return{line:n,column:t+1-r}}});var os=h(kn=>{"use strict";Object.defineProperty(kn,"__esModule",{value:!0});kn.printLocation=vf;kn.printSourceLocation=Qu;var hf=Fn();function vf(e){return Qu(e.source,(0,hf.getLocation)(e.source,e.start))}function Qu(e,t){let r=e.locationOffset.column-1,n="".padStart(r)+e.body,i=t.line-1,s=e.locationOffset.line-1,a=t.line+s,o=t.line===1?r:0,u=t.column+o,c=`${e.name}:${a}:${u} +`,l=n.split(/\r\n|[\n\r]/g),f=l[i];if(f.length>120){let d=Math.floor(u/80),y=u%80,T=[];for(let I=0;I["|",I]),["|","^".padStart(y)],["|",T[d+1]]])}return c+Cu([[`${a-1} |`,l[i-1]],[`${a} |`,f],["|","^".padStart(u)],[`${a+1} |`,l[i+1]]])}function Cu(e){let t=e.filter(([n,i])=>i!==void 0),r=Math.max(...t.map(([n])=>n.length));return t.map(([n,i])=>n.padStart(r)+(i?" "+i:"")).join(` +`)}});var q=h(mr=>{"use strict";Object.defineProperty(mr,"__esModule",{value:!0});mr.GraphQLError=void 0;mr.formatError=Ef;mr.printError=gf;var Tf=pt(),Uu=Fn(),Ku=os();function bf(e){let t=e[0];return t==null||"kind"in t||"length"in t?{nodes:t,source:e[1],positions:e[2],path:e[3],originalError:e[4],extensions:e[5]}:t}var us=class e extends Error{constructor(t,...r){var n,i,s;let{nodes:a,source:o,positions:u,path:c,originalError:l,extensions:f}=bf(r);super(t),this.name="GraphQLError",this.path=c??void 0,this.originalError=l??void 0,this.nodes=$u(Array.isArray(a)?a:a?[a]:void 0);let d=$u((n=this.nodes)===null||n===void 0?void 0:n.map(T=>T.loc).filter(T=>T!=null));this.source=o??(d==null||(i=d[0])===null||i===void 0?void 0:i.source),this.positions=u??d?.map(T=>T.start),this.locations=u&&o?u.map(T=>(0,Uu.getLocation)(o,T)):d?.map(T=>(0,Uu.getLocation)(T.source,T.start));let y=(0,Tf.isObjectLike)(l?.extensions)?l?.extensions:void 0;this.extensions=(s=f??y)!==null&&s!==void 0?s:Object.create(null),Object.defineProperties(this,{message:{writable:!0,enumerable:!0},name:{enumerable:!1},nodes:{enumerable:!1},source:{enumerable:!1},positions:{enumerable:!1},originalError:{enumerable:!1}}),l!=null&&l.stack?Object.defineProperty(this,"stack",{value:l.stack,writable:!0,configurable:!0}):Error.captureStackTrace?Error.captureStackTrace(this,e):Object.defineProperty(this,"stack",{value:Error().stack,writable:!0,configurable:!0})}get[Symbol.toStringTag](){return"GraphQLError"}toString(){let t=this.message;if(this.nodes)for(let r of this.nodes)r.loc&&(t+=` + +`+(0,Ku.printLocation)(r.loc));else if(this.source&&this.locations)for(let r of this.locations)t+=` + +`+(0,Ku.printSourceLocation)(this.source,r);return t}toJSON(){let t={message:this.message};return this.locations!=null&&(t.locations=this.locations),this.path!=null&&(t.path=this.path),this.extensions!=null&&Object.keys(this.extensions).length>0&&(t.extensions=this.extensions),t}};mr.GraphQLError=us;function $u(e){return e===void 0||e.length===0?void 0:e}function gf(e){return e.toString()}function Ef(e){return e.toJSON()}});var qn=h(cs=>{"use strict";Object.defineProperty(cs,"__esModule",{value:!0});cs.syntaxError=Nf;var _f=q();function Nf(e,t,r){return new _f.GraphQLError(`Syntax Error: ${r}`,{source:e,positions:[t]})}});var dt=h(Ke=>{"use strict";Object.defineProperty(Ke,"__esModule",{value:!0});Ke.Token=Ke.QueryDocumentKeys=Ke.OperationTypeNode=Ke.Location=void 0;Ke.isNode=If;var ls=class{constructor(t,r,n){this.start=t.start,this.end=r.end,this.startToken=t,this.endToken=r,this.source=n}get[Symbol.toStringTag](){return"Location"}toJSON(){return{start:this.start,end:this.end}}};Ke.Location=ls;var ps=class{constructor(t,r,n,i,s,a){this.kind=t,this.start=r,this.end=n,this.line=i,this.column=s,this.value=a,this.prev=null,this.next=null}get[Symbol.toStringTag](){return"Token"}toJSON(){return{kind:this.kind,value:this.value,line:this.line,column:this.column}}};Ke.Token=ps;var Bu={Name:[],Document:["definitions"],OperationDefinition:["name","variableDefinitions","directives","selectionSet"],VariableDefinition:["variable","type","defaultValue","directives"],Variable:["name"],SelectionSet:["selections"],Field:["alias","name","arguments","directives","selectionSet"],Argument:["name","value"],FragmentSpread:["name","directives"],InlineFragment:["typeCondition","directives","selectionSet"],FragmentDefinition:["name","variableDefinitions","typeCondition","directives","selectionSet"],IntValue:[],FloatValue:[],StringValue:[],BooleanValue:[],NullValue:[],EnumValue:[],ListValue:["values"],ObjectValue:["fields"],ObjectField:["name","value"],Directive:["name","arguments"],NamedType:["name"],ListType:["type"],NonNullType:["type"],SchemaDefinition:["description","directives","operationTypes"],OperationTypeDefinition:["type"],ScalarTypeDefinition:["description","name","directives"],ObjectTypeDefinition:["description","name","interfaces","directives","fields"],FieldDefinition:["description","name","arguments","type","directives"],InputValueDefinition:["description","name","type","defaultValue","directives"],InterfaceTypeDefinition:["description","name","interfaces","directives","fields"],UnionTypeDefinition:["description","name","directives","types"],EnumTypeDefinition:["description","name","directives","values"],EnumValueDefinition:["description","name","directives"],InputObjectTypeDefinition:["description","name","directives","fields"],DirectiveDefinition:["description","name","arguments","locations"],SchemaExtension:["directives","operationTypes"],ScalarTypeExtension:["name","directives"],ObjectTypeExtension:["name","interfaces","directives","fields"],InterfaceTypeExtension:["name","interfaces","directives","fields"],UnionTypeExtension:["name","directives","types"],EnumTypeExtension:["name","directives","values"],InputObjectTypeExtension:["name","directives","fields"]};Ke.QueryDocumentKeys=Bu;var Of=new Set(Object.keys(Bu));function If(e){let t=e?.kind;return typeof t=="string"&&Of.has(t)}var ds;Ke.OperationTypeNode=ds;(function(e){e.QUERY="query",e.MUTATION="mutation",e.SUBSCRIPTION="subscription"})(ds||(Ke.OperationTypeNode=ds={}))});var yr=h(wr=>{"use strict";Object.defineProperty(wr,"__esModule",{value:!0});wr.DirectiveLocation=void 0;var fs;wr.DirectiveLocation=fs;(function(e){e.QUERY="QUERY",e.MUTATION="MUTATION",e.SUBSCRIPTION="SUBSCRIPTION",e.FIELD="FIELD",e.FRAGMENT_DEFINITION="FRAGMENT_DEFINITION",e.FRAGMENT_SPREAD="FRAGMENT_SPREAD",e.INLINE_FRAGMENT="INLINE_FRAGMENT",e.VARIABLE_DEFINITION="VARIABLE_DEFINITION",e.SCHEMA="SCHEMA",e.SCALAR="SCALAR",e.OBJECT="OBJECT",e.FIELD_DEFINITION="FIELD_DEFINITION",e.ARGUMENT_DEFINITION="ARGUMENT_DEFINITION",e.INTERFACE="INTERFACE",e.UNION="UNION",e.ENUM="ENUM",e.ENUM_VALUE="ENUM_VALUE",e.INPUT_OBJECT="INPUT_OBJECT",e.INPUT_FIELD_DEFINITION="INPUT_FIELD_DEFINITION"})(fs||(wr.DirectiveLocation=fs={}))});var H=h(Fr=>{"use strict";Object.defineProperty(Fr,"__esModule",{value:!0});Fr.Kind=void 0;var ms;Fr.Kind=ms;(function(e){e.NAME="Name",e.DOCUMENT="Document",e.OPERATION_DEFINITION="OperationDefinition",e.VARIABLE_DEFINITION="VariableDefinition",e.SELECTION_SET="SelectionSet",e.FIELD="Field",e.ARGUMENT="Argument",e.FRAGMENT_SPREAD="FragmentSpread",e.INLINE_FRAGMENT="InlineFragment",e.FRAGMENT_DEFINITION="FragmentDefinition",e.VARIABLE="Variable",e.INT="IntValue",e.FLOAT="FloatValue",e.STRING="StringValue",e.BOOLEAN="BooleanValue",e.NULL="NullValue",e.ENUM="EnumValue",e.LIST="ListValue",e.OBJECT="ObjectValue",e.OBJECT_FIELD="ObjectField",e.DIRECTIVE="Directive",e.NAMED_TYPE="NamedType",e.LIST_TYPE="ListType",e.NON_NULL_TYPE="NonNullType",e.SCHEMA_DEFINITION="SchemaDefinition",e.OPERATION_TYPE_DEFINITION="OperationTypeDefinition",e.SCALAR_TYPE_DEFINITION="ScalarTypeDefinition",e.OBJECT_TYPE_DEFINITION="ObjectTypeDefinition",e.FIELD_DEFINITION="FieldDefinition",e.INPUT_VALUE_DEFINITION="InputValueDefinition",e.INTERFACE_TYPE_DEFINITION="InterfaceTypeDefinition",e.UNION_TYPE_DEFINITION="UnionTypeDefinition",e.ENUM_TYPE_DEFINITION="EnumTypeDefinition",e.ENUM_VALUE_DEFINITION="EnumValueDefinition",e.INPUT_OBJECT_TYPE_DEFINITION="InputObjectTypeDefinition",e.DIRECTIVE_DEFINITION="DirectiveDefinition",e.SCHEMA_EXTENSION="SchemaExtension",e.SCALAR_TYPE_EXTENSION="ScalarTypeExtension",e.OBJECT_TYPE_EXTENSION="ObjectTypeExtension",e.INTERFACE_TYPE_EXTENSION="InterfaceTypeExtension",e.UNION_TYPE_EXTENSION="UnionTypeExtension",e.ENUM_TYPE_EXTENSION="EnumTypeExtension",e.INPUT_OBJECT_TYPE_EXTENSION="InputObjectTypeExtension"})(ms||(Fr.Kind=ms={}))});var xn=h(Qt=>{"use strict";Object.defineProperty(Qt,"__esModule",{value:!0});Qt.isDigit=Yu;Qt.isLetter=ys;Qt.isNameContinue=Df;Qt.isNameStart=Sf;Qt.isWhiteSpace=Lf;function Lf(e){return e===9||e===32}function Yu(e){return e>=48&&e<=57}function ys(e){return e>=97&&e<=122||e>=65&&e<=90}function Sf(e){return ys(e)||e===95}function Df(e){return ys(e)||Yu(e)||e===95}});var qr=h(kr=>{"use strict";Object.defineProperty(kr,"__esModule",{value:!0});kr.dedentBlockStringLines=Af;kr.isPrintableAsBlockString=jf;kr.printBlockString=Pf;var hs=xn();function Af(e){var t;let r=Number.MAX_SAFE_INTEGER,n=null,i=-1;for(let a=0;ao===0?a:a.slice(r)).slice((t=n)!==null&&t!==void 0?t:0,i+1)}function Rf(e){let t=0;for(;t1&&n.slice(1).every(y=>y.length===0||(0,hs.isWhiteSpace)(y.charCodeAt(0))),a=r.endsWith('\\"""'),o=e.endsWith('"')&&!a,u=e.endsWith("\\"),c=o||u,l=!(t!=null&&t.minimize)&&(!i||e.length>70||c||s||a),f="",d=i&&(0,hs.isWhiteSpace)(e.charCodeAt(0));return(l&&!d||s)&&(f+=` +`),f+=r,(l||c)&&(f+=` +`),'"""'+f+'"""'}});var Vr=h(xr=>{"use strict";Object.defineProperty(xr,"__esModule",{value:!0});xr.TokenKind=void 0;var vs;xr.TokenKind=vs;(function(e){e.SOF="",e.EOF="",e.BANG="!",e.DOLLAR="$",e.AMP="&",e.PAREN_L="(",e.PAREN_R=")",e.SPREAD="...",e.COLON=":",e.EQUALS="=",e.AT="@",e.BRACKET_L="[",e.BRACKET_R="]",e.BRACE_L="{",e.PIPE="|",e.BRACE_R="}",e.NAME="Name",e.INT="Int",e.FLOAT="Float",e.STRING="String",e.BLOCK_STRING="BlockString",e.COMMENT="Comment"})(vs||(xr.TokenKind=vs={}))});var Mn=h(Gr=>{"use strict";Object.defineProperty(Gr,"__esModule",{value:!0});Gr.Lexer=void 0;Gr.isPunctuatorTokenKind=Ff;var rt=qn(),Xu=dt(),wf=qr(),Ut=xn(),U=Vr(),bs=class{constructor(t){let r=new Xu.Token(U.TokenKind.SOF,0,0,0,0);this.source=t,this.lastToken=r,this.token=r,this.line=1,this.lineStart=0}get[Symbol.toStringTag](){return"Lexer"}advance(){return this.lastToken=this.token,this.token=this.lookahead()}lookahead(){let t=this.token;if(t.kind!==U.TokenKind.EOF)do if(t.next)t=t.next;else{let r=kf(this,t.end);t.next=r,r.prev=t,t=r}while(t.kind===U.TokenKind.COMMENT);return t}};Gr.Lexer=bs;function Ff(e){return e===U.TokenKind.BANG||e===U.TokenKind.DOLLAR||e===U.TokenKind.AMP||e===U.TokenKind.PAREN_L||e===U.TokenKind.PAREN_R||e===U.TokenKind.SPREAD||e===U.TokenKind.COLON||e===U.TokenKind.EQUALS||e===U.TokenKind.AT||e===U.TokenKind.BRACKET_L||e===U.TokenKind.BRACKET_R||e===U.TokenKind.BRACE_L||e===U.TokenKind.PIPE||e===U.TokenKind.BRACE_R}function hr(e){return e>=0&&e<=55295||e>=57344&&e<=1114111}function Vn(e,t){return Hu(e.charCodeAt(t))&&zu(e.charCodeAt(t+1))}function Hu(e){return e>=55296&&e<=56319}function zu(e){return e>=56320&&e<=57343}function Kt(e,t){let r=e.source.body.codePointAt(t);if(r===void 0)return U.TokenKind.EOF;if(r>=32&&r<=126){let n=String.fromCodePoint(r);return n==='"'?`'"'`:`"${n}"`}return"U+"+r.toString(16).toUpperCase().padStart(4,"0")}function ve(e,t,r,n,i){let s=e.line,a=1+r-e.lineStart;return new Xu.Token(t,r,n,s,a,i)}function kf(e,t){let r=e.source.body,n=r.length,i=t;for(;i=48&&e<=57?e-48:e>=65&&e<=70?e-55:e>=97&&e<=102?e-87:-1}function Cf(e,t){let r=e.source.body;switch(r.charCodeAt(t+1)){case 34:return{value:'"',size:2};case 92:return{value:"\\",size:2};case 47:return{value:"/",size:2};case 98:return{value:"\b",size:2};case 102:return{value:"\f",size:2};case 110:return{value:` +`,size:2};case 114:return{value:"\r",size:2};case 116:return{value:" ",size:2}}throw(0,rt.syntaxError)(e.source,t,`Invalid character escape sequence: "${r.slice(t,t+2)}".`)}function Qf(e,t){let r=e.source.body,n=r.length,i=e.lineStart,s=t+3,a=s,o="",u=[];for(;s{"use strict";Object.defineProperty(gs,"__esModule",{value:!0});gs.inspect=$f;var Kf=10,Wu=2;function $f(e){return Gn(e,[])}function Gn(e,t){switch(typeof e){case"string":return JSON.stringify(e);case"function":return e.name?`[function ${e.name}]`:"[function]";case"object":return Bf(e,t);default:return String(e)}}function Bf(e,t){if(e===null)return"null";if(t.includes(e))return"[Circular]";let r=[...t,e];if(Yf(e)){let n=e.toJSON();if(n!==e)return typeof n=="string"?n:Gn(n,r)}else if(Array.isArray(e))return Xf(e,r);return Jf(e,r)}function Yf(e){return typeof e.toJSON=="function"}function Jf(e,t){let r=Object.entries(e);return r.length===0?"{}":t.length>Wu?"["+Hf(e)+"]":"{ "+r.map(([i,s])=>i+": "+Gn(s,t)).join(", ")+" }"}function Xf(e,t){if(e.length===0)return"[]";if(t.length>Wu)return"[Array]";let r=Math.min(Kf,e.length),n=e.length-r,i=[];for(let s=0;s1&&i.push(`... ${n} more items`),"["+i.join(", ")+"]"}function Hf(e){let t=Object.prototype.toString.call(e).replace(/^\[object /,"").replace(/]$/,"");if(t==="Object"&&typeof e.constructor=="function"){let r=e.constructor.name;if(typeof r=="string"&&r!=="")return r}return t}});var Cr=h(Cn=>{"use strict";Object.defineProperty(Cn,"__esModule",{value:!0});Cn.instanceOf=void 0;var zf=re(),Wf=globalThis.process&&globalThis.process.env.NODE_ENV==="production"?function(t,r){return t instanceof r}:function(t,r){if(t instanceof r)return!0;if(typeof t=="object"&&t!==null){var n;let i=r.prototype[Symbol.toStringTag],s=Symbol.toStringTag in t?t[Symbol.toStringTag]:(n=t.constructor)===null||n===void 0?void 0:n.name;if(i===s){let a=(0,zf.inspect)(t);throw new Error(`Cannot use ${i} "${a}" from another module or realm. + +Ensure that there is only one instance of "graphql" in the node_modules +directory. If different versions of "graphql" are the dependencies of other +relied on modules, use "resolutions" to ensure only one version is installed. + +https://yarnpkg.com/en/docs/selective-version-resolutions + +Duplicate "graphql" modules cannot be used at the same time since different +versions may have different capabilities and behavior. The data from one +version used in the function from another could produce confusing and +spurious results.`)}}return!1};Cn.instanceOf=Wf});var Un=h(Qr=>{"use strict";Object.defineProperty(Qr,"__esModule",{value:!0});Qr.Source=void 0;Qr.isSource=tm;var Es=Fe(),Zf=re(),em=Cr(),Qn=class{constructor(t,r="GraphQL request",n={line:1,column:1}){typeof t=="string"||(0,Es.devAssert)(!1,`Body must be a string. Received: ${(0,Zf.inspect)(t)}.`),this.body=t,this.name=r,this.locationOffset=n,this.locationOffset.line>0||(0,Es.devAssert)(!1,"line in locationOffset is 1-indexed and must be positive."),this.locationOffset.column>0||(0,Es.devAssert)(!1,"column in locationOffset is 1-indexed and must be positive.")}get[Symbol.toStringTag](){return"Source"}};Qr.Source=Qn;function tm(e){return(0,em.instanceOf)(e,Qn)}});var vr=h(Pt=>{"use strict";Object.defineProperty(Pt,"__esModule",{value:!0});Pt.Parser=void 0;Pt.parse=nm;Pt.parseConstValue=sm;Pt.parseType=am;Pt.parseValue=im;var $t=qn(),Ur=dt(),rm=yr(),M=H(),ec=Mn(),Zu=Un(),O=Vr();function nm(e,t){return new Bt(e,t).parseDocument()}function im(e,t){let r=new Bt(e,t);r.expectToken(O.TokenKind.SOF);let n=r.parseValueLiteral(!1);return r.expectToken(O.TokenKind.EOF),n}function sm(e,t){let r=new Bt(e,t);r.expectToken(O.TokenKind.SOF);let n=r.parseConstValueLiteral();return r.expectToken(O.TokenKind.EOF),n}function am(e,t){let r=new Bt(e,t);r.expectToken(O.TokenKind.SOF);let n=r.parseTypeReference();return r.expectToken(O.TokenKind.EOF),n}var Bt=class{constructor(t,r={}){let n=(0,Zu.isSource)(t)?t:new Zu.Source(t);this._lexer=new ec.Lexer(n),this._options=r,this._tokenCounter=0}parseName(){let t=this.expectToken(O.TokenKind.NAME);return this.node(t,{kind:M.Kind.NAME,value:t.value})}parseDocument(){return this.node(this._lexer.token,{kind:M.Kind.DOCUMENT,definitions:this.many(O.TokenKind.SOF,this.parseDefinition,O.TokenKind.EOF)})}parseDefinition(){if(this.peek(O.TokenKind.BRACE_L))return this.parseOperationDefinition();let t=this.peekDescription(),r=t?this._lexer.lookahead():this._lexer.token;if(r.kind===O.TokenKind.NAME){switch(r.value){case"schema":return this.parseSchemaDefinition();case"scalar":return this.parseScalarTypeDefinition();case"type":return this.parseObjectTypeDefinition();case"interface":return this.parseInterfaceTypeDefinition();case"union":return this.parseUnionTypeDefinition();case"enum":return this.parseEnumTypeDefinition();case"input":return this.parseInputObjectTypeDefinition();case"directive":return this.parseDirectiveDefinition()}if(t)throw(0,$t.syntaxError)(this._lexer.source,this._lexer.token.start,"Unexpected description, descriptions are supported only on type definitions.");switch(r.value){case"query":case"mutation":case"subscription":return this.parseOperationDefinition();case"fragment":return this.parseFragmentDefinition();case"extend":return this.parseTypeSystemExtension()}}throw this.unexpected(r)}parseOperationDefinition(){let t=this._lexer.token;if(this.peek(O.TokenKind.BRACE_L))return this.node(t,{kind:M.Kind.OPERATION_DEFINITION,operation:Ur.OperationTypeNode.QUERY,name:void 0,variableDefinitions:[],directives:[],selectionSet:this.parseSelectionSet()});let r=this.parseOperationType(),n;return this.peek(O.TokenKind.NAME)&&(n=this.parseName()),this.node(t,{kind:M.Kind.OPERATION_DEFINITION,operation:r,name:n,variableDefinitions:this.parseVariableDefinitions(),directives:this.parseDirectives(!1),selectionSet:this.parseSelectionSet()})}parseOperationType(){let t=this.expectToken(O.TokenKind.NAME);switch(t.value){case"query":return Ur.OperationTypeNode.QUERY;case"mutation":return Ur.OperationTypeNode.MUTATION;case"subscription":return Ur.OperationTypeNode.SUBSCRIPTION}throw this.unexpected(t)}parseVariableDefinitions(){return this.optionalMany(O.TokenKind.PAREN_L,this.parseVariableDefinition,O.TokenKind.PAREN_R)}parseVariableDefinition(){return this.node(this._lexer.token,{kind:M.Kind.VARIABLE_DEFINITION,variable:this.parseVariable(),type:(this.expectToken(O.TokenKind.COLON),this.parseTypeReference()),defaultValue:this.expectOptionalToken(O.TokenKind.EQUALS)?this.parseConstValueLiteral():void 0,directives:this.parseConstDirectives()})}parseVariable(){let t=this._lexer.token;return this.expectToken(O.TokenKind.DOLLAR),this.node(t,{kind:M.Kind.VARIABLE,name:this.parseName()})}parseSelectionSet(){return this.node(this._lexer.token,{kind:M.Kind.SELECTION_SET,selections:this.many(O.TokenKind.BRACE_L,this.parseSelection,O.TokenKind.BRACE_R)})}parseSelection(){return this.peek(O.TokenKind.SPREAD)?this.parseFragment():this.parseField()}parseField(){let t=this._lexer.token,r=this.parseName(),n,i;return this.expectOptionalToken(O.TokenKind.COLON)?(n=r,i=this.parseName()):i=r,this.node(t,{kind:M.Kind.FIELD,alias:n,name:i,arguments:this.parseArguments(!1),directives:this.parseDirectives(!1),selectionSet:this.peek(O.TokenKind.BRACE_L)?this.parseSelectionSet():void 0})}parseArguments(t){let r=t?this.parseConstArgument:this.parseArgument;return this.optionalMany(O.TokenKind.PAREN_L,r,O.TokenKind.PAREN_R)}parseArgument(t=!1){let r=this._lexer.token,n=this.parseName();return this.expectToken(O.TokenKind.COLON),this.node(r,{kind:M.Kind.ARGUMENT,name:n,value:this.parseValueLiteral(t)})}parseConstArgument(){return this.parseArgument(!0)}parseFragment(){let t=this._lexer.token;this.expectToken(O.TokenKind.SPREAD);let r=this.expectOptionalKeyword("on");return!r&&this.peek(O.TokenKind.NAME)?this.node(t,{kind:M.Kind.FRAGMENT_SPREAD,name:this.parseFragmentName(),directives:this.parseDirectives(!1)}):this.node(t,{kind:M.Kind.INLINE_FRAGMENT,typeCondition:r?this.parseNamedType():void 0,directives:this.parseDirectives(!1),selectionSet:this.parseSelectionSet()})}parseFragmentDefinition(){let t=this._lexer.token;return this.expectKeyword("fragment"),this._options.allowLegacyFragmentVariables===!0?this.node(t,{kind:M.Kind.FRAGMENT_DEFINITION,name:this.parseFragmentName(),variableDefinitions:this.parseVariableDefinitions(),typeCondition:(this.expectKeyword("on"),this.parseNamedType()),directives:this.parseDirectives(!1),selectionSet:this.parseSelectionSet()}):this.node(t,{kind:M.Kind.FRAGMENT_DEFINITION,name:this.parseFragmentName(),typeCondition:(this.expectKeyword("on"),this.parseNamedType()),directives:this.parseDirectives(!1),selectionSet:this.parseSelectionSet()})}parseFragmentName(){if(this._lexer.token.value==="on")throw this.unexpected();return this.parseName()}parseValueLiteral(t){let r=this._lexer.token;switch(r.kind){case O.TokenKind.BRACKET_L:return this.parseList(t);case O.TokenKind.BRACE_L:return this.parseObject(t);case O.TokenKind.INT:return this.advanceLexer(),this.node(r,{kind:M.Kind.INT,value:r.value});case O.TokenKind.FLOAT:return this.advanceLexer(),this.node(r,{kind:M.Kind.FLOAT,value:r.value});case O.TokenKind.STRING:case O.TokenKind.BLOCK_STRING:return this.parseStringLiteral();case O.TokenKind.NAME:switch(this.advanceLexer(),r.value){case"true":return this.node(r,{kind:M.Kind.BOOLEAN,value:!0});case"false":return this.node(r,{kind:M.Kind.BOOLEAN,value:!1});case"null":return this.node(r,{kind:M.Kind.NULL});default:return this.node(r,{kind:M.Kind.ENUM,value:r.value})}case O.TokenKind.DOLLAR:if(t)if(this.expectToken(O.TokenKind.DOLLAR),this._lexer.token.kind===O.TokenKind.NAME){let n=this._lexer.token.value;throw(0,$t.syntaxError)(this._lexer.source,r.start,`Unexpected variable "$${n}" in constant value.`)}else throw this.unexpected(r);return this.parseVariable();default:throw this.unexpected()}}parseConstValueLiteral(){return this.parseValueLiteral(!0)}parseStringLiteral(){let t=this._lexer.token;return this.advanceLexer(),this.node(t,{kind:M.Kind.STRING,value:t.value,block:t.kind===O.TokenKind.BLOCK_STRING})}parseList(t){let r=()=>this.parseValueLiteral(t);return this.node(this._lexer.token,{kind:M.Kind.LIST,values:this.any(O.TokenKind.BRACKET_L,r,O.TokenKind.BRACKET_R)})}parseObject(t){let r=()=>this.parseObjectField(t);return this.node(this._lexer.token,{kind:M.Kind.OBJECT,fields:this.any(O.TokenKind.BRACE_L,r,O.TokenKind.BRACE_R)})}parseObjectField(t){let r=this._lexer.token,n=this.parseName();return this.expectToken(O.TokenKind.COLON),this.node(r,{kind:M.Kind.OBJECT_FIELD,name:n,value:this.parseValueLiteral(t)})}parseDirectives(t){let r=[];for(;this.peek(O.TokenKind.AT);)r.push(this.parseDirective(t));return r}parseConstDirectives(){return this.parseDirectives(!0)}parseDirective(t){let r=this._lexer.token;return this.expectToken(O.TokenKind.AT),this.node(r,{kind:M.Kind.DIRECTIVE,name:this.parseName(),arguments:this.parseArguments(t)})}parseTypeReference(){let t=this._lexer.token,r;if(this.expectOptionalToken(O.TokenKind.BRACKET_L)){let n=this.parseTypeReference();this.expectToken(O.TokenKind.BRACKET_R),r=this.node(t,{kind:M.Kind.LIST_TYPE,type:n})}else r=this.parseNamedType();return this.expectOptionalToken(O.TokenKind.BANG)?this.node(t,{kind:M.Kind.NON_NULL_TYPE,type:r}):r}parseNamedType(){return this.node(this._lexer.token,{kind:M.Kind.NAMED_TYPE,name:this.parseName()})}peekDescription(){return this.peek(O.TokenKind.STRING)||this.peek(O.TokenKind.BLOCK_STRING)}parseDescription(){if(this.peekDescription())return this.parseStringLiteral()}parseSchemaDefinition(){let t=this._lexer.token,r=this.parseDescription();this.expectKeyword("schema");let n=this.parseConstDirectives(),i=this.many(O.TokenKind.BRACE_L,this.parseOperationTypeDefinition,O.TokenKind.BRACE_R);return this.node(t,{kind:M.Kind.SCHEMA_DEFINITION,description:r,directives:n,operationTypes:i})}parseOperationTypeDefinition(){let t=this._lexer.token,r=this.parseOperationType();this.expectToken(O.TokenKind.COLON);let n=this.parseNamedType();return this.node(t,{kind:M.Kind.OPERATION_TYPE_DEFINITION,operation:r,type:n})}parseScalarTypeDefinition(){let t=this._lexer.token,r=this.parseDescription();this.expectKeyword("scalar");let n=this.parseName(),i=this.parseConstDirectives();return this.node(t,{kind:M.Kind.SCALAR_TYPE_DEFINITION,description:r,name:n,directives:i})}parseObjectTypeDefinition(){let t=this._lexer.token,r=this.parseDescription();this.expectKeyword("type");let n=this.parseName(),i=this.parseImplementsInterfaces(),s=this.parseConstDirectives(),a=this.parseFieldsDefinition();return this.node(t,{kind:M.Kind.OBJECT_TYPE_DEFINITION,description:r,name:n,interfaces:i,directives:s,fields:a})}parseImplementsInterfaces(){return this.expectOptionalKeyword("implements")?this.delimitedMany(O.TokenKind.AMP,this.parseNamedType):[]}parseFieldsDefinition(){return this.optionalMany(O.TokenKind.BRACE_L,this.parseFieldDefinition,O.TokenKind.BRACE_R)}parseFieldDefinition(){let t=this._lexer.token,r=this.parseDescription(),n=this.parseName(),i=this.parseArgumentDefs();this.expectToken(O.TokenKind.COLON);let s=this.parseTypeReference(),a=this.parseConstDirectives();return this.node(t,{kind:M.Kind.FIELD_DEFINITION,description:r,name:n,arguments:i,type:s,directives:a})}parseArgumentDefs(){return this.optionalMany(O.TokenKind.PAREN_L,this.parseInputValueDef,O.TokenKind.PAREN_R)}parseInputValueDef(){let t=this._lexer.token,r=this.parseDescription(),n=this.parseName();this.expectToken(O.TokenKind.COLON);let i=this.parseTypeReference(),s;this.expectOptionalToken(O.TokenKind.EQUALS)&&(s=this.parseConstValueLiteral());let a=this.parseConstDirectives();return this.node(t,{kind:M.Kind.INPUT_VALUE_DEFINITION,description:r,name:n,type:i,defaultValue:s,directives:a})}parseInterfaceTypeDefinition(){let t=this._lexer.token,r=this.parseDescription();this.expectKeyword("interface");let n=this.parseName(),i=this.parseImplementsInterfaces(),s=this.parseConstDirectives(),a=this.parseFieldsDefinition();return this.node(t,{kind:M.Kind.INTERFACE_TYPE_DEFINITION,description:r,name:n,interfaces:i,directives:s,fields:a})}parseUnionTypeDefinition(){let t=this._lexer.token,r=this.parseDescription();this.expectKeyword("union");let n=this.parseName(),i=this.parseConstDirectives(),s=this.parseUnionMemberTypes();return this.node(t,{kind:M.Kind.UNION_TYPE_DEFINITION,description:r,name:n,directives:i,types:s})}parseUnionMemberTypes(){return this.expectOptionalToken(O.TokenKind.EQUALS)?this.delimitedMany(O.TokenKind.PIPE,this.parseNamedType):[]}parseEnumTypeDefinition(){let t=this._lexer.token,r=this.parseDescription();this.expectKeyword("enum");let n=this.parseName(),i=this.parseConstDirectives(),s=this.parseEnumValuesDefinition();return this.node(t,{kind:M.Kind.ENUM_TYPE_DEFINITION,description:r,name:n,directives:i,values:s})}parseEnumValuesDefinition(){return this.optionalMany(O.TokenKind.BRACE_L,this.parseEnumValueDefinition,O.TokenKind.BRACE_R)}parseEnumValueDefinition(){let t=this._lexer.token,r=this.parseDescription(),n=this.parseEnumValueName(),i=this.parseConstDirectives();return this.node(t,{kind:M.Kind.ENUM_VALUE_DEFINITION,description:r,name:n,directives:i})}parseEnumValueName(){if(this._lexer.token.value==="true"||this._lexer.token.value==="false"||this._lexer.token.value==="null")throw(0,$t.syntaxError)(this._lexer.source,this._lexer.token.start,`${Kn(this._lexer.token)} is reserved and cannot be used for an enum value.`);return this.parseName()}parseInputObjectTypeDefinition(){let t=this._lexer.token,r=this.parseDescription();this.expectKeyword("input");let n=this.parseName(),i=this.parseConstDirectives(),s=this.parseInputFieldsDefinition();return this.node(t,{kind:M.Kind.INPUT_OBJECT_TYPE_DEFINITION,description:r,name:n,directives:i,fields:s})}parseInputFieldsDefinition(){return this.optionalMany(O.TokenKind.BRACE_L,this.parseInputValueDef,O.TokenKind.BRACE_R)}parseTypeSystemExtension(){let t=this._lexer.lookahead();if(t.kind===O.TokenKind.NAME)switch(t.value){case"schema":return this.parseSchemaExtension();case"scalar":return this.parseScalarTypeExtension();case"type":return this.parseObjectTypeExtension();case"interface":return this.parseInterfaceTypeExtension();case"union":return this.parseUnionTypeExtension();case"enum":return this.parseEnumTypeExtension();case"input":return this.parseInputObjectTypeExtension()}throw this.unexpected(t)}parseSchemaExtension(){let t=this._lexer.token;this.expectKeyword("extend"),this.expectKeyword("schema");let r=this.parseConstDirectives(),n=this.optionalMany(O.TokenKind.BRACE_L,this.parseOperationTypeDefinition,O.TokenKind.BRACE_R);if(r.length===0&&n.length===0)throw this.unexpected();return this.node(t,{kind:M.Kind.SCHEMA_EXTENSION,directives:r,operationTypes:n})}parseScalarTypeExtension(){let t=this._lexer.token;this.expectKeyword("extend"),this.expectKeyword("scalar");let r=this.parseName(),n=this.parseConstDirectives();if(n.length===0)throw this.unexpected();return this.node(t,{kind:M.Kind.SCALAR_TYPE_EXTENSION,name:r,directives:n})}parseObjectTypeExtension(){let t=this._lexer.token;this.expectKeyword("extend"),this.expectKeyword("type");let r=this.parseName(),n=this.parseImplementsInterfaces(),i=this.parseConstDirectives(),s=this.parseFieldsDefinition();if(n.length===0&&i.length===0&&s.length===0)throw this.unexpected();return this.node(t,{kind:M.Kind.OBJECT_TYPE_EXTENSION,name:r,interfaces:n,directives:i,fields:s})}parseInterfaceTypeExtension(){let t=this._lexer.token;this.expectKeyword("extend"),this.expectKeyword("interface");let r=this.parseName(),n=this.parseImplementsInterfaces(),i=this.parseConstDirectives(),s=this.parseFieldsDefinition();if(n.length===0&&i.length===0&&s.length===0)throw this.unexpected();return this.node(t,{kind:M.Kind.INTERFACE_TYPE_EXTENSION,name:r,interfaces:n,directives:i,fields:s})}parseUnionTypeExtension(){let t=this._lexer.token;this.expectKeyword("extend"),this.expectKeyword("union");let r=this.parseName(),n=this.parseConstDirectives(),i=this.parseUnionMemberTypes();if(n.length===0&&i.length===0)throw this.unexpected();return this.node(t,{kind:M.Kind.UNION_TYPE_EXTENSION,name:r,directives:n,types:i})}parseEnumTypeExtension(){let t=this._lexer.token;this.expectKeyword("extend"),this.expectKeyword("enum");let r=this.parseName(),n=this.parseConstDirectives(),i=this.parseEnumValuesDefinition();if(n.length===0&&i.length===0)throw this.unexpected();return this.node(t,{kind:M.Kind.ENUM_TYPE_EXTENSION,name:r,directives:n,values:i})}parseInputObjectTypeExtension(){let t=this._lexer.token;this.expectKeyword("extend"),this.expectKeyword("input");let r=this.parseName(),n=this.parseConstDirectives(),i=this.parseInputFieldsDefinition();if(n.length===0&&i.length===0)throw this.unexpected();return this.node(t,{kind:M.Kind.INPUT_OBJECT_TYPE_EXTENSION,name:r,directives:n,fields:i})}parseDirectiveDefinition(){let t=this._lexer.token,r=this.parseDescription();this.expectKeyword("directive"),this.expectToken(O.TokenKind.AT);let n=this.parseName(),i=this.parseArgumentDefs(),s=this.expectOptionalKeyword("repeatable");this.expectKeyword("on");let a=this.parseDirectiveLocations();return this.node(t,{kind:M.Kind.DIRECTIVE_DEFINITION,description:r,name:n,arguments:i,repeatable:s,locations:a})}parseDirectiveLocations(){return this.delimitedMany(O.TokenKind.PIPE,this.parseDirectiveLocation)}parseDirectiveLocation(){let t=this._lexer.token,r=this.parseName();if(Object.prototype.hasOwnProperty.call(rm.DirectiveLocation,r.value))return r;throw this.unexpected(t)}node(t,r){return this._options.noLocation!==!0&&(r.loc=new Ur.Location(t,this._lexer.lastToken,this._lexer.source)),r}peek(t){return this._lexer.token.kind===t}expectToken(t){let r=this._lexer.token;if(r.kind===t)return this.advanceLexer(),r;throw(0,$t.syntaxError)(this._lexer.source,r.start,`Expected ${tc(t)}, found ${Kn(r)}.`)}expectOptionalToken(t){return this._lexer.token.kind===t?(this.advanceLexer(),!0):!1}expectKeyword(t){let r=this._lexer.token;if(r.kind===O.TokenKind.NAME&&r.value===t)this.advanceLexer();else throw(0,$t.syntaxError)(this._lexer.source,r.start,`Expected "${t}", found ${Kn(r)}.`)}expectOptionalKeyword(t){let r=this._lexer.token;return r.kind===O.TokenKind.NAME&&r.value===t?(this.advanceLexer(),!0):!1}unexpected(t){let r=t??this._lexer.token;return(0,$t.syntaxError)(this._lexer.source,r.start,`Unexpected ${Kn(r)}.`)}any(t,r,n){this.expectToken(t);let i=[];for(;!this.expectOptionalToken(n);)i.push(r.call(this));return i}optionalMany(t,r,n){if(this.expectOptionalToken(t)){let i=[];do i.push(r.call(this));while(!this.expectOptionalToken(n));return i}return[]}many(t,r,n){this.expectToken(t);let i=[];do i.push(r.call(this));while(!this.expectOptionalToken(n));return i}delimitedMany(t,r){this.expectOptionalToken(t);let n=[];do n.push(r.call(this));while(this.expectOptionalToken(t));return n}advanceLexer(){let{maxTokens:t}=this._options,r=this._lexer.advance();if(t!==void 0&&r.kind!==O.TokenKind.EOF&&(++this._tokenCounter,this._tokenCounter>t))throw(0,$t.syntaxError)(this._lexer.source,r.start,`Document contains more that ${t} tokens. Parsing aborted.`)}};Pt.Parser=Bt;function Kn(e){let t=e.value;return tc(e.kind)+(t!=null?` "${t}"`:"")}function tc(e){return(0,ec.isPunctuatorTokenKind)(e)?`"${e}"`:e}});var wt=h(_s=>{"use strict";Object.defineProperty(_s,"__esModule",{value:!0});_s.didYouMean=um;var om=5;function um(e,t){let[r,n]=t?[e,t]:[void 0,e],i=" Did you mean ";r&&(i+=r+" ");let s=n.map(u=>`"${u}"`);switch(s.length){case 0:return"";case 1:return i+s[0]+"?";case 2:return i+s[0]+" or "+s[1]+"?"}let a=s.slice(0,om),o=a.pop();return i+a.join(", ")+", or "+o+"?"}});var rc=h(Ns=>{"use strict";Object.defineProperty(Ns,"__esModule",{value:!0});Ns.identityFunc=cm;function cm(e){return e}});var Ft=h(Os=>{"use strict";Object.defineProperty(Os,"__esModule",{value:!0});Os.keyMap=lm;function lm(e,t){let r=Object.create(null);for(let n of e)r[t(n)]=n;return r}});var Kr=h(Is=>{"use strict";Object.defineProperty(Is,"__esModule",{value:!0});Is.keyValMap=pm;function pm(e,t,r){let n=Object.create(null);for(let i of e)n[t(i)]=r(i);return n}});var Ss=h(Ls=>{"use strict";Object.defineProperty(Ls,"__esModule",{value:!0});Ls.mapValue=dm;function dm(e,t){let r=Object.create(null);for(let n of Object.keys(e))r[n]=t(e[n],n);return r}});var $r=h(As=>{"use strict";Object.defineProperty(As,"__esModule",{value:!0});As.naturalCompare=fm;function fm(e,t){let r=0,n=0;for(;r0);let o=0;do++n,o=o*10+s-Ds,s=t.charCodeAt(n);while($n(s)&&o>0);if(ao)return 1}else{if(is)return 1;++r,++n}}return e.length-t.length}var Ds=48,mm=57;function $n(e){return!isNaN(e)&&Ds<=e&&e<=mm}});var kt=h(js=>{"use strict";Object.defineProperty(js,"__esModule",{value:!0});js.suggestionList=hm;var ym=$r();function hm(e,t){let r=Object.create(null),n=new Rs(e),i=Math.floor(e.length*.4)+1;for(let s of t){let a=n.measure(s,i);a!==void 0&&(r[s]=a)}return Object.keys(r).sort((s,a)=>{let o=r[s]-r[a];return o!==0?o:(0,ym.naturalCompare)(s,a)})}var Rs=class{constructor(t){this._input=t,this._inputLowerCase=t.toLowerCase(),this._inputArray=nc(this._inputLowerCase),this._rows=[new Array(t.length+1).fill(0),new Array(t.length+1).fill(0),new Array(t.length+1).fill(0)]}measure(t,r){if(this._input===t)return 0;let n=t.toLowerCase();if(this._inputLowerCase===n)return 1;let i=nc(n),s=this._inputArray;if(i.lengthr)return;let u=this._rows;for(let l=0;l<=o;l++)u[0][l]=l;for(let l=1;l<=a;l++){let f=u[(l-1)%3],d=u[l%3],y=d[0]=l;for(let T=1;T<=o;T++){let I=i[l-1]===s[T-1]?0:1,E=Math.min(f[T]+1,d[T-1]+1,f[T-1]+I);if(l>1&&T>1&&i[l-1]===s[T-2]&&i[l-2]===s[T-1]){let L=u[(l-2)%3][T-2];E=Math.min(E,L+1)}Er)return}let c=u[a%3][o];return c<=r?c:void 0}};function nc(e){let t=e.length,r=new Array(t);for(let n=0;n{"use strict";Object.defineProperty(Ps,"__esModule",{value:!0});Ps.toObjMap=vm;function vm(e){if(e==null)return Object.create(null);if(Object.getPrototypeOf(e)===null)return e;let t=Object.create(null);for(let[r,n]of Object.entries(e))t[r]=n;return t}});var ic=h(ws=>{"use strict";Object.defineProperty(ws,"__esModule",{value:!0});ws.printString=Tm;function Tm(e){return`"${e.replace(bm,gm)}"`}var bm=/[\x00-\x1f\x22\x5c\x7f-\x9f]/g;function gm(e){return Em[e.charCodeAt(0)]}var Em=["\\u0000","\\u0001","\\u0002","\\u0003","\\u0004","\\u0005","\\u0006","\\u0007","\\b","\\t","\\n","\\u000B","\\f","\\r","\\u000E","\\u000F","\\u0010","\\u0011","\\u0012","\\u0013","\\u0014","\\u0015","\\u0016","\\u0017","\\u0018","\\u0019","\\u001A","\\u001B","\\u001C","\\u001D","\\u001E","\\u001F","","",'\\"',"","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","\\\\","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","","\\u007F","\\u0080","\\u0081","\\u0082","\\u0083","\\u0084","\\u0085","\\u0086","\\u0087","\\u0088","\\u0089","\\u008A","\\u008B","\\u008C","\\u008D","\\u008E","\\u008F","\\u0090","\\u0091","\\u0092","\\u0093","\\u0094","\\u0095","\\u0096","\\u0097","\\u0098","\\u0099","\\u009A","\\u009B","\\u009C","\\u009D","\\u009E","\\u009F"]});var Yt=h(qt=>{"use strict";Object.defineProperty(qt,"__esModule",{value:!0});qt.BREAK=void 0;qt.getEnterLeaveForKind=Yn;qt.getVisitFn=Lm;qt.visit=Om;qt.visitInParallel=Im;var _m=Fe(),Nm=re(),Fs=dt(),sc=H(),Tr=Object.freeze({});qt.BREAK=Tr;function Om(e,t,r=Fs.QueryDocumentKeys){let n=new Map;for(let L of Object.values(sc.Kind))n.set(L,Yn(t,L));let i,s=Array.isArray(e),a=[e],o=-1,u=[],c=e,l,f,d=[],y=[];do{o++;let L=o===a.length,F=L&&u.length!==0;if(L){if(l=y.length===0?void 0:d[d.length-1],c=f,f=y.pop(),F)if(s){c=c.slice();let P=0;for(let[G,C]of u){let w=G-P;C===null?(c.splice(w,1),P++):c[w]=C}}else{c=Object.defineProperties({},Object.getOwnPropertyDescriptors(c));for(let[P,G]of u)c[P]=G}o=i.index,a=i.keys,u=i.edits,s=i.inArray,i=i.prev}else if(f){if(l=s?o:a[o],c=f[l],c==null)continue;d.push(l)}let k;if(!Array.isArray(c)){var T,I;(0,Fs.isNode)(c)||(0,_m.devAssert)(!1,`Invalid AST Node: ${(0,Nm.inspect)(c)}.`);let P=L?(T=n.get(c.kind))===null||T===void 0?void 0:T.leave:(I=n.get(c.kind))===null||I===void 0?void 0:I.enter;if(k=P?.call(t,c,l,f,d,y),k===Tr)break;if(k===!1){if(!L){d.pop();continue}}else if(k!==void 0&&(u.push([l,k]),!L))if((0,Fs.isNode)(k))c=k;else{d.pop();continue}}if(k===void 0&&F&&u.push([l,c]),L)d.pop();else{var E;i={inArray:s,index:o,keys:a,edits:u,prev:i},s=Array.isArray(c),a=s?c:(E=r[c.kind])!==null&&E!==void 0?E:[],o=-1,u=[],f&&y.push(f),f=c}}while(i!==void 0);return u.length!==0?u[u.length-1][1]:e}function Im(e){let t=new Array(e.length).fill(null),r=Object.create(null);for(let n of Object.values(sc.Kind)){let i=!1,s=new Array(e.length).fill(void 0),a=new Array(e.length).fill(void 0);for(let u=0;u{"use strict";Object.defineProperty(ks,"__esModule",{value:!0});ks.print=Rm;var Sm=qr(),Dm=ic(),Am=Yt();function Rm(e){return(0,Am.visit)(e,Pm)}var jm=80,Pm={Name:{leave:e=>e.value},Variable:{leave:e=>"$"+e.name},Document:{leave:e=>A(e.definitions,` + +`)},OperationDefinition:{leave(e){let t=B("(",A(e.variableDefinitions,", "),")"),r=A([e.operation,A([e.name,t]),A(e.directives," ")]," ");return(r==="query"?"":r+" ")+e.selectionSet}},VariableDefinition:{leave:({variable:e,type:t,defaultValue:r,directives:n})=>e+": "+t+B(" = ",r)+B(" ",A(n," "))},SelectionSet:{leave:({selections:e})=>nt(e)},Field:{leave({alias:e,name:t,arguments:r,directives:n,selectionSet:i}){let s=B("",e,": ")+t,a=s+B("(",A(r,", "),")");return a.length>jm&&(a=s+B(`( +`,Jn(A(r,` +`)),` +)`)),A([a,A(n," "),i]," ")}},Argument:{leave:({name:e,value:t})=>e+": "+t},FragmentSpread:{leave:({name:e,directives:t})=>"..."+e+B(" ",A(t," "))},InlineFragment:{leave:({typeCondition:e,directives:t,selectionSet:r})=>A(["...",B("on ",e),A(t," "),r]," ")},FragmentDefinition:{leave:({name:e,typeCondition:t,variableDefinitions:r,directives:n,selectionSet:i})=>`fragment ${e}${B("(",A(r,", "),")")} on ${t} ${B("",A(n," ")," ")}`+i},IntValue:{leave:({value:e})=>e},FloatValue:{leave:({value:e})=>e},StringValue:{leave:({value:e,block:t})=>t?(0,Sm.printBlockString)(e):(0,Dm.printString)(e)},BooleanValue:{leave:({value:e})=>e?"true":"false"},NullValue:{leave:()=>"null"},EnumValue:{leave:({value:e})=>e},ListValue:{leave:({values:e})=>"["+A(e,", ")+"]"},ObjectValue:{leave:({fields:e})=>"{"+A(e,", ")+"}"},ObjectField:{leave:({name:e,value:t})=>e+": "+t},Directive:{leave:({name:e,arguments:t})=>"@"+e+B("(",A(t,", "),")")},NamedType:{leave:({name:e})=>e},ListType:{leave:({type:e})=>"["+e+"]"},NonNullType:{leave:({type:e})=>e+"!"},SchemaDefinition:{leave:({description:e,directives:t,operationTypes:r})=>B("",e,` +`)+A(["schema",A(t," "),nt(r)]," ")},OperationTypeDefinition:{leave:({operation:e,type:t})=>e+": "+t},ScalarTypeDefinition:{leave:({description:e,name:t,directives:r})=>B("",e,` +`)+A(["scalar",t,A(r," ")]," ")},ObjectTypeDefinition:{leave:({description:e,name:t,interfaces:r,directives:n,fields:i})=>B("",e,` +`)+A(["type",t,B("implements ",A(r," & ")),A(n," "),nt(i)]," ")},FieldDefinition:{leave:({description:e,name:t,arguments:r,type:n,directives:i})=>B("",e,` +`)+t+(ac(r)?B(`( +`,Jn(A(r,` +`)),` +)`):B("(",A(r,", "),")"))+": "+n+B(" ",A(i," "))},InputValueDefinition:{leave:({description:e,name:t,type:r,defaultValue:n,directives:i})=>B("",e,` +`)+A([t+": "+r,B("= ",n),A(i," ")]," ")},InterfaceTypeDefinition:{leave:({description:e,name:t,interfaces:r,directives:n,fields:i})=>B("",e,` +`)+A(["interface",t,B("implements ",A(r," & ")),A(n," "),nt(i)]," ")},UnionTypeDefinition:{leave:({description:e,name:t,directives:r,types:n})=>B("",e,` +`)+A(["union",t,A(r," "),B("= ",A(n," | "))]," ")},EnumTypeDefinition:{leave:({description:e,name:t,directives:r,values:n})=>B("",e,` +`)+A(["enum",t,A(r," "),nt(n)]," ")},EnumValueDefinition:{leave:({description:e,name:t,directives:r})=>B("",e,` +`)+A([t,A(r," ")]," ")},InputObjectTypeDefinition:{leave:({description:e,name:t,directives:r,fields:n})=>B("",e,` +`)+A(["input",t,A(r," "),nt(n)]," ")},DirectiveDefinition:{leave:({description:e,name:t,arguments:r,repeatable:n,locations:i})=>B("",e,` +`)+"directive @"+t+(ac(r)?B(`( +`,Jn(A(r,` +`)),` +)`):B("(",A(r,", "),")"))+(n?" repeatable":"")+" on "+A(i," | ")},SchemaExtension:{leave:({directives:e,operationTypes:t})=>A(["extend schema",A(e," "),nt(t)]," ")},ScalarTypeExtension:{leave:({name:e,directives:t})=>A(["extend scalar",e,A(t," ")]," ")},ObjectTypeExtension:{leave:({name:e,interfaces:t,directives:r,fields:n})=>A(["extend type",e,B("implements ",A(t," & ")),A(r," "),nt(n)]," ")},InterfaceTypeExtension:{leave:({name:e,interfaces:t,directives:r,fields:n})=>A(["extend interface",e,B("implements ",A(t," & ")),A(r," "),nt(n)]," ")},UnionTypeExtension:{leave:({name:e,directives:t,types:r})=>A(["extend union",e,A(t," "),B("= ",A(r," | "))]," ")},EnumTypeExtension:{leave:({name:e,directives:t,values:r})=>A(["extend enum",e,A(t," "),nt(r)]," ")},InputObjectTypeExtension:{leave:({name:e,directives:t,fields:r})=>A(["extend input",e,A(t," "),nt(r)]," ")}};function A(e,t=""){var r;return(r=e?.filter(n=>n).join(t))!==null&&r!==void 0?r:""}function nt(e){return B(`{ +`,Jn(A(e,` +`)),` +}`)}function B(e,t,r=""){return t!=null&&t!==""?e+t+r:""}function Jn(e){return B(" ",e.replace(/\n/g,` + `))}function ac(e){var t;return(t=e?.some(r=>r.includes(` +`)))!==null&&t!==void 0?t:!1}});var Vs=h(xs=>{"use strict";Object.defineProperty(xs,"__esModule",{value:!0});xs.valueFromASTUntyped=qs;var wm=Kr(),Ot=H();function qs(e,t){switch(e.kind){case Ot.Kind.NULL:return null;case Ot.Kind.INT:return parseInt(e.value,10);case Ot.Kind.FLOAT:return parseFloat(e.value);case Ot.Kind.STRING:case Ot.Kind.ENUM:case Ot.Kind.BOOLEAN:return e.value;case Ot.Kind.LIST:return e.values.map(r=>qs(r,t));case Ot.Kind.OBJECT:return(0,wm.keyValMap)(e.fields,r=>r.name.value,r=>qs(r.value,t));case Ot.Kind.VARIABLE:return t?.[e.name.value]}}});var Br=h(Hn=>{"use strict";Object.defineProperty(Hn,"__esModule",{value:!0});Hn.assertEnumValueName=Fm;Hn.assertName=cc;var oc=Fe(),Xn=q(),uc=xn();function cc(e){if(e!=null||(0,oc.devAssert)(!1,"Must provide name."),typeof e=="string"||(0,oc.devAssert)(!1,"Expected name to be a string."),e.length===0)throw new Xn.GraphQLError("Expected name to be a non-empty string.");for(let t=1;t{"use strict";Object.defineProperty(j,"__esModule",{value:!0});j.GraphQLUnionType=j.GraphQLScalarType=j.GraphQLObjectType=j.GraphQLNonNull=j.GraphQLList=j.GraphQLInterfaceType=j.GraphQLInputObjectType=j.GraphQLEnumType=void 0;j.argsToArgsConfig=Ec;j.assertAbstractType=ey;j.assertCompositeType=Zm;j.assertEnumType=Bm;j.assertInputObjectType=Ym;j.assertInputType=Hm;j.assertInterfaceType=Km;j.assertLeafType=Wm;j.assertListType=Jm;j.assertNamedType=iy;j.assertNonNullType=Xm;j.assertNullableType=ry;j.assertObjectType=Um;j.assertOutputType=zm;j.assertScalarType=Qm;j.assertType=Cm;j.assertUnionType=$m;j.assertWrappingType=ty;j.defineArguments=bc;j.getNamedType=sy;j.getNullableType=ny;j.isAbstractType=yc;j.isCompositeType=mc;j.isEnumType=zt;j.isInputObjectType=Jr;j.isInputType=Ms;j.isInterfaceType=Xt;j.isLeafType=fc;j.isListType=ui;j.isNamedType=hc;j.isNonNullType=Vt;j.isNullableType=Cs;j.isObjectType=gr;j.isOutputType=Gs;j.isRequiredArgument=ay;j.isRequiredInputField=ly;j.isScalarType=Jt;j.isType=oi;j.isUnionType=Ht;j.isWrappingType=Xr;j.resolveObjMapThunk=Us;j.resolveReadonlyArrayThunk=Qs;var Se=Fe(),km=wt(),lc=rc(),se=re(),xt=Cr(),qm=pt(),xm=Ft(),dc=Kr(),ai=Ss(),Vm=kt(),ft=Bn(),Yr=q(),Mm=H(),pc=Ce(),Gm=Vs(),mt=Br();function oi(e){return Jt(e)||gr(e)||Xt(e)||Ht(e)||zt(e)||Jr(e)||ui(e)||Vt(e)}function Cm(e){if(!oi(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL type.`);return e}function Jt(e){return(0,xt.instanceOf)(e,ei)}function Qm(e){if(!Jt(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL Scalar type.`);return e}function gr(e){return(0,xt.instanceOf)(e,ti)}function Um(e){if(!gr(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL Object type.`);return e}function Xt(e){return(0,xt.instanceOf)(e,ri)}function Km(e){if(!Xt(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL Interface type.`);return e}function Ht(e){return(0,xt.instanceOf)(e,ni)}function $m(e){if(!Ht(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL Union type.`);return e}function zt(e){return(0,xt.instanceOf)(e,ii)}function Bm(e){if(!zt(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL Enum type.`);return e}function Jr(e){return(0,xt.instanceOf)(e,si)}function Ym(e){if(!Jr(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL Input Object type.`);return e}function ui(e){return(0,xt.instanceOf)(e,Wn)}function Jm(e){if(!ui(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL List type.`);return e}function Vt(e){return(0,xt.instanceOf)(e,Zn)}function Xm(e){if(!Vt(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL Non-Null type.`);return e}function Ms(e){return Jt(e)||zt(e)||Jr(e)||Xr(e)&&Ms(e.ofType)}function Hm(e){if(!Ms(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL input type.`);return e}function Gs(e){return Jt(e)||gr(e)||Xt(e)||Ht(e)||zt(e)||Xr(e)&&Gs(e.ofType)}function zm(e){if(!Gs(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL output type.`);return e}function fc(e){return Jt(e)||zt(e)}function Wm(e){if(!fc(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL leaf type.`);return e}function mc(e){return gr(e)||Xt(e)||Ht(e)}function Zm(e){if(!mc(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL composite type.`);return e}function yc(e){return Xt(e)||Ht(e)}function ey(e){if(!yc(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL abstract type.`);return e}var Wn=class{constructor(t){oi(t)||(0,Se.devAssert)(!1,`Expected ${(0,se.inspect)(t)} to be a GraphQL type.`),this.ofType=t}get[Symbol.toStringTag](){return"GraphQLList"}toString(){return"["+String(this.ofType)+"]"}toJSON(){return this.toString()}};j.GraphQLList=Wn;var Zn=class{constructor(t){Cs(t)||(0,Se.devAssert)(!1,`Expected ${(0,se.inspect)(t)} to be a GraphQL nullable type.`),this.ofType=t}get[Symbol.toStringTag](){return"GraphQLNonNull"}toString(){return String(this.ofType)+"!"}toJSON(){return this.toString()}};j.GraphQLNonNull=Zn;function Xr(e){return ui(e)||Vt(e)}function ty(e){if(!Xr(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL wrapping type.`);return e}function Cs(e){return oi(e)&&!Vt(e)}function ry(e){if(!Cs(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL nullable type.`);return e}function ny(e){if(e)return Vt(e)?e.ofType:e}function hc(e){return Jt(e)||gr(e)||Xt(e)||Ht(e)||zt(e)||Jr(e)}function iy(e){if(!hc(e))throw new Error(`Expected ${(0,se.inspect)(e)} to be a GraphQL named type.`);return e}function sy(e){if(e){let t=e;for(;Xr(t);)t=t.ofType;return t}}function Qs(e){return typeof e=="function"?e():e}function Us(e){return typeof e=="function"?e():e}var ei=class{constructor(t){var r,n,i,s;let a=(r=t.parseValue)!==null&&r!==void 0?r:lc.identityFunc;this.name=(0,mt.assertName)(t.name),this.description=t.description,this.specifiedByURL=t.specifiedByURL,this.serialize=(n=t.serialize)!==null&&n!==void 0?n:lc.identityFunc,this.parseValue=a,this.parseLiteral=(i=t.parseLiteral)!==null&&i!==void 0?i:(o,u)=>a((0,Gm.valueFromASTUntyped)(o,u)),this.extensions=(0,ft.toObjMap)(t.extensions),this.astNode=t.astNode,this.extensionASTNodes=(s=t.extensionASTNodes)!==null&&s!==void 0?s:[],t.specifiedByURL==null||typeof t.specifiedByURL=="string"||(0,Se.devAssert)(!1,`${this.name} must provide "specifiedByURL" as a string, but got: ${(0,se.inspect)(t.specifiedByURL)}.`),t.serialize==null||typeof t.serialize=="function"||(0,Se.devAssert)(!1,`${this.name} must provide "serialize" function. If this custom Scalar is also used as an input type, ensure "parseValue" and "parseLiteral" functions are also provided.`),t.parseLiteral&&(typeof t.parseValue=="function"&&typeof t.parseLiteral=="function"||(0,Se.devAssert)(!1,`${this.name} must provide both "parseValue" and "parseLiteral" functions.`))}get[Symbol.toStringTag](){return"GraphQLScalarType"}toConfig(){return{name:this.name,description:this.description,specifiedByURL:this.specifiedByURL,serialize:this.serialize,parseValue:this.parseValue,parseLiteral:this.parseLiteral,extensions:this.extensions,astNode:this.astNode,extensionASTNodes:this.extensionASTNodes}}toString(){return this.name}toJSON(){return this.toString()}};j.GraphQLScalarType=ei;var ti=class{constructor(t){var r;this.name=(0,mt.assertName)(t.name),this.description=t.description,this.isTypeOf=t.isTypeOf,this.extensions=(0,ft.toObjMap)(t.extensions),this.astNode=t.astNode,this.extensionASTNodes=(r=t.extensionASTNodes)!==null&&r!==void 0?r:[],this._fields=()=>Tc(t),this._interfaces=()=>vc(t),t.isTypeOf==null||typeof t.isTypeOf=="function"||(0,Se.devAssert)(!1,`${this.name} must provide "isTypeOf" as a function, but got: ${(0,se.inspect)(t.isTypeOf)}.`)}get[Symbol.toStringTag](){return"GraphQLObjectType"}getFields(){return typeof this._fields=="function"&&(this._fields=this._fields()),this._fields}getInterfaces(){return typeof this._interfaces=="function"&&(this._interfaces=this._interfaces()),this._interfaces}toConfig(){return{name:this.name,description:this.description,interfaces:this.getInterfaces(),fields:gc(this.getFields()),isTypeOf:this.isTypeOf,extensions:this.extensions,astNode:this.astNode,extensionASTNodes:this.extensionASTNodes}}toString(){return this.name}toJSON(){return this.toString()}};j.GraphQLObjectType=ti;function vc(e){var t;let r=Qs((t=e.interfaces)!==null&&t!==void 0?t:[]);return Array.isArray(r)||(0,Se.devAssert)(!1,`${e.name} interfaces must be an Array or a function which returns an Array.`),r}function Tc(e){let t=Us(e.fields);return br(t)||(0,Se.devAssert)(!1,`${e.name} fields must be an object with field names as keys or a function which returns such an object.`),(0,ai.mapValue)(t,(r,n)=>{var i;br(r)||(0,Se.devAssert)(!1,`${e.name}.${n} field config must be an object.`),r.resolve==null||typeof r.resolve=="function"||(0,Se.devAssert)(!1,`${e.name}.${n} field resolver must be a function if provided, but got: ${(0,se.inspect)(r.resolve)}.`);let s=(i=r.args)!==null&&i!==void 0?i:{};return br(s)||(0,Se.devAssert)(!1,`${e.name}.${n} args must be an object with argument names as keys.`),{name:(0,mt.assertName)(n),description:r.description,type:r.type,args:bc(s),resolve:r.resolve,subscribe:r.subscribe,deprecationReason:r.deprecationReason,extensions:(0,ft.toObjMap)(r.extensions),astNode:r.astNode}})}function bc(e){return Object.entries(e).map(([t,r])=>({name:(0,mt.assertName)(t),description:r.description,type:r.type,defaultValue:r.defaultValue,deprecationReason:r.deprecationReason,extensions:(0,ft.toObjMap)(r.extensions),astNode:r.astNode}))}function br(e){return(0,qm.isObjectLike)(e)&&!Array.isArray(e)}function gc(e){return(0,ai.mapValue)(e,t=>({description:t.description,type:t.type,args:Ec(t.args),resolve:t.resolve,subscribe:t.subscribe,deprecationReason:t.deprecationReason,extensions:t.extensions,astNode:t.astNode}))}function Ec(e){return(0,dc.keyValMap)(e,t=>t.name,t=>({description:t.description,type:t.type,defaultValue:t.defaultValue,deprecationReason:t.deprecationReason,extensions:t.extensions,astNode:t.astNode}))}function ay(e){return Vt(e.type)&&e.defaultValue===void 0}var ri=class{constructor(t){var r;this.name=(0,mt.assertName)(t.name),this.description=t.description,this.resolveType=t.resolveType,this.extensions=(0,ft.toObjMap)(t.extensions),this.astNode=t.astNode,this.extensionASTNodes=(r=t.extensionASTNodes)!==null&&r!==void 0?r:[],this._fields=Tc.bind(void 0,t),this._interfaces=vc.bind(void 0,t),t.resolveType==null||typeof t.resolveType=="function"||(0,Se.devAssert)(!1,`${this.name} must provide "resolveType" as a function, but got: ${(0,se.inspect)(t.resolveType)}.`)}get[Symbol.toStringTag](){return"GraphQLInterfaceType"}getFields(){return typeof this._fields=="function"&&(this._fields=this._fields()),this._fields}getInterfaces(){return typeof this._interfaces=="function"&&(this._interfaces=this._interfaces()),this._interfaces}toConfig(){return{name:this.name,description:this.description,interfaces:this.getInterfaces(),fields:gc(this.getFields()),resolveType:this.resolveType,extensions:this.extensions,astNode:this.astNode,extensionASTNodes:this.extensionASTNodes}}toString(){return this.name}toJSON(){return this.toString()}};j.GraphQLInterfaceType=ri;var ni=class{constructor(t){var r;this.name=(0,mt.assertName)(t.name),this.description=t.description,this.resolveType=t.resolveType,this.extensions=(0,ft.toObjMap)(t.extensions),this.astNode=t.astNode,this.extensionASTNodes=(r=t.extensionASTNodes)!==null&&r!==void 0?r:[],this._types=oy.bind(void 0,t),t.resolveType==null||typeof t.resolveType=="function"||(0,Se.devAssert)(!1,`${this.name} must provide "resolveType" as a function, but got: ${(0,se.inspect)(t.resolveType)}.`)}get[Symbol.toStringTag](){return"GraphQLUnionType"}getTypes(){return typeof this._types=="function"&&(this._types=this._types()),this._types}toConfig(){return{name:this.name,description:this.description,types:this.getTypes(),resolveType:this.resolveType,extensions:this.extensions,astNode:this.astNode,extensionASTNodes:this.extensionASTNodes}}toString(){return this.name}toJSON(){return this.toString()}};j.GraphQLUnionType=ni;function oy(e){let t=Qs(e.types);return Array.isArray(t)||(0,Se.devAssert)(!1,`Must provide Array of types or a function which returns such an array for Union ${e.name}.`),t}var ii=class{constructor(t){var r;this.name=(0,mt.assertName)(t.name),this.description=t.description,this.extensions=(0,ft.toObjMap)(t.extensions),this.astNode=t.astNode,this.extensionASTNodes=(r=t.extensionASTNodes)!==null&&r!==void 0?r:[],this._values=uy(this.name,t.values),this._valueLookup=new Map(this._values.map(n=>[n.value,n])),this._nameLookup=(0,xm.keyMap)(this._values,n=>n.name)}get[Symbol.toStringTag](){return"GraphQLEnumType"}getValues(){return this._values}getValue(t){return this._nameLookup[t]}serialize(t){let r=this._valueLookup.get(t);if(r===void 0)throw new Yr.GraphQLError(`Enum "${this.name}" cannot represent value: ${(0,se.inspect)(t)}`);return r.name}parseValue(t){if(typeof t!="string"){let n=(0,se.inspect)(t);throw new Yr.GraphQLError(`Enum "${this.name}" cannot represent non-string value: ${n}.`+zn(this,n))}let r=this.getValue(t);if(r==null)throw new Yr.GraphQLError(`Value "${t}" does not exist in "${this.name}" enum.`+zn(this,t));return r.value}parseLiteral(t,r){if(t.kind!==Mm.Kind.ENUM){let i=(0,pc.print)(t);throw new Yr.GraphQLError(`Enum "${this.name}" cannot represent non-enum value: ${i}.`+zn(this,i),{nodes:t})}let n=this.getValue(t.value);if(n==null){let i=(0,pc.print)(t);throw new Yr.GraphQLError(`Value "${i}" does not exist in "${this.name}" enum.`+zn(this,i),{nodes:t})}return n.value}toConfig(){let t=(0,dc.keyValMap)(this.getValues(),r=>r.name,r=>({description:r.description,value:r.value,deprecationReason:r.deprecationReason,extensions:r.extensions,astNode:r.astNode}));return{name:this.name,description:this.description,values:t,extensions:this.extensions,astNode:this.astNode,extensionASTNodes:this.extensionASTNodes}}toString(){return this.name}toJSON(){return this.toString()}};j.GraphQLEnumType=ii;function zn(e,t){let r=e.getValues().map(i=>i.name),n=(0,Vm.suggestionList)(t,r);return(0,km.didYouMean)("the enum value",n)}function uy(e,t){return br(t)||(0,Se.devAssert)(!1,`${e} values must be an object with value names as keys.`),Object.entries(t).map(([r,n])=>(br(n)||(0,Se.devAssert)(!1,`${e}.${r} must refer to an object with a "value" key representing an internal value but got: ${(0,se.inspect)(n)}.`),{name:(0,mt.assertEnumValueName)(r),description:n.description,value:n.value!==void 0?n.value:r,deprecationReason:n.deprecationReason,extensions:(0,ft.toObjMap)(n.extensions),astNode:n.astNode}))}var si=class{constructor(t){var r;this.name=(0,mt.assertName)(t.name),this.description=t.description,this.extensions=(0,ft.toObjMap)(t.extensions),this.astNode=t.astNode,this.extensionASTNodes=(r=t.extensionASTNodes)!==null&&r!==void 0?r:[],this._fields=cy.bind(void 0,t)}get[Symbol.toStringTag](){return"GraphQLInputObjectType"}getFields(){return typeof this._fields=="function"&&(this._fields=this._fields()),this._fields}toConfig(){let t=(0,ai.mapValue)(this.getFields(),r=>({description:r.description,type:r.type,defaultValue:r.defaultValue,deprecationReason:r.deprecationReason,extensions:r.extensions,astNode:r.astNode}));return{name:this.name,description:this.description,fields:t,extensions:this.extensions,astNode:this.astNode,extensionASTNodes:this.extensionASTNodes}}toString(){return this.name}toJSON(){return this.toString()}};j.GraphQLInputObjectType=si;function cy(e){let t=Us(e.fields);return br(t)||(0,Se.devAssert)(!1,`${e.name} fields must be an object with field names as keys or a function which returns such an object.`),(0,ai.mapValue)(t,(r,n)=>(!("resolve"in r)||(0,Se.devAssert)(!1,`${e.name}.${n} field has a resolve property, but Input Types cannot define resolvers.`),{name:(0,mt.assertName)(n),description:r.description,type:r.type,defaultValue:r.defaultValue,deprecationReason:r.deprecationReason,extensions:(0,ft.toObjMap)(r.extensions),astNode:r.astNode}))}function ly(e){return Vt(e.type)&&e.defaultValue===void 0}});var zr=h(Hr=>{"use strict";Object.defineProperty(Hr,"__esModule",{value:!0});Hr.doTypesOverlap=py;Hr.isEqualType=Ks;Hr.isTypeSubTypeOf=ci;var Pe=Y();function Ks(e,t){return e===t?!0:(0,Pe.isNonNullType)(e)&&(0,Pe.isNonNullType)(t)||(0,Pe.isListType)(e)&&(0,Pe.isListType)(t)?Ks(e.ofType,t.ofType):!1}function ci(e,t,r){return t===r?!0:(0,Pe.isNonNullType)(r)?(0,Pe.isNonNullType)(t)?ci(e,t.ofType,r.ofType):!1:(0,Pe.isNonNullType)(t)?ci(e,t.ofType,r):(0,Pe.isListType)(r)?(0,Pe.isListType)(t)?ci(e,t.ofType,r.ofType):!1:(0,Pe.isListType)(t)?!1:(0,Pe.isAbstractType)(r)&&((0,Pe.isInterfaceType)(t)||(0,Pe.isObjectType)(t))&&e.isSubType(r,t)}function py(e,t,r){return t===r?!0:(0,Pe.isAbstractType)(t)?(0,Pe.isAbstractType)(r)?e.getPossibleTypes(t).some(n=>e.isSubType(r,n)):e.isSubType(t,r):(0,Pe.isAbstractType)(r)?e.isSubType(r,t):!1}});var yt=h(ge=>{"use strict";Object.defineProperty(ge,"__esModule",{value:!0});ge.GraphQLString=ge.GraphQLInt=ge.GraphQLID=ge.GraphQLFloat=ge.GraphQLBoolean=ge.GRAPHQL_MIN_INT=ge.GRAPHQL_MAX_INT=void 0;ge.isSpecifiedScalarType=dy;ge.specifiedScalarTypes=void 0;var it=re(),_c=pt(),De=q(),Wt=H(),Wr=Ce(),Zr=Y(),li=2147483647;ge.GRAPHQL_MAX_INT=li;var pi=-2147483648;ge.GRAPHQL_MIN_INT=pi;var Nc=new Zr.GraphQLScalarType({name:"Int",description:"The `Int` scalar type represents non-fractional signed whole numeric values. Int can represent values between -(2^31) and 2^31 - 1.",serialize(e){let t=en(e);if(typeof t=="boolean")return t?1:0;let r=t;if(typeof t=="string"&&t!==""&&(r=Number(t)),typeof r!="number"||!Number.isInteger(r))throw new De.GraphQLError(`Int cannot represent non-integer value: ${(0,it.inspect)(t)}`);if(r>li||rli||eli||te.name===t)}function en(e){if((0,_c.isObjectLike)(e)){if(typeof e.valueOf=="function"){let t=e.valueOf();if(!(0,_c.isObjectLike)(t))return t}if(typeof e.toJSON=="function")return e.toJSON()}return e}});var Ve=h(Ee=>{"use strict";Object.defineProperty(Ee,"__esModule",{value:!0});Ee.GraphQLSpecifiedByDirective=Ee.GraphQLSkipDirective=Ee.GraphQLIncludeDirective=Ee.GraphQLDirective=Ee.GraphQLDeprecatedDirective=Ee.DEFAULT_DEPRECATION_REASON=void 0;Ee.assertDirective=Ty;Ee.isDirective=Rc;Ee.isSpecifiedDirective=by;Ee.specifiedDirectives=void 0;var Ac=Fe(),fy=re(),my=Cr(),yy=pt(),hy=Bn(),st=yr(),vy=Br(),tn=Y(),di=yt();function Rc(e){return(0,my.instanceOf)(e,Mt)}function Ty(e){if(!Rc(e))throw new Error(`Expected ${(0,fy.inspect)(e)} to be a GraphQL directive.`);return e}var Mt=class{constructor(t){var r,n;this.name=(0,vy.assertName)(t.name),this.description=t.description,this.locations=t.locations,this.isRepeatable=(r=t.isRepeatable)!==null&&r!==void 0?r:!1,this.extensions=(0,hy.toObjMap)(t.extensions),this.astNode=t.astNode,Array.isArray(t.locations)||(0,Ac.devAssert)(!1,`@${t.name} locations must be an Array.`);let i=(n=t.args)!==null&&n!==void 0?n:{};(0,yy.isObjectLike)(i)&&!Array.isArray(i)||(0,Ac.devAssert)(!1,`@${t.name} args must be an object with argument names as keys.`),this.args=(0,tn.defineArguments)(i)}get[Symbol.toStringTag](){return"GraphQLDirective"}toConfig(){return{name:this.name,description:this.description,locations:this.locations,args:(0,tn.argsToArgsConfig)(this.args),isRepeatable:this.isRepeatable,extensions:this.extensions,astNode:this.astNode}}toString(){return"@"+this.name}toJSON(){return this.toString()}};Ee.GraphQLDirective=Mt;var jc=new Mt({name:"include",description:"Directs the executor to include this field or fragment only when the `if` argument is true.",locations:[st.DirectiveLocation.FIELD,st.DirectiveLocation.FRAGMENT_SPREAD,st.DirectiveLocation.INLINE_FRAGMENT],args:{if:{type:new tn.GraphQLNonNull(di.GraphQLBoolean),description:"Included when true."}}});Ee.GraphQLIncludeDirective=jc;var Pc=new Mt({name:"skip",description:"Directs the executor to skip this field or fragment when the `if` argument is true.",locations:[st.DirectiveLocation.FIELD,st.DirectiveLocation.FRAGMENT_SPREAD,st.DirectiveLocation.INLINE_FRAGMENT],args:{if:{type:new tn.GraphQLNonNull(di.GraphQLBoolean),description:"Skipped when true."}}});Ee.GraphQLSkipDirective=Pc;var wc="No longer supported";Ee.DEFAULT_DEPRECATION_REASON=wc;var Fc=new Mt({name:"deprecated",description:"Marks an element of a GraphQL schema as no longer supported.",locations:[st.DirectiveLocation.FIELD_DEFINITION,st.DirectiveLocation.ARGUMENT_DEFINITION,st.DirectiveLocation.INPUT_FIELD_DEFINITION,st.DirectiveLocation.ENUM_VALUE],args:{reason:{type:di.GraphQLString,description:"Explains why this element was deprecated, usually also including a suggestion for how to access supported similar data. Formatted using the Markdown syntax, as specified by [CommonMark](https://commonmark.org/).",defaultValue:wc}}});Ee.GraphQLDeprecatedDirective=Fc;var kc=new Mt({name:"specifiedBy",description:"Exposes a URL that specifies the behavior of this scalar.",locations:[st.DirectiveLocation.SCALAR],args:{url:{type:new tn.GraphQLNonNull(di.GraphQLString),description:"The URL that specifies the behavior of this scalar."}}});Ee.GraphQLSpecifiedByDirective=kc;var qc=Object.freeze([jc,Pc,Fc,kc]);Ee.specifiedDirectives=qc;function by(e){return qc.some(({name:t})=>t===e.name)}});var fi=h($s=>{"use strict";Object.defineProperty($s,"__esModule",{value:!0});$s.isIterableObject=gy;function gy(e){return typeof e=="object"&&typeof e?.[Symbol.iterator]=="function"}});var sn=h(Bs=>{"use strict";Object.defineProperty(Bs,"__esModule",{value:!0});Bs.astFromValue=nn;var xc=re(),Ey=je(),_y=fi(),Ny=pt(),$e=H(),rn=Y(),Oy=yt();function nn(e,t){if((0,rn.isNonNullType)(t)){let r=nn(e,t.ofType);return r?.kind===$e.Kind.NULL?null:r}if(e===null)return{kind:$e.Kind.NULL};if(e===void 0)return null;if((0,rn.isListType)(t)){let r=t.ofType;if((0,_y.isIterableObject)(e)){let n=[];for(let i of e){let s=nn(i,r);s!=null&&n.push(s)}return{kind:$e.Kind.LIST,values:n}}return nn(e,r)}if((0,rn.isInputObjectType)(t)){if(!(0,Ny.isObjectLike)(e))return null;let r=[];for(let n of Object.values(t.getFields())){let i=nn(e[n.name],n.type);i&&r.push({kind:$e.Kind.OBJECT_FIELD,name:{kind:$e.Kind.NAME,value:n.name},value:i})}return{kind:$e.Kind.OBJECT,fields:r}}if((0,rn.isLeafType)(t)){let r=t.serialize(e);if(r==null)return null;if(typeof r=="boolean")return{kind:$e.Kind.BOOLEAN,value:r};if(typeof r=="number"&&Number.isFinite(r)){let n=String(r);return Vc.test(n)?{kind:$e.Kind.INT,value:n}:{kind:$e.Kind.FLOAT,value:n}}if(typeof r=="string")return(0,rn.isEnumType)(t)?{kind:$e.Kind.ENUM,value:r}:t===Oy.GraphQLID&&Vc.test(r)?{kind:$e.Kind.INT,value:r}:{kind:$e.Kind.STRING,value:r};throw new TypeError(`Cannot convert value to AST: ${(0,xc.inspect)(r)}.`)}(0,Ey.invariant)(!1,"Unexpected input type: "+(0,xc.inspect)(t))}var Vc=/^-?(?:0|[1-9][0-9]*)$/});var Ye=h(ne=>{"use strict";Object.defineProperty(ne,"__esModule",{value:!0});ne.introspectionTypes=ne.__TypeKind=ne.__Type=ne.__Schema=ne.__InputValue=ne.__Field=ne.__EnumValue=ne.__DirectiveLocation=ne.__Directive=ne.TypeNameMetaFieldDef=ne.TypeMetaFieldDef=ne.TypeKind=ne.SchemaMetaFieldDef=void 0;ne.isIntrospectionType=Py;var Iy=re(),Ly=je(),_e=yr(),Sy=Ce(),Dy=sn(),R=Y(),ae=yt(),Ys=new R.GraphQLObjectType({name:"__Schema",description:"A GraphQL Schema defines the capabilities of a GraphQL server. It exposes all available types and directives on the server, as well as the entry points for query, mutation, and subscription operations.",fields:()=>({description:{type:ae.GraphQLString,resolve:e=>e.description},types:{description:"A list of all types supported by this server.",type:new R.GraphQLNonNull(new R.GraphQLList(new R.GraphQLNonNull(Be))),resolve(e){return Object.values(e.getTypeMap())}},queryType:{description:"The type that query operations will be rooted at.",type:new R.GraphQLNonNull(Be),resolve:e=>e.getQueryType()},mutationType:{description:"If this server supports mutation, the type that mutation operations will be rooted at.",type:Be,resolve:e=>e.getMutationType()},subscriptionType:{description:"If this server support subscription, the type that subscription operations will be rooted at.",type:Be,resolve:e=>e.getSubscriptionType()},directives:{description:"A list of all directives supported by this server.",type:new R.GraphQLNonNull(new R.GraphQLList(new R.GraphQLNonNull(Js))),resolve:e=>e.getDirectives()}})});ne.__Schema=Ys;var Js=new R.GraphQLObjectType({name:"__Directive",description:`A Directive provides a way to describe alternate runtime execution and type validation behavior in a GraphQL document. + +In some cases, you need to provide options to alter GraphQL's execution behavior in ways field arguments will not suffice, such as conditionally including or skipping a field. Directives provide this by describing additional information to the executor.`,fields:()=>({name:{type:new R.GraphQLNonNull(ae.GraphQLString),resolve:e=>e.name},description:{type:ae.GraphQLString,resolve:e=>e.description},isRepeatable:{type:new R.GraphQLNonNull(ae.GraphQLBoolean),resolve:e=>e.isRepeatable},locations:{type:new R.GraphQLNonNull(new R.GraphQLList(new R.GraphQLNonNull(Xs))),resolve:e=>e.locations},args:{type:new R.GraphQLNonNull(new R.GraphQLList(new R.GraphQLNonNull(an))),args:{includeDeprecated:{type:ae.GraphQLBoolean,defaultValue:!1}},resolve(e,{includeDeprecated:t}){return t?e.args:e.args.filter(r=>r.deprecationReason==null)}}})});ne.__Directive=Js;var Xs=new R.GraphQLEnumType({name:"__DirectiveLocation",description:"A Directive can be adjacent to many parts of the GraphQL language, a __DirectiveLocation describes one such possible adjacencies.",values:{QUERY:{value:_e.DirectiveLocation.QUERY,description:"Location adjacent to a query operation."},MUTATION:{value:_e.DirectiveLocation.MUTATION,description:"Location adjacent to a mutation operation."},SUBSCRIPTION:{value:_e.DirectiveLocation.SUBSCRIPTION,description:"Location adjacent to a subscription operation."},FIELD:{value:_e.DirectiveLocation.FIELD,description:"Location adjacent to a field."},FRAGMENT_DEFINITION:{value:_e.DirectiveLocation.FRAGMENT_DEFINITION,description:"Location adjacent to a fragment definition."},FRAGMENT_SPREAD:{value:_e.DirectiveLocation.FRAGMENT_SPREAD,description:"Location adjacent to a fragment spread."},INLINE_FRAGMENT:{value:_e.DirectiveLocation.INLINE_FRAGMENT,description:"Location adjacent to an inline fragment."},VARIABLE_DEFINITION:{value:_e.DirectiveLocation.VARIABLE_DEFINITION,description:"Location adjacent to a variable definition."},SCHEMA:{value:_e.DirectiveLocation.SCHEMA,description:"Location adjacent to a schema definition."},SCALAR:{value:_e.DirectiveLocation.SCALAR,description:"Location adjacent to a scalar definition."},OBJECT:{value:_e.DirectiveLocation.OBJECT,description:"Location adjacent to an object type definition."},FIELD_DEFINITION:{value:_e.DirectiveLocation.FIELD_DEFINITION,description:"Location adjacent to a field definition."},ARGUMENT_DEFINITION:{value:_e.DirectiveLocation.ARGUMENT_DEFINITION,description:"Location adjacent to an argument definition."},INTERFACE:{value:_e.DirectiveLocation.INTERFACE,description:"Location adjacent to an interface definition."},UNION:{value:_e.DirectiveLocation.UNION,description:"Location adjacent to a union definition."},ENUM:{value:_e.DirectiveLocation.ENUM,description:"Location adjacent to an enum definition."},ENUM_VALUE:{value:_e.DirectiveLocation.ENUM_VALUE,description:"Location adjacent to an enum value definition."},INPUT_OBJECT:{value:_e.DirectiveLocation.INPUT_OBJECT,description:"Location adjacent to an input object type definition."},INPUT_FIELD_DEFINITION:{value:_e.DirectiveLocation.INPUT_FIELD_DEFINITION,description:"Location adjacent to an input object field definition."}}});ne.__DirectiveLocation=Xs;var Be=new R.GraphQLObjectType({name:"__Type",description:"The fundamental unit of any GraphQL Schema is the type. There are many kinds of types in GraphQL as represented by the `__TypeKind` enum.\n\nDepending on the kind of a type, certain fields describe information about that type. Scalar types provide no information beyond a name, description and optional `specifiedByURL`, while Enum types provide their values. Object and Interface types provide the fields they describe. Abstract types, Union and Interface, provide the Object types possible at runtime. List and NonNull types compose other types.",fields:()=>({kind:{type:new R.GraphQLNonNull(Ws),resolve(e){if((0,R.isScalarType)(e))return Ne.SCALAR;if((0,R.isObjectType)(e))return Ne.OBJECT;if((0,R.isInterfaceType)(e))return Ne.INTERFACE;if((0,R.isUnionType)(e))return Ne.UNION;if((0,R.isEnumType)(e))return Ne.ENUM;if((0,R.isInputObjectType)(e))return Ne.INPUT_OBJECT;if((0,R.isListType)(e))return Ne.LIST;if((0,R.isNonNullType)(e))return Ne.NON_NULL;(0,Ly.invariant)(!1,`Unexpected type: "${(0,Iy.inspect)(e)}".`)}},name:{type:ae.GraphQLString,resolve:e=>"name"in e?e.name:void 0},description:{type:ae.GraphQLString,resolve:e=>"description"in e?e.description:void 0},specifiedByURL:{type:ae.GraphQLString,resolve:e=>"specifiedByURL"in e?e.specifiedByURL:void 0},fields:{type:new R.GraphQLList(new R.GraphQLNonNull(Hs)),args:{includeDeprecated:{type:ae.GraphQLBoolean,defaultValue:!1}},resolve(e,{includeDeprecated:t}){if((0,R.isObjectType)(e)||(0,R.isInterfaceType)(e)){let r=Object.values(e.getFields());return t?r:r.filter(n=>n.deprecationReason==null)}}},interfaces:{type:new R.GraphQLList(new R.GraphQLNonNull(Be)),resolve(e){if((0,R.isObjectType)(e)||(0,R.isInterfaceType)(e))return e.getInterfaces()}},possibleTypes:{type:new R.GraphQLList(new R.GraphQLNonNull(Be)),resolve(e,t,r,{schema:n}){if((0,R.isAbstractType)(e))return n.getPossibleTypes(e)}},enumValues:{type:new R.GraphQLList(new R.GraphQLNonNull(zs)),args:{includeDeprecated:{type:ae.GraphQLBoolean,defaultValue:!1}},resolve(e,{includeDeprecated:t}){if((0,R.isEnumType)(e)){let r=e.getValues();return t?r:r.filter(n=>n.deprecationReason==null)}}},inputFields:{type:new R.GraphQLList(new R.GraphQLNonNull(an)),args:{includeDeprecated:{type:ae.GraphQLBoolean,defaultValue:!1}},resolve(e,{includeDeprecated:t}){if((0,R.isInputObjectType)(e)){let r=Object.values(e.getFields());return t?r:r.filter(n=>n.deprecationReason==null)}}},ofType:{type:Be,resolve:e=>"ofType"in e?e.ofType:void 0}})});ne.__Type=Be;var Hs=new R.GraphQLObjectType({name:"__Field",description:"Object and Interface types are described by a list of Fields, each of which has a name, potentially a list of arguments, and a return type.",fields:()=>({name:{type:new R.GraphQLNonNull(ae.GraphQLString),resolve:e=>e.name},description:{type:ae.GraphQLString,resolve:e=>e.description},args:{type:new R.GraphQLNonNull(new R.GraphQLList(new R.GraphQLNonNull(an))),args:{includeDeprecated:{type:ae.GraphQLBoolean,defaultValue:!1}},resolve(e,{includeDeprecated:t}){return t?e.args:e.args.filter(r=>r.deprecationReason==null)}},type:{type:new R.GraphQLNonNull(Be),resolve:e=>e.type},isDeprecated:{type:new R.GraphQLNonNull(ae.GraphQLBoolean),resolve:e=>e.deprecationReason!=null},deprecationReason:{type:ae.GraphQLString,resolve:e=>e.deprecationReason}})});ne.__Field=Hs;var an=new R.GraphQLObjectType({name:"__InputValue",description:"Arguments provided to Fields or Directives and the input fields of an InputObject are represented as Input Values which describe their type and optionally a default value.",fields:()=>({name:{type:new R.GraphQLNonNull(ae.GraphQLString),resolve:e=>e.name},description:{type:ae.GraphQLString,resolve:e=>e.description},type:{type:new R.GraphQLNonNull(Be),resolve:e=>e.type},defaultValue:{type:ae.GraphQLString,description:"A GraphQL-formatted string representing the default value for this input value.",resolve(e){let{type:t,defaultValue:r}=e,n=(0,Dy.astFromValue)(r,t);return n?(0,Sy.print)(n):null}},isDeprecated:{type:new R.GraphQLNonNull(ae.GraphQLBoolean),resolve:e=>e.deprecationReason!=null},deprecationReason:{type:ae.GraphQLString,resolve:e=>e.deprecationReason}})});ne.__InputValue=an;var zs=new R.GraphQLObjectType({name:"__EnumValue",description:"One possible value for a given Enum. Enum values are unique values, not a placeholder for a string or numeric value. However an Enum value is returned in a JSON response as a string.",fields:()=>({name:{type:new R.GraphQLNonNull(ae.GraphQLString),resolve:e=>e.name},description:{type:ae.GraphQLString,resolve:e=>e.description},isDeprecated:{type:new R.GraphQLNonNull(ae.GraphQLBoolean),resolve:e=>e.deprecationReason!=null},deprecationReason:{type:ae.GraphQLString,resolve:e=>e.deprecationReason}})});ne.__EnumValue=zs;var Ne;ne.TypeKind=Ne;(function(e){e.SCALAR="SCALAR",e.OBJECT="OBJECT",e.INTERFACE="INTERFACE",e.UNION="UNION",e.ENUM="ENUM",e.INPUT_OBJECT="INPUT_OBJECT",e.LIST="LIST",e.NON_NULL="NON_NULL"})(Ne||(ne.TypeKind=Ne={}));var Ws=new R.GraphQLEnumType({name:"__TypeKind",description:"An enum describing what kind of type a given `__Type` is.",values:{SCALAR:{value:Ne.SCALAR,description:"Indicates this type is a scalar."},OBJECT:{value:Ne.OBJECT,description:"Indicates this type is an object. `fields` and `interfaces` are valid fields."},INTERFACE:{value:Ne.INTERFACE,description:"Indicates this type is an interface. `fields`, `interfaces`, and `possibleTypes` are valid fields."},UNION:{value:Ne.UNION,description:"Indicates this type is a union. `possibleTypes` is a valid field."},ENUM:{value:Ne.ENUM,description:"Indicates this type is an enum. `enumValues` is a valid field."},INPUT_OBJECT:{value:Ne.INPUT_OBJECT,description:"Indicates this type is an input object. `inputFields` is a valid field."},LIST:{value:Ne.LIST,description:"Indicates this type is a list. `ofType` is a valid field."},NON_NULL:{value:Ne.NON_NULL,description:"Indicates this type is a non-null. `ofType` is a valid field."}}});ne.__TypeKind=Ws;var Ay={name:"__schema",type:new R.GraphQLNonNull(Ys),description:"Access the current type schema of this server.",args:[],resolve:(e,t,r,{schema:n})=>n,deprecationReason:void 0,extensions:Object.create(null),astNode:void 0};ne.SchemaMetaFieldDef=Ay;var Ry={name:"__type",type:Be,description:"Request the type information of a single type.",args:[{name:"name",description:void 0,type:new R.GraphQLNonNull(ae.GraphQLString),defaultValue:void 0,deprecationReason:void 0,extensions:Object.create(null),astNode:void 0}],resolve:(e,{name:t},r,{schema:n})=>n.getType(t),deprecationReason:void 0,extensions:Object.create(null),astNode:void 0};ne.TypeMetaFieldDef=Ry;var jy={name:"__typename",type:new R.GraphQLNonNull(ae.GraphQLString),description:"The name of the current Object type at runtime.",args:[],resolve:(e,t,r,{parentType:n})=>n.name,deprecationReason:void 0,extensions:Object.create(null),astNode:void 0};ne.TypeNameMetaFieldDef=jy;var Mc=Object.freeze([Ys,Js,Xs,Be,Hs,an,zs,Ws]);ne.introspectionTypes=Mc;function Py(e){return Mc.some(({name:t})=>e.name===t)}});var Zt=h(Er=>{"use strict";Object.defineProperty(Er,"__esModule",{value:!0});Er.GraphQLSchema=void 0;Er.assertSchema=xy;Er.isSchema=Cc;var mi=Fe(),ea=re(),wy=Cr(),Fy=pt(),ky=Bn(),Zs=dt(),at=Y(),Gc=Ve(),qy=Ye();function Cc(e){return(0,wy.instanceOf)(e,yi)}function xy(e){if(!Cc(e))throw new Error(`Expected ${(0,ea.inspect)(e)} to be a GraphQL schema.`);return e}var yi=class{constructor(t){var r,n;this.__validationErrors=t.assumeValid===!0?[]:void 0,(0,Fy.isObjectLike)(t)||(0,mi.devAssert)(!1,"Must provide configuration object."),!t.types||Array.isArray(t.types)||(0,mi.devAssert)(!1,`"types" must be Array if provided but got: ${(0,ea.inspect)(t.types)}.`),!t.directives||Array.isArray(t.directives)||(0,mi.devAssert)(!1,`"directives" must be Array if provided but got: ${(0,ea.inspect)(t.directives)}.`),this.description=t.description,this.extensions=(0,ky.toObjMap)(t.extensions),this.astNode=t.astNode,this.extensionASTNodes=(r=t.extensionASTNodes)!==null&&r!==void 0?r:[],this._queryType=t.query,this._mutationType=t.mutation,this._subscriptionType=t.subscription,this._directives=(n=t.directives)!==null&&n!==void 0?n:Gc.specifiedDirectives;let i=new Set(t.types);if(t.types!=null)for(let s of t.types)i.delete(s),ot(s,i);this._queryType!=null&&ot(this._queryType,i),this._mutationType!=null&&ot(this._mutationType,i),this._subscriptionType!=null&&ot(this._subscriptionType,i);for(let s of this._directives)if((0,Gc.isDirective)(s))for(let a of s.args)ot(a.type,i);ot(qy.__Schema,i),this._typeMap=Object.create(null),this._subTypeMap=Object.create(null),this._implementationsMap=Object.create(null);for(let s of i){if(s==null)continue;let a=s.name;if(a||(0,mi.devAssert)(!1,"One of the provided types for building the Schema is missing a name."),this._typeMap[a]!==void 0)throw new Error(`Schema must contain uniquely named types but contains multiple types named "${a}".`);if(this._typeMap[a]=s,(0,at.isInterfaceType)(s)){for(let o of s.getInterfaces())if((0,at.isInterfaceType)(o)){let u=this._implementationsMap[o.name];u===void 0&&(u=this._implementationsMap[o.name]={objects:[],interfaces:[]}),u.interfaces.push(s)}}else if((0,at.isObjectType)(s)){for(let o of s.getInterfaces())if((0,at.isInterfaceType)(o)){let u=this._implementationsMap[o.name];u===void 0&&(u=this._implementationsMap[o.name]={objects:[],interfaces:[]}),u.objects.push(s)}}}}get[Symbol.toStringTag](){return"GraphQLSchema"}getQueryType(){return this._queryType}getMutationType(){return this._mutationType}getSubscriptionType(){return this._subscriptionType}getRootType(t){switch(t){case Zs.OperationTypeNode.QUERY:return this.getQueryType();case Zs.OperationTypeNode.MUTATION:return this.getMutationType();case Zs.OperationTypeNode.SUBSCRIPTION:return this.getSubscriptionType()}}getTypeMap(){return this._typeMap}getType(t){return this.getTypeMap()[t]}getPossibleTypes(t){return(0,at.isUnionType)(t)?t.getTypes():this.getImplementations(t).objects}getImplementations(t){let r=this._implementationsMap[t.name];return r??{objects:[],interfaces:[]}}isSubType(t,r){let n=this._subTypeMap[t.name];if(n===void 0){if(n=Object.create(null),(0,at.isUnionType)(t))for(let i of t.getTypes())n[i.name]=!0;else{let i=this.getImplementations(t);for(let s of i.objects)n[s.name]=!0;for(let s of i.interfaces)n[s.name]=!0}this._subTypeMap[t.name]=n}return n[r.name]!==void 0}getDirectives(){return this._directives}getDirective(t){return this.getDirectives().find(r=>r.name===t)}toConfig(){return{description:this.description,query:this.getQueryType(),mutation:this.getMutationType(),subscription:this.getSubscriptionType(),types:Object.values(this.getTypeMap()),directives:this.getDirectives(),extensions:this.extensions,astNode:this.astNode,extensionASTNodes:this.extensionASTNodes,assumeValid:this.__validationErrors!==void 0}}};Er.GraphQLSchema=yi;function ot(e,t){let r=(0,at.getNamedType)(e);if(!t.has(r)){if(t.add(r),(0,at.isUnionType)(r))for(let n of r.getTypes())ot(n,t);else if((0,at.isObjectType)(r)||(0,at.isInterfaceType)(r)){for(let n of r.getInterfaces())ot(n,t);for(let n of Object.values(r.getFields())){ot(n.type,t);for(let i of n.args)ot(i.type,t)}}else if((0,at.isInputObjectType)(r))for(let n of Object.values(r.getFields()))ot(n.type,t)}return t}});var un=h(hi=>{"use strict";Object.defineProperty(hi,"__esModule",{value:!0});hi.assertValidSchema=Cy;hi.validateSchema=Yc;var we=re(),Vy=q(),ta=dt(),Qc=zr(),he=Y(),Bc=Ve(),My=Ye(),Gy=Zt();function Yc(e){if((0,Gy.assertSchema)(e),e.__validationErrors)return e.__validationErrors;let t=new na(e);Qy(t),Uy(t),Ky(t);let r=t.getErrors();return e.__validationErrors=r,r}function Cy(e){let t=Yc(e);if(t.length!==0)throw new Error(t.map(r=>r.message).join(` + +`))}var na=class{constructor(t){this._errors=[],this.schema=t}reportError(t,r){let n=Array.isArray(r)?r.filter(Boolean):r;this._errors.push(new Vy.GraphQLError(t,{nodes:n}))}getErrors(){return this._errors}};function Qy(e){let t=e.schema,r=t.getQueryType();if(!r)e.reportError("Query root type must be provided.",t.astNode);else if(!(0,he.isObjectType)(r)){var n;e.reportError(`Query root type must be Object type, it cannot be ${(0,we.inspect)(r)}.`,(n=ra(t,ta.OperationTypeNode.QUERY))!==null&&n!==void 0?n:r.astNode)}let i=t.getMutationType();if(i&&!(0,he.isObjectType)(i)){var s;e.reportError(`Mutation root type must be Object type if provided, it cannot be ${(0,we.inspect)(i)}.`,(s=ra(t,ta.OperationTypeNode.MUTATION))!==null&&s!==void 0?s:i.astNode)}let a=t.getSubscriptionType();if(a&&!(0,he.isObjectType)(a)){var o;e.reportError(`Subscription root type must be Object type if provided, it cannot be ${(0,we.inspect)(a)}.`,(o=ra(t,ta.OperationTypeNode.SUBSCRIPTION))!==null&&o!==void 0?o:a.astNode)}}function ra(e,t){var r;return(r=[e.astNode,...e.extensionASTNodes].flatMap(n=>{var i;return(i=n?.operationTypes)!==null&&i!==void 0?i:[]}).find(n=>n.operation===t))===null||r===void 0?void 0:r.type}function Uy(e){for(let r of e.schema.getDirectives()){if(!(0,Bc.isDirective)(r)){e.reportError(`Expected directive but got: ${(0,we.inspect)(r)}.`,r?.astNode);continue}er(e,r);for(let n of r.args)if(er(e,n),(0,he.isInputType)(n.type)||e.reportError(`The type of @${r.name}(${n.name}:) must be Input Type but got: ${(0,we.inspect)(n.type)}.`,n.astNode),(0,he.isRequiredArgument)(n)&&n.deprecationReason!=null){var t;e.reportError(`Required argument @${r.name}(${n.name}:) cannot be deprecated.`,[ia(n.astNode),(t=n.astNode)===null||t===void 0?void 0:t.type])}}}function er(e,t){t.name.startsWith("__")&&e.reportError(`Name "${t.name}" must not begin with "__", which is reserved by GraphQL introspection.`,t.astNode)}function Ky(e){let t=Hy(e),r=e.schema.getTypeMap();for(let n of Object.values(r)){if(!(0,he.isNamedType)(n)){e.reportError(`Expected GraphQL named type but got: ${(0,we.inspect)(n)}.`,n.astNode);continue}(0,My.isIntrospectionType)(n)||er(e,n),(0,he.isObjectType)(n)||(0,he.isInterfaceType)(n)?(Uc(e,n),Kc(e,n)):(0,he.isUnionType)(n)?Yy(e,n):(0,he.isEnumType)(n)?Jy(e,n):(0,he.isInputObjectType)(n)&&(Xy(e,n),t(n))}}function Uc(e,t){let r=Object.values(t.getFields());r.length===0&&e.reportError(`Type ${t.name} must define one or more fields.`,[t.astNode,...t.extensionASTNodes]);for(let a of r){if(er(e,a),!(0,he.isOutputType)(a.type)){var n;e.reportError(`The type of ${t.name}.${a.name} must be Output Type but got: ${(0,we.inspect)(a.type)}.`,(n=a.astNode)===null||n===void 0?void 0:n.type)}for(let o of a.args){let u=o.name;if(er(e,o),!(0,he.isInputType)(o.type)){var i;e.reportError(`The type of ${t.name}.${a.name}(${u}:) must be Input Type but got: ${(0,we.inspect)(o.type)}.`,(i=o.astNode)===null||i===void 0?void 0:i.type)}if((0,he.isRequiredArgument)(o)&&o.deprecationReason!=null){var s;e.reportError(`Required argument ${t.name}.${a.name}(${u}:) cannot be deprecated.`,[ia(o.astNode),(s=o.astNode)===null||s===void 0?void 0:s.type])}}}}function Kc(e,t){let r=Object.create(null);for(let n of t.getInterfaces()){if(!(0,he.isInterfaceType)(n)){e.reportError(`Type ${(0,we.inspect)(t)} must only implement Interface types, it cannot implement ${(0,we.inspect)(n)}.`,on(t,n));continue}if(t===n){e.reportError(`Type ${t.name} cannot implement itself because it would create a circular reference.`,on(t,n));continue}if(r[n.name]){e.reportError(`Type ${t.name} can only implement ${n.name} once.`,on(t,n));continue}r[n.name]=!0,By(e,t,n),$y(e,t,n)}}function $y(e,t,r){let n=t.getFields();for(let u of Object.values(r.getFields())){let c=u.name,l=n[c];if(!l){e.reportError(`Interface field ${r.name}.${c} expected but ${t.name} does not provide it.`,[u.astNode,t.astNode,...t.extensionASTNodes]);continue}if(!(0,Qc.isTypeSubTypeOf)(e.schema,l.type,u.type)){var i,s;e.reportError(`Interface field ${r.name}.${c} expects type ${(0,we.inspect)(u.type)} but ${t.name}.${c} is type ${(0,we.inspect)(l.type)}.`,[(i=u.astNode)===null||i===void 0?void 0:i.type,(s=l.astNode)===null||s===void 0?void 0:s.type])}for(let f of u.args){let d=f.name,y=l.args.find(T=>T.name===d);if(!y){e.reportError(`Interface field argument ${r.name}.${c}(${d}:) expected but ${t.name}.${c} does not provide it.`,[f.astNode,l.astNode]);continue}if(!(0,Qc.isEqualType)(f.type,y.type)){var a,o;e.reportError(`Interface field argument ${r.name}.${c}(${d}:) expects type ${(0,we.inspect)(f.type)} but ${t.name}.${c}(${d}:) is type ${(0,we.inspect)(y.type)}.`,[(a=f.astNode)===null||a===void 0?void 0:a.type,(o=y.astNode)===null||o===void 0?void 0:o.type])}}for(let f of l.args){let d=f.name;!u.args.find(T=>T.name===d)&&(0,he.isRequiredArgument)(f)&&e.reportError(`Object field ${t.name}.${c} includes required argument ${d} that is missing from the Interface field ${r.name}.${c}.`,[f.astNode,u.astNode])}}}function By(e,t,r){let n=t.getInterfaces();for(let i of r.getInterfaces())n.includes(i)||e.reportError(i===t?`Type ${t.name} cannot implement ${r.name} because it would create a circular reference.`:`Type ${t.name} must implement ${i.name} because it is implemented by ${r.name}.`,[...on(r,i),...on(t,r)])}function Yy(e,t){let r=t.getTypes();r.length===0&&e.reportError(`Union type ${t.name} must define one or more member types.`,[t.astNode,...t.extensionASTNodes]);let n=Object.create(null);for(let i of r){if(n[i.name]){e.reportError(`Union type ${t.name} can only include type ${i.name} once.`,$c(t,i.name));continue}n[i.name]=!0,(0,he.isObjectType)(i)||e.reportError(`Union type ${t.name} can only include Object types, it cannot include ${(0,we.inspect)(i)}.`,$c(t,String(i)))}}function Jy(e,t){let r=t.getValues();r.length===0&&e.reportError(`Enum type ${t.name} must define one or more values.`,[t.astNode,...t.extensionASTNodes]);for(let n of r)er(e,n)}function Xy(e,t){let r=Object.values(t.getFields());r.length===0&&e.reportError(`Input Object type ${t.name} must define one or more fields.`,[t.astNode,...t.extensionASTNodes]);for(let s of r){if(er(e,s),!(0,he.isInputType)(s.type)){var n;e.reportError(`The type of ${t.name}.${s.name} must be Input Type but got: ${(0,we.inspect)(s.type)}.`,(n=s.astNode)===null||n===void 0?void 0:n.type)}if((0,he.isRequiredInputField)(s)&&s.deprecationReason!=null){var i;e.reportError(`Required input field ${t.name}.${s.name} cannot be deprecated.`,[ia(s.astNode),(i=s.astNode)===null||i===void 0?void 0:i.type])}}}function Hy(e){let t=Object.create(null),r=[],n=Object.create(null);return i;function i(s){if(t[s.name])return;t[s.name]=!0,n[s.name]=r.length;let a=Object.values(s.getFields());for(let o of a)if((0,he.isNonNullType)(o.type)&&(0,he.isInputObjectType)(o.type.ofType)){let u=o.type.ofType,c=n[u.name];if(r.push(o),c===void 0)i(u);else{let l=r.slice(c),f=l.map(d=>d.name).join(".");e.reportError(`Cannot reference Input Object "${u.name}" within itself through a series of non-null fields: "${f}".`,l.map(d=>d.astNode))}r.pop()}n[s.name]=void 0}}function on(e,t){let{astNode:r,extensionASTNodes:n}=e;return(r!=null?[r,...n]:n).flatMap(s=>{var a;return(a=s.interfaces)!==null&&a!==void 0?a:[]}).filter(s=>s.name.value===t.name)}function $c(e,t){let{astNode:r,extensionASTNodes:n}=e;return(r!=null?[r,...n]:n).flatMap(s=>{var a;return(a=s.types)!==null&&a!==void 0?a:[]}).filter(s=>s.name.value===t)}function ia(e){var t;return e==null||(t=e.directives)===null||t===void 0?void 0:t.find(r=>r.name.value===Bc.GraphQLDeprecatedDirective.name)}});var ht=h(oa=>{"use strict";Object.defineProperty(oa,"__esModule",{value:!0});oa.typeFromAST=aa;var sa=H(),Jc=Y();function aa(e,t){switch(t.kind){case sa.Kind.LIST_TYPE:{let r=aa(e,t.type);return r&&new Jc.GraphQLList(r)}case sa.Kind.NON_NULL_TYPE:{let r=aa(e,t.type);return r&&new Jc.GraphQLNonNull(r)}case sa.Kind.NAMED_TYPE:return e.getType(t.name.value)}}});var vi=h(cn=>{"use strict";Object.defineProperty(cn,"__esModule",{value:!0});cn.TypeInfo=void 0;cn.visitWithTypeInfo=Zy;var zy=dt(),de=H(),Xc=Yt(),fe=Y(),_r=Ye(),Hc=ht(),ua=class{constructor(t,r,n){this._schema=t,this._typeStack=[],this._parentTypeStack=[],this._inputTypeStack=[],this._fieldDefStack=[],this._defaultValueStack=[],this._directive=null,this._argument=null,this._enumValue=null,this._getFieldDef=n??Wy,r&&((0,fe.isInputType)(r)&&this._inputTypeStack.push(r),(0,fe.isCompositeType)(r)&&this._parentTypeStack.push(r),(0,fe.isOutputType)(r)&&this._typeStack.push(r))}get[Symbol.toStringTag](){return"TypeInfo"}getType(){if(this._typeStack.length>0)return this._typeStack[this._typeStack.length-1]}getParentType(){if(this._parentTypeStack.length>0)return this._parentTypeStack[this._parentTypeStack.length-1]}getInputType(){if(this._inputTypeStack.length>0)return this._inputTypeStack[this._inputTypeStack.length-1]}getParentInputType(){if(this._inputTypeStack.length>1)return this._inputTypeStack[this._inputTypeStack.length-2]}getFieldDef(){if(this._fieldDefStack.length>0)return this._fieldDefStack[this._fieldDefStack.length-1]}getDefaultValue(){if(this._defaultValueStack.length>0)return this._defaultValueStack[this._defaultValueStack.length-1]}getDirective(){return this._directive}getArgument(){return this._argument}getEnumValue(){return this._enumValue}enter(t){let r=this._schema;switch(t.kind){case de.Kind.SELECTION_SET:{let i=(0,fe.getNamedType)(this.getType());this._parentTypeStack.push((0,fe.isCompositeType)(i)?i:void 0);break}case de.Kind.FIELD:{let i=this.getParentType(),s,a;i&&(s=this._getFieldDef(r,i,t),s&&(a=s.type)),this._fieldDefStack.push(s),this._typeStack.push((0,fe.isOutputType)(a)?a:void 0);break}case de.Kind.DIRECTIVE:this._directive=r.getDirective(t.name.value);break;case de.Kind.OPERATION_DEFINITION:{let i=r.getRootType(t.operation);this._typeStack.push((0,fe.isObjectType)(i)?i:void 0);break}case de.Kind.INLINE_FRAGMENT:case de.Kind.FRAGMENT_DEFINITION:{let i=t.typeCondition,s=i?(0,Hc.typeFromAST)(r,i):(0,fe.getNamedType)(this.getType());this._typeStack.push((0,fe.isOutputType)(s)?s:void 0);break}case de.Kind.VARIABLE_DEFINITION:{let i=(0,Hc.typeFromAST)(r,t.type);this._inputTypeStack.push((0,fe.isInputType)(i)?i:void 0);break}case de.Kind.ARGUMENT:{var n;let i,s,a=(n=this.getDirective())!==null&&n!==void 0?n:this.getFieldDef();a&&(i=a.args.find(o=>o.name===t.name.value),i&&(s=i.type)),this._argument=i,this._defaultValueStack.push(i?i.defaultValue:void 0),this._inputTypeStack.push((0,fe.isInputType)(s)?s:void 0);break}case de.Kind.LIST:{let i=(0,fe.getNullableType)(this.getInputType()),s=(0,fe.isListType)(i)?i.ofType:i;this._defaultValueStack.push(void 0),this._inputTypeStack.push((0,fe.isInputType)(s)?s:void 0);break}case de.Kind.OBJECT_FIELD:{let i=(0,fe.getNamedType)(this.getInputType()),s,a;(0,fe.isInputObjectType)(i)&&(a=i.getFields()[t.name.value],a&&(s=a.type)),this._defaultValueStack.push(a?a.defaultValue:void 0),this._inputTypeStack.push((0,fe.isInputType)(s)?s:void 0);break}case de.Kind.ENUM:{let i=(0,fe.getNamedType)(this.getInputType()),s;(0,fe.isEnumType)(i)&&(s=i.getValue(t.value)),this._enumValue=s;break}default:}}leave(t){switch(t.kind){case de.Kind.SELECTION_SET:this._parentTypeStack.pop();break;case de.Kind.FIELD:this._fieldDefStack.pop(),this._typeStack.pop();break;case de.Kind.DIRECTIVE:this._directive=null;break;case de.Kind.OPERATION_DEFINITION:case de.Kind.INLINE_FRAGMENT:case de.Kind.FRAGMENT_DEFINITION:this._typeStack.pop();break;case de.Kind.VARIABLE_DEFINITION:this._inputTypeStack.pop();break;case de.Kind.ARGUMENT:this._argument=null,this._defaultValueStack.pop(),this._inputTypeStack.pop();break;case de.Kind.LIST:case de.Kind.OBJECT_FIELD:this._defaultValueStack.pop(),this._inputTypeStack.pop();break;case de.Kind.ENUM:this._enumValue=null;break;default:}}};cn.TypeInfo=ua;function Wy(e,t,r){let n=r.name.value;if(n===_r.SchemaMetaFieldDef.name&&e.getQueryType()===t)return _r.SchemaMetaFieldDef;if(n===_r.TypeMetaFieldDef.name&&e.getQueryType()===t)return _r.TypeMetaFieldDef;if(n===_r.TypeNameMetaFieldDef.name&&(0,fe.isCompositeType)(t))return _r.TypeNameMetaFieldDef;if((0,fe.isObjectType)(t)||(0,fe.isInterfaceType)(t))return t.getFields()[n]}function Zy(e,t){return{enter(...r){let n=r[0];e.enter(n);let i=(0,Xc.getEnterLeaveForKind)(t,n.kind).enter;if(i){let s=i.apply(t,r);return s!==void 0&&(e.leave(n),(0,zy.isNode)(s)&&e.enter(s)),s}},leave(...r){let n=r[0],i=(0,Xc.getEnterLeaveForKind)(t,n.kind).leave,s;return i&&(s=i.apply(t,r)),e.leave(n),s}}}});var tr=h(Je=>{"use strict";Object.defineProperty(Je,"__esModule",{value:!0});Je.isConstValueNode=ca;Je.isDefinitionNode=eh;Je.isExecutableDefinitionNode=zc;Je.isSelectionNode=th;Je.isTypeDefinitionNode=el;Je.isTypeExtensionNode=rl;Je.isTypeNode=rh;Je.isTypeSystemDefinitionNode=Zc;Je.isTypeSystemExtensionNode=tl;Je.isValueNode=Wc;var J=H();function eh(e){return zc(e)||Zc(e)||tl(e)}function zc(e){return e.kind===J.Kind.OPERATION_DEFINITION||e.kind===J.Kind.FRAGMENT_DEFINITION}function th(e){return e.kind===J.Kind.FIELD||e.kind===J.Kind.FRAGMENT_SPREAD||e.kind===J.Kind.INLINE_FRAGMENT}function Wc(e){return e.kind===J.Kind.VARIABLE||e.kind===J.Kind.INT||e.kind===J.Kind.FLOAT||e.kind===J.Kind.STRING||e.kind===J.Kind.BOOLEAN||e.kind===J.Kind.NULL||e.kind===J.Kind.ENUM||e.kind===J.Kind.LIST||e.kind===J.Kind.OBJECT}function ca(e){return Wc(e)&&(e.kind===J.Kind.LIST?e.values.some(ca):e.kind===J.Kind.OBJECT?e.fields.some(t=>ca(t.value)):e.kind!==J.Kind.VARIABLE)}function rh(e){return e.kind===J.Kind.NAMED_TYPE||e.kind===J.Kind.LIST_TYPE||e.kind===J.Kind.NON_NULL_TYPE}function Zc(e){return e.kind===J.Kind.SCHEMA_DEFINITION||el(e)||e.kind===J.Kind.DIRECTIVE_DEFINITION}function el(e){return e.kind===J.Kind.SCALAR_TYPE_DEFINITION||e.kind===J.Kind.OBJECT_TYPE_DEFINITION||e.kind===J.Kind.INTERFACE_TYPE_DEFINITION||e.kind===J.Kind.UNION_TYPE_DEFINITION||e.kind===J.Kind.ENUM_TYPE_DEFINITION||e.kind===J.Kind.INPUT_OBJECT_TYPE_DEFINITION}function tl(e){return e.kind===J.Kind.SCHEMA_EXTENSION||rl(e)}function rl(e){return e.kind===J.Kind.SCALAR_TYPE_EXTENSION||e.kind===J.Kind.OBJECT_TYPE_EXTENSION||e.kind===J.Kind.INTERFACE_TYPE_EXTENSION||e.kind===J.Kind.UNION_TYPE_EXTENSION||e.kind===J.Kind.ENUM_TYPE_EXTENSION||e.kind===J.Kind.INPUT_OBJECT_TYPE_EXTENSION}});var pa=h(la=>{"use strict";Object.defineProperty(la,"__esModule",{value:!0});la.ExecutableDefinitionsRule=sh;var nh=q(),nl=H(),ih=tr();function sh(e){return{Document(t){for(let r of t.definitions)if(!(0,ih.isExecutableDefinitionNode)(r)){let n=r.kind===nl.Kind.SCHEMA_DEFINITION||r.kind===nl.Kind.SCHEMA_EXTENSION?"schema":'"'+r.name.value+'"';e.reportError(new nh.GraphQLError(`The ${n} definition is not executable.`,{nodes:r}))}return!1}}}});var fa=h(da=>{"use strict";Object.defineProperty(da,"__esModule",{value:!0});da.FieldsOnCorrectTypeRule=ch;var il=wt(),ah=$r(),oh=kt(),uh=q(),ln=Y();function ch(e){return{Field(t){let r=e.getParentType();if(r&&!e.getFieldDef()){let i=e.getSchema(),s=t.name.value,a=(0,il.didYouMean)("to use an inline fragment on",lh(i,r,s));a===""&&(a=(0,il.didYouMean)(ph(r,s))),e.reportError(new uh.GraphQLError(`Cannot query field "${s}" on type "${r.name}".`+a,{nodes:t}))}}}}function lh(e,t,r){if(!(0,ln.isAbstractType)(t))return[];let n=new Set,i=Object.create(null);for(let a of e.getPossibleTypes(t))if(a.getFields()[r]){n.add(a),i[a.name]=1;for(let o of a.getInterfaces()){var s;o.getFields()[r]&&(n.add(o),i[o.name]=((s=i[o.name])!==null&&s!==void 0?s:0)+1)}}return[...n].sort((a,o)=>{let u=i[o.name]-i[a.name];return u!==0?u:(0,ln.isInterfaceType)(a)&&e.isSubType(a,o)?-1:(0,ln.isInterfaceType)(o)&&e.isSubType(o,a)?1:(0,ah.naturalCompare)(a.name,o.name)}).map(a=>a.name)}function ph(e,t){if((0,ln.isObjectType)(e)||(0,ln.isInterfaceType)(e)){let r=Object.keys(e.getFields());return(0,oh.suggestionList)(t,r)}return[]}});var ya=h(ma=>{"use strict";Object.defineProperty(ma,"__esModule",{value:!0});ma.FragmentsOnCompositeTypesRule=dh;var sl=q(),al=Ce(),ol=Y(),ul=ht();function dh(e){return{InlineFragment(t){let r=t.typeCondition;if(r){let n=(0,ul.typeFromAST)(e.getSchema(),r);if(n&&!(0,ol.isCompositeType)(n)){let i=(0,al.print)(r);e.reportError(new sl.GraphQLError(`Fragment cannot condition on non composite type "${i}".`,{nodes:r}))}}},FragmentDefinition(t){let r=(0,ul.typeFromAST)(e.getSchema(),t.typeCondition);if(r&&!(0,ol.isCompositeType)(r)){let n=(0,al.print)(t.typeCondition);e.reportError(new sl.GraphQLError(`Fragment "${t.name.value}" cannot condition on non composite type "${n}".`,{nodes:t.typeCondition}))}}}}});var ha=h(Ti=>{"use strict";Object.defineProperty(Ti,"__esModule",{value:!0});Ti.KnownArgumentNamesOnDirectivesRule=dl;Ti.KnownArgumentNamesRule=yh;var cl=wt(),ll=kt(),pl=q(),fh=H(),mh=Ve();function yh(e){return{...dl(e),Argument(t){let r=e.getArgument(),n=e.getFieldDef(),i=e.getParentType();if(!r&&n&&i){let s=t.name.value,a=n.args.map(u=>u.name),o=(0,ll.suggestionList)(s,a);e.reportError(new pl.GraphQLError(`Unknown argument "${s}" on field "${i.name}.${n.name}".`+(0,cl.didYouMean)(o),{nodes:t}))}}}}function dl(e){let t=Object.create(null),r=e.getSchema(),n=r?r.getDirectives():mh.specifiedDirectives;for(let a of n)t[a.name]=a.args.map(o=>o.name);let i=e.getDocument().definitions;for(let a of i)if(a.kind===fh.Kind.DIRECTIVE_DEFINITION){var s;let o=(s=a.arguments)!==null&&s!==void 0?s:[];t[a.name.value]=o.map(u=>u.name.value)}return{Directive(a){let o=a.name.value,u=t[o];if(a.arguments&&u)for(let c of a.arguments){let l=c.name.value;if(!u.includes(l)){let f=(0,ll.suggestionList)(l,u);e.reportError(new pl.GraphQLError(`Unknown argument "${l}" on directive "@${o}".`+(0,cl.didYouMean)(f),{nodes:c}))}}return!1}}}});var ga=h(ba=>{"use strict";Object.defineProperty(ba,"__esModule",{value:!0});ba.KnownDirectivesRule=Th;var hh=re(),va=je(),fl=q(),Ta=dt(),Oe=yr(),ue=H(),vh=Ve();function Th(e){let t=Object.create(null),r=e.getSchema(),n=r?r.getDirectives():vh.specifiedDirectives;for(let s of n)t[s.name]=s.locations;let i=e.getDocument().definitions;for(let s of i)s.kind===ue.Kind.DIRECTIVE_DEFINITION&&(t[s.name.value]=s.locations.map(a=>a.value));return{Directive(s,a,o,u,c){let l=s.name.value,f=t[l];if(!f){e.reportError(new fl.GraphQLError(`Unknown directive "@${l}".`,{nodes:s}));return}let d=bh(c);d&&!f.includes(d)&&e.reportError(new fl.GraphQLError(`Directive "@${l}" may not be used on ${d}.`,{nodes:s}))}}}function bh(e){let t=e[e.length-1];switch("kind"in t||(0,va.invariant)(!1),t.kind){case ue.Kind.OPERATION_DEFINITION:return gh(t.operation);case ue.Kind.FIELD:return Oe.DirectiveLocation.FIELD;case ue.Kind.FRAGMENT_SPREAD:return Oe.DirectiveLocation.FRAGMENT_SPREAD;case ue.Kind.INLINE_FRAGMENT:return Oe.DirectiveLocation.INLINE_FRAGMENT;case ue.Kind.FRAGMENT_DEFINITION:return Oe.DirectiveLocation.FRAGMENT_DEFINITION;case ue.Kind.VARIABLE_DEFINITION:return Oe.DirectiveLocation.VARIABLE_DEFINITION;case ue.Kind.SCHEMA_DEFINITION:case ue.Kind.SCHEMA_EXTENSION:return Oe.DirectiveLocation.SCHEMA;case ue.Kind.SCALAR_TYPE_DEFINITION:case ue.Kind.SCALAR_TYPE_EXTENSION:return Oe.DirectiveLocation.SCALAR;case ue.Kind.OBJECT_TYPE_DEFINITION:case ue.Kind.OBJECT_TYPE_EXTENSION:return Oe.DirectiveLocation.OBJECT;case ue.Kind.FIELD_DEFINITION:return Oe.DirectiveLocation.FIELD_DEFINITION;case ue.Kind.INTERFACE_TYPE_DEFINITION:case ue.Kind.INTERFACE_TYPE_EXTENSION:return Oe.DirectiveLocation.INTERFACE;case ue.Kind.UNION_TYPE_DEFINITION:case ue.Kind.UNION_TYPE_EXTENSION:return Oe.DirectiveLocation.UNION;case ue.Kind.ENUM_TYPE_DEFINITION:case ue.Kind.ENUM_TYPE_EXTENSION:return Oe.DirectiveLocation.ENUM;case ue.Kind.ENUM_VALUE_DEFINITION:return Oe.DirectiveLocation.ENUM_VALUE;case ue.Kind.INPUT_OBJECT_TYPE_DEFINITION:case ue.Kind.INPUT_OBJECT_TYPE_EXTENSION:return Oe.DirectiveLocation.INPUT_OBJECT;case ue.Kind.INPUT_VALUE_DEFINITION:{let r=e[e.length-3];return"kind"in r||(0,va.invariant)(!1),r.kind===ue.Kind.INPUT_OBJECT_TYPE_DEFINITION?Oe.DirectiveLocation.INPUT_FIELD_DEFINITION:Oe.DirectiveLocation.ARGUMENT_DEFINITION}default:(0,va.invariant)(!1,"Unexpected kind: "+(0,hh.inspect)(t.kind))}}function gh(e){switch(e){case Ta.OperationTypeNode.QUERY:return Oe.DirectiveLocation.QUERY;case Ta.OperationTypeNode.MUTATION:return Oe.DirectiveLocation.MUTATION;case Ta.OperationTypeNode.SUBSCRIPTION:return Oe.DirectiveLocation.SUBSCRIPTION}}});var _a=h(Ea=>{"use strict";Object.defineProperty(Ea,"__esModule",{value:!0});Ea.KnownFragmentNamesRule=_h;var Eh=q();function _h(e){return{FragmentSpread(t){let r=t.name.value;e.getFragment(r)||e.reportError(new Eh.GraphQLError(`Unknown fragment "${r}".`,{nodes:t.name}))}}}});var Ia=h(Oa=>{"use strict";Object.defineProperty(Oa,"__esModule",{value:!0});Oa.KnownTypeNamesRule=Dh;var Nh=wt(),Oh=kt(),Ih=q(),Na=tr(),Lh=Ye(),Sh=yt();function Dh(e){let t=e.getSchema(),r=t?t.getTypeMap():Object.create(null),n=Object.create(null);for(let s of e.getDocument().definitions)(0,Na.isTypeDefinitionNode)(s)&&(n[s.name.value]=!0);let i=[...Object.keys(r),...Object.keys(n)];return{NamedType(s,a,o,u,c){let l=s.name.value;if(!r[l]&&!n[l]){var f;let d=(f=c[2])!==null&&f!==void 0?f:o,y=d!=null&&Ah(d);if(y&&ml.includes(l))return;let T=(0,Oh.suggestionList)(l,y?ml.concat(i):i);e.reportError(new Ih.GraphQLError(`Unknown type "${l}".`+(0,Nh.didYouMean)(T),{nodes:s}))}}}}var ml=[...Sh.specifiedScalarTypes,...Lh.introspectionTypes].map(e=>e.name);function Ah(e){return"kind"in e&&((0,Na.isTypeSystemDefinitionNode)(e)||(0,Na.isTypeSystemExtensionNode)(e))}});var Sa=h(La=>{"use strict";Object.defineProperty(La,"__esModule",{value:!0});La.LoneAnonymousOperationRule=Ph;var Rh=q(),jh=H();function Ph(e){let t=0;return{Document(r){t=r.definitions.filter(n=>n.kind===jh.Kind.OPERATION_DEFINITION).length},OperationDefinition(r){!r.name&&t>1&&e.reportError(new Rh.GraphQLError("This anonymous operation must be the only defined operation.",{nodes:r}))}}}});var Aa=h(Da=>{"use strict";Object.defineProperty(Da,"__esModule",{value:!0});Da.LoneSchemaDefinitionRule=wh;var yl=q();function wh(e){var t,r,n;let i=e.getSchema(),s=(t=(r=(n=i?.astNode)!==null&&n!==void 0?n:i?.getQueryType())!==null&&r!==void 0?r:i?.getMutationType())!==null&&t!==void 0?t:i?.getSubscriptionType(),a=0;return{SchemaDefinition(o){if(s){e.reportError(new yl.GraphQLError("Cannot define a new schema within a schema extension.",{nodes:o}));return}a>0&&e.reportError(new yl.GraphQLError("Must provide only one schema definition.",{nodes:o})),++a}}}});var ja=h(Ra=>{"use strict";Object.defineProperty(Ra,"__esModule",{value:!0});Ra.NoFragmentCyclesRule=kh;var Fh=q();function kh(e){let t=Object.create(null),r=[],n=Object.create(null);return{OperationDefinition:()=>!1,FragmentDefinition(s){return i(s),!1}};function i(s){if(t[s.name.value])return;let a=s.name.value;t[a]=!0;let o=e.getFragmentSpreads(s.selectionSet);if(o.length!==0){n[a]=r.length;for(let u of o){let c=u.name.value,l=n[c];if(r.push(u),l===void 0){let f=e.getFragment(c);f&&i(f)}else{let f=r.slice(l),d=f.slice(0,-1).map(y=>'"'+y.name.value+'"').join(", ");e.reportError(new Fh.GraphQLError(`Cannot spread fragment "${c}" within itself`+(d!==""?` via ${d}.`:"."),{nodes:f}))}r.pop()}n[a]=void 0}}}});var wa=h(Pa=>{"use strict";Object.defineProperty(Pa,"__esModule",{value:!0});Pa.NoUndefinedVariablesRule=xh;var qh=q();function xh(e){let t=Object.create(null);return{OperationDefinition:{enter(){t=Object.create(null)},leave(r){let n=e.getRecursiveVariableUsages(r);for(let{node:i}of n){let s=i.name.value;t[s]!==!0&&e.reportError(new qh.GraphQLError(r.name?`Variable "$${s}" is not defined by operation "${r.name.value}".`:`Variable "$${s}" is not defined.`,{nodes:[i,r]}))}}},VariableDefinition(r){t[r.variable.name.value]=!0}}}});var ka=h(Fa=>{"use strict";Object.defineProperty(Fa,"__esModule",{value:!0});Fa.NoUnusedFragmentsRule=Mh;var Vh=q();function Mh(e){let t=[],r=[];return{OperationDefinition(n){return t.push(n),!1},FragmentDefinition(n){return r.push(n),!1},Document:{leave(){let n=Object.create(null);for(let i of t)for(let s of e.getRecursivelyReferencedFragments(i))n[s.name.value]=!0;for(let i of r){let s=i.name.value;n[s]!==!0&&e.reportError(new Vh.GraphQLError(`Fragment "${s}" is never used.`,{nodes:i}))}}}}}});var xa=h(qa=>{"use strict";Object.defineProperty(qa,"__esModule",{value:!0});qa.NoUnusedVariablesRule=Ch;var Gh=q();function Ch(e){let t=[];return{OperationDefinition:{enter(){t=[]},leave(r){let n=Object.create(null),i=e.getRecursiveVariableUsages(r);for(let{node:s}of i)n[s.name.value]=!0;for(let s of t){let a=s.variable.name.value;n[a]!==!0&&e.reportError(new Gh.GraphQLError(r.name?`Variable "$${a}" is never used in operation "${r.name.value}".`:`Variable "$${a}" is never used.`,{nodes:s}))}}},VariableDefinition(r){t.push(r)}}}});var Ga=h(Ma=>{"use strict";Object.defineProperty(Ma,"__esModule",{value:!0});Ma.sortValueNode=Va;var Qh=$r(),It=H();function Va(e){switch(e.kind){case It.Kind.OBJECT:return{...e,fields:Uh(e.fields)};case It.Kind.LIST:return{...e,values:e.values.map(Va)};case It.Kind.INT:case It.Kind.FLOAT:case It.Kind.STRING:case It.Kind.BOOLEAN:case It.Kind.NULL:case It.Kind.ENUM:case It.Kind.VARIABLE:return e}}function Uh(e){return e.map(t=>({...t,value:Va(t.value)})).sort((t,r)=>(0,Qh.naturalCompare)(t.name.value,r.name.value))}});var Ya=h(Ba=>{"use strict";Object.defineProperty(Ba,"__esModule",{value:!0});Ba.OverlappingFieldsCanBeMergedRule=Yh;var hl=re(),Kh=q(),Ca=H(),$h=Ce(),Me=Y(),Bh=Ga(),Tl=ht();function bl(e){return Array.isArray(e)?e.map(([t,r])=>`subfields "${t}" conflict because `+bl(r)).join(" and "):e}function Yh(e){let t=new Ka,r=new Map;return{SelectionSet(n){let i=Jh(e,r,t,e.getParentType(),n);for(let[[s,a],o,u]of i){let c=bl(a);e.reportError(new Kh.GraphQLError(`Fields "${s}" conflict because ${c}. Use different aliases on the fields to fetch both if this was intentional.`,{nodes:o.concat(u)}))}}}}function Jh(e,t,r,n,i){let s=[],[a,o]=Ei(e,t,n,i);if(Hh(e,s,t,r,a),o.length!==0)for(let u=0;u1)for(let o=0;o[s.value,a]));return r.every(s=>{let a=s.value,o=i.get(s.name.value);return o===void 0?!1:vl(a)===vl(o)})}function vl(e){return(0,$h.print)((0,Bh.sortValueNode)(e))}function Qa(e,t){return(0,Me.isListType)(e)?(0,Me.isListType)(t)?Qa(e.ofType,t.ofType):!0:(0,Me.isListType)(t)?!0:(0,Me.isNonNullType)(e)?(0,Me.isNonNullType)(t)?Qa(e.ofType,t.ofType):!0:(0,Me.isNonNullType)(t)?!0:(0,Me.isLeafType)(e)||(0,Me.isLeafType)(t)?e!==t:!1}function Ei(e,t,r,n){let i=t.get(n);if(i)return i;let s=Object.create(null),a=Object.create(null);El(e,r,n,s,a);let o=[s,Object.keys(a)];return t.set(n,o),o}function Ua(e,t,r){let n=t.get(r.selectionSet);if(n)return n;let i=(0,Tl.typeFromAST)(e.getSchema(),r.typeCondition);return Ei(e,t,i,r.selectionSet)}function El(e,t,r,n,i){for(let s of r.selections)switch(s.kind){case Ca.Kind.FIELD:{let a=s.name.value,o;((0,Me.isObjectType)(t)||(0,Me.isInterfaceType)(t))&&(o=t.getFields()[a]);let u=s.alias?s.alias.value:a;n[u]||(n[u]=[]),n[u].push([t,s,o]);break}case Ca.Kind.FRAGMENT_SPREAD:i[s.name.value]=!0;break;case Ca.Kind.INLINE_FRAGMENT:{let a=s.typeCondition,o=a?(0,Tl.typeFromAST)(e.getSchema(),a):t;El(e,o,s.selectionSet,n,i);break}}}function Wh(e,t,r,n){if(e.length>0)return[[t,e.map(([i])=>i)],[r,...e.map(([,i])=>i).flat()],[n,...e.map(([,,i])=>i).flat()]]}var Ka=class{constructor(){this._data=new Map}has(t,r,n){var i;let[s,a]=t{"use strict";Object.defineProperty(Xa,"__esModule",{value:!0});Xa.PossibleFragmentSpreadsRule=ev;var _i=re(),_l=q(),Ja=Y(),Nl=zr(),Zh=ht();function ev(e){return{InlineFragment(t){let r=e.getType(),n=e.getParentType();if((0,Ja.isCompositeType)(r)&&(0,Ja.isCompositeType)(n)&&!(0,Nl.doTypesOverlap)(e.getSchema(),r,n)){let i=(0,_i.inspect)(n),s=(0,_i.inspect)(r);e.reportError(new _l.GraphQLError(`Fragment cannot be spread here as objects of type "${i}" can never be of type "${s}".`,{nodes:t}))}},FragmentSpread(t){let r=t.name.value,n=tv(e,r),i=e.getParentType();if(n&&i&&!(0,Nl.doTypesOverlap)(e.getSchema(),n,i)){let s=(0,_i.inspect)(i),a=(0,_i.inspect)(n);e.reportError(new _l.GraphQLError(`Fragment "${r}" cannot be spread here as objects of type "${s}" can never be of type "${a}".`,{nodes:t}))}}}}function tv(e,t){let r=e.getFragment(t);if(r){let n=(0,Zh.typeFromAST)(e.getSchema(),r.typeCondition);if((0,Ja.isCompositeType)(n))return n}}});var Wa=h(za=>{"use strict";Object.defineProperty(za,"__esModule",{value:!0});za.PossibleTypeExtensionsRule=sv;var rv=wt(),Il=re(),Ll=je(),nv=kt(),Ol=q(),le=H(),iv=tr(),Nr=Y();function sv(e){let t=e.getSchema(),r=Object.create(null);for(let i of e.getDocument().definitions)(0,iv.isTypeDefinitionNode)(i)&&(r[i.name.value]=i);return{ScalarTypeExtension:n,ObjectTypeExtension:n,InterfaceTypeExtension:n,UnionTypeExtension:n,EnumTypeExtension:n,InputObjectTypeExtension:n};function n(i){let s=i.name.value,a=r[s],o=t?.getType(s),u;if(a?u=av[a.kind]:o&&(u=ov(o)),u){if(u!==i.kind){let c=uv(i.kind);e.reportError(new Ol.GraphQLError(`Cannot extend non-${c} type "${s}".`,{nodes:a?[a,i]:i}))}}else{let c=Object.keys({...r,...t?.getTypeMap()}),l=(0,nv.suggestionList)(s,c);e.reportError(new Ol.GraphQLError(`Cannot extend type "${s}" because it is not defined.`+(0,rv.didYouMean)(l),{nodes:i.name}))}}}var av={[le.Kind.SCALAR_TYPE_DEFINITION]:le.Kind.SCALAR_TYPE_EXTENSION,[le.Kind.OBJECT_TYPE_DEFINITION]:le.Kind.OBJECT_TYPE_EXTENSION,[le.Kind.INTERFACE_TYPE_DEFINITION]:le.Kind.INTERFACE_TYPE_EXTENSION,[le.Kind.UNION_TYPE_DEFINITION]:le.Kind.UNION_TYPE_EXTENSION,[le.Kind.ENUM_TYPE_DEFINITION]:le.Kind.ENUM_TYPE_EXTENSION,[le.Kind.INPUT_OBJECT_TYPE_DEFINITION]:le.Kind.INPUT_OBJECT_TYPE_EXTENSION};function ov(e){if((0,Nr.isScalarType)(e))return le.Kind.SCALAR_TYPE_EXTENSION;if((0,Nr.isObjectType)(e))return le.Kind.OBJECT_TYPE_EXTENSION;if((0,Nr.isInterfaceType)(e))return le.Kind.INTERFACE_TYPE_EXTENSION;if((0,Nr.isUnionType)(e))return le.Kind.UNION_TYPE_EXTENSION;if((0,Nr.isEnumType)(e))return le.Kind.ENUM_TYPE_EXTENSION;if((0,Nr.isInputObjectType)(e))return le.Kind.INPUT_OBJECT_TYPE_EXTENSION;(0,Ll.invariant)(!1,"Unexpected type: "+(0,Il.inspect)(e))}function uv(e){switch(e){case le.Kind.SCALAR_TYPE_EXTENSION:return"scalar";case le.Kind.OBJECT_TYPE_EXTENSION:return"object";case le.Kind.INTERFACE_TYPE_EXTENSION:return"interface";case le.Kind.UNION_TYPE_EXTENSION:return"union";case le.Kind.ENUM_TYPE_EXTENSION:return"enum";case le.Kind.INPUT_OBJECT_TYPE_EXTENSION:return"input object";default:(0,Ll.invariant)(!1,"Unexpected kind: "+(0,Il.inspect)(e))}}});var eo=h(Ni=>{"use strict";Object.defineProperty(Ni,"__esModule",{value:!0});Ni.ProvidedRequiredArgumentsOnDirectivesRule=jl;Ni.ProvidedRequiredArgumentsRule=pv;var Dl=re(),Sl=Ft(),Al=q(),Rl=H(),cv=Ce(),Za=Y(),lv=Ve();function pv(e){return{...jl(e),Field:{leave(t){var r;let n=e.getFieldDef();if(!n)return!1;let i=new Set((r=t.arguments)===null||r===void 0?void 0:r.map(s=>s.name.value));for(let s of n.args)if(!i.has(s.name)&&(0,Za.isRequiredArgument)(s)){let a=(0,Dl.inspect)(s.type);e.reportError(new Al.GraphQLError(`Field "${n.name}" argument "${s.name}" of type "${a}" is required, but it was not provided.`,{nodes:t}))}}}}}function jl(e){var t;let r=Object.create(null),n=e.getSchema(),i=(t=n?.getDirectives())!==null&&t!==void 0?t:lv.specifiedDirectives;for(let o of i)r[o.name]=(0,Sl.keyMap)(o.args.filter(Za.isRequiredArgument),u=>u.name);let s=e.getDocument().definitions;for(let o of s)if(o.kind===Rl.Kind.DIRECTIVE_DEFINITION){var a;let u=(a=o.arguments)!==null&&a!==void 0?a:[];r[o.name.value]=(0,Sl.keyMap)(u.filter(dv),c=>c.name.value)}return{Directive:{leave(o){let u=o.name.value,c=r[u];if(c){var l;let f=(l=o.arguments)!==null&&l!==void 0?l:[],d=new Set(f.map(y=>y.name.value));for(let[y,T]of Object.entries(c))if(!d.has(y)){let I=(0,Za.isType)(T.type)?(0,Dl.inspect)(T.type):(0,cv.print)(T.type);e.reportError(new Al.GraphQLError(`Directive "@${u}" argument "${y}" of type "${I}" is required, but it was not provided.`,{nodes:o}))}}}}}}function dv(e){return e.type.kind===Rl.Kind.NON_NULL_TYPE&&e.defaultValue==null}});var ro=h(to=>{"use strict";Object.defineProperty(to,"__esModule",{value:!0});to.ScalarLeafsRule=fv;var Pl=re(),wl=q(),Fl=Y();function fv(e){return{Field(t){let r=e.getType(),n=t.selectionSet;if(r){if((0,Fl.isLeafType)((0,Fl.getNamedType)(r))){if(n){let i=t.name.value,s=(0,Pl.inspect)(r);e.reportError(new wl.GraphQLError(`Field "${i}" must not have a selection since type "${s}" has no subfields.`,{nodes:n}))}}else if(!n){let i=t.name.value,s=(0,Pl.inspect)(r);e.reportError(new wl.GraphQLError(`Field "${i}" of type "${s}" must have a selection of subfields. Did you mean "${i} { ... }"?`,{nodes:t}))}}}}}});var io=h(no=>{"use strict";Object.defineProperty(no,"__esModule",{value:!0});no.printPathArray=mv;function mv(e){return e.map(t=>typeof t=="number"?"["+t.toString()+"]":"."+t).join("")}});var pn=h(Oi=>{"use strict";Object.defineProperty(Oi,"__esModule",{value:!0});Oi.addPath=yv;Oi.pathToArray=hv;function yv(e,t,r){return{prev:e,key:t,typename:r}}function hv(e){let t=[],r=e;for(;r;)t.push(r.key),r=r.prev;return t.reverse()}});var ao=h(so=>{"use strict";Object.defineProperty(so,"__esModule",{value:!0});so.coerceInputValue=Nv;var vv=wt(),Ii=re(),Tv=je(),bv=fi(),gv=pt(),Lt=pn(),Ev=io(),_v=kt(),rr=q(),dn=Y();function Nv(e,t,r=Ov){return fn(e,t,r,void 0)}function Ov(e,t,r){let n="Invalid value "+(0,Ii.inspect)(t);throw e.length>0&&(n+=` at "value${(0,Ev.printPathArray)(e)}"`),r.message=n+": "+r.message,r}function fn(e,t,r,n){if((0,dn.isNonNullType)(t)){if(e!=null)return fn(e,t.ofType,r,n);r((0,Lt.pathToArray)(n),e,new rr.GraphQLError(`Expected non-nullable type "${(0,Ii.inspect)(t)}" not to be null.`));return}if(e==null)return null;if((0,dn.isListType)(t)){let i=t.ofType;return(0,bv.isIterableObject)(e)?Array.from(e,(s,a)=>{let o=(0,Lt.addPath)(n,a,void 0);return fn(s,i,r,o)}):[fn(e,i,r,n)]}if((0,dn.isInputObjectType)(t)){if(!(0,gv.isObjectLike)(e)){r((0,Lt.pathToArray)(n),e,new rr.GraphQLError(`Expected type "${t.name}" to be an object.`));return}let i={},s=t.getFields();for(let a of Object.values(s)){let o=e[a.name];if(o===void 0){if(a.defaultValue!==void 0)i[a.name]=a.defaultValue;else if((0,dn.isNonNullType)(a.type)){let u=(0,Ii.inspect)(a.type);r((0,Lt.pathToArray)(n),e,new rr.GraphQLError(`Field "${a.name}" of required type "${u}" was not provided.`))}continue}i[a.name]=fn(o,a.type,r,(0,Lt.addPath)(n,a.name,t.name))}for(let a of Object.keys(e))if(!s[a]){let o=(0,_v.suggestionList)(a,Object.keys(t.getFields()));r((0,Lt.pathToArray)(n),e,new rr.GraphQLError(`Field "${a}" is not defined by type "${t.name}".`+(0,vv.didYouMean)(o)))}return i}if((0,dn.isLeafType)(t)){let i;try{i=t.parseValue(e)}catch(s){s instanceof rr.GraphQLError?r((0,Lt.pathToArray)(n),e,s):r((0,Lt.pathToArray)(n),e,new rr.GraphQLError(`Expected type "${t.name}". `+s.message,{originalError:s}));return}return i===void 0&&r((0,Lt.pathToArray)(n),e,new rr.GraphQLError(`Expected type "${t.name}".`)),i}(0,Tv.invariant)(!1,"Unexpected input type: "+(0,Ii.inspect)(t))}});var yn=h(oo=>{"use strict";Object.defineProperty(oo,"__esModule",{value:!0});oo.valueFromAST=mn;var Iv=re(),Lv=je(),Sv=Ft(),Or=H(),nr=Y();function mn(e,t,r){if(e){if(e.kind===Or.Kind.VARIABLE){let n=e.name.value;if(r==null||r[n]===void 0)return;let i=r[n];return i===null&&(0,nr.isNonNullType)(t)?void 0:i}if((0,nr.isNonNullType)(t))return e.kind===Or.Kind.NULL?void 0:mn(e,t.ofType,r);if(e.kind===Or.Kind.NULL)return null;if((0,nr.isListType)(t)){let n=t.ofType;if(e.kind===Or.Kind.LIST){let s=[];for(let a of e.values)if(kl(a,r)){if((0,nr.isNonNullType)(n))return;s.push(null)}else{let o=mn(a,n,r);if(o===void 0)return;s.push(o)}return s}let i=mn(e,n,r);return i===void 0?void 0:[i]}if((0,nr.isInputObjectType)(t)){if(e.kind!==Or.Kind.OBJECT)return;let n=Object.create(null),i=(0,Sv.keyMap)(e.fields,s=>s.name.value);for(let s of Object.values(t.getFields())){let a=i[s.name];if(!a||kl(a.value,r)){if(s.defaultValue!==void 0)n[s.name]=s.defaultValue;else if((0,nr.isNonNullType)(s.type))return;continue}let o=mn(a.value,s.type,r);if(o===void 0)return;n[s.name]=o}return n}if((0,nr.isLeafType)(t)){let n;try{n=t.parseLiteral(e,r)}catch{return}return n===void 0?void 0:n}(0,Lv.invariant)(!1,"Unexpected input type: "+(0,Iv.inspect)(t))}}function kl(e,t){return e.kind===Or.Kind.VARIABLE&&(t==null||t[e.name.value]===void 0)}});var Sr=h(hn=>{"use strict";Object.defineProperty(hn,"__esModule",{value:!0});hn.getArgumentValues=Ml;hn.getDirectiveValues=Fv;hn.getVariableValues=Pv;var Ir=re(),Dv=Ft(),Av=io(),St=q(),ql=H(),xl=Ce(),Lr=Y(),Rv=ao(),jv=ht(),Vl=yn();function Pv(e,t,r,n){let i=[],s=n?.maxErrors;try{let a=wv(e,t,r,o=>{if(s!=null&&i.length>=s)throw new St.GraphQLError("Too many errors processing variables, error limit reached. Execution aborted.");i.push(o)});if(i.length===0)return{coerced:a}}catch(a){i.push(a)}return{errors:i}}function wv(e,t,r,n){let i={};for(let s of t){let a=s.variable.name.value,o=(0,jv.typeFromAST)(e,s.type);if(!(0,Lr.isInputType)(o)){let c=(0,xl.print)(s.type);n(new St.GraphQLError(`Variable "$${a}" expected value of type "${c}" which cannot be used as an input type.`,{nodes:s.type}));continue}if(!Gl(r,a)){if(s.defaultValue)i[a]=(0,Vl.valueFromAST)(s.defaultValue,o);else if((0,Lr.isNonNullType)(o)){let c=(0,Ir.inspect)(o);n(new St.GraphQLError(`Variable "$${a}" of required type "${c}" was not provided.`,{nodes:s}))}continue}let u=r[a];if(u===null&&(0,Lr.isNonNullType)(o)){let c=(0,Ir.inspect)(o);n(new St.GraphQLError(`Variable "$${a}" of non-null type "${c}" must not be null.`,{nodes:s}));continue}i[a]=(0,Rv.coerceInputValue)(u,o,(c,l,f)=>{let d=`Variable "$${a}" got invalid value `+(0,Ir.inspect)(l);c.length>0&&(d+=` at "${a}${(0,Av.printPathArray)(c)}"`),n(new St.GraphQLError(d+"; "+f.message,{nodes:s,originalError:f}))})}return i}function Ml(e,t,r){var n;let i={},s=(n=t.arguments)!==null&&n!==void 0?n:[],a=(0,Dv.keyMap)(s,o=>o.name.value);for(let o of e.args){let u=o.name,c=o.type,l=a[u];if(!l){if(o.defaultValue!==void 0)i[u]=o.defaultValue;else if((0,Lr.isNonNullType)(c))throw new St.GraphQLError(`Argument "${u}" of required type "${(0,Ir.inspect)(c)}" was not provided.`,{nodes:t});continue}let f=l.value,d=f.kind===ql.Kind.NULL;if(f.kind===ql.Kind.VARIABLE){let T=f.name.value;if(r==null||!Gl(r,T)){if(o.defaultValue!==void 0)i[u]=o.defaultValue;else if((0,Lr.isNonNullType)(c))throw new St.GraphQLError(`Argument "${u}" of required type "${(0,Ir.inspect)(c)}" was provided the variable "$${T}" which was not provided a runtime value.`,{nodes:f});continue}d=r[T]==null}if(d&&(0,Lr.isNonNullType)(c))throw new St.GraphQLError(`Argument "${u}" of non-null type "${(0,Ir.inspect)(c)}" must not be null.`,{nodes:f});let y=(0,Vl.valueFromAST)(f,c,r);if(y===void 0)throw new St.GraphQLError(`Argument "${u}" has invalid value ${(0,xl.print)(f)}.`,{nodes:f});i[u]=y}return i}function Fv(e,t,r){var n;let i=(n=t.directives)===null||n===void 0?void 0:n.find(s=>s.name.value===e.name);if(i)return Ml(e,i,r)}function Gl(e,t){return Object.prototype.hasOwnProperty.call(e,t)}});var Di=h(Si=>{"use strict";Object.defineProperty(Si,"__esModule",{value:!0});Si.collectFields=xv;Si.collectSubfields=Vv;var uo=H(),kv=Y(),Cl=Ve(),qv=ht(),Ql=Sr();function xv(e,t,r,n,i){let s=new Map;return Li(e,t,r,n,i,s,new Set),s}function Vv(e,t,r,n,i){let s=new Map,a=new Set;for(let o of i)o.selectionSet&&Li(e,t,r,n,o.selectionSet,s,a);return s}function Li(e,t,r,n,i,s,a){for(let o of i.selections)switch(o.kind){case uo.Kind.FIELD:{if(!co(r,o))continue;let u=Mv(o),c=s.get(u);c!==void 0?c.push(o):s.set(u,[o]);break}case uo.Kind.INLINE_FRAGMENT:{if(!co(r,o)||!Ul(e,o,n))continue;Li(e,t,r,n,o.selectionSet,s,a);break}case uo.Kind.FRAGMENT_SPREAD:{let u=o.name.value;if(a.has(u)||!co(r,o))continue;a.add(u);let c=t[u];if(!c||!Ul(e,c,n))continue;Li(e,t,r,n,c.selectionSet,s,a);break}}}function co(e,t){let r=(0,Ql.getDirectiveValues)(Cl.GraphQLSkipDirective,t,e);if(r?.if===!0)return!1;let n=(0,Ql.getDirectiveValues)(Cl.GraphQLIncludeDirective,t,e);return n?.if!==!1}function Ul(e,t,r){let n=t.typeCondition;if(!n)return!0;let i=(0,qv.typeFromAST)(e,n);return i===r?!0:(0,kv.isAbstractType)(i)?e.isSubType(i,r):!1}function Mv(e){return e.alias?e.alias.value:e.name.value}});var po=h(lo=>{"use strict";Object.defineProperty(lo,"__esModule",{value:!0});lo.SingleFieldSubscriptionsRule=Qv;var Kl=q(),Gv=H(),Cv=Di();function Qv(e){return{OperationDefinition(t){if(t.operation==="subscription"){let r=e.getSchema(),n=r.getSubscriptionType();if(n){let i=t.name?t.name.value:null,s=Object.create(null),a=e.getDocument(),o=Object.create(null);for(let c of a.definitions)c.kind===Gv.Kind.FRAGMENT_DEFINITION&&(o[c.name.value]=c);let u=(0,Cv.collectFields)(r,o,s,n,t.selectionSet);if(u.size>1){let f=[...u.values()].slice(1).flat();e.reportError(new Kl.GraphQLError(i!=null?`Subscription "${i}" must select only one top level field.`:"Anonymous Subscription must select only one top level field.",{nodes:f}))}for(let c of u.values())c[0].name.value.startsWith("__")&&e.reportError(new Kl.GraphQLError(i!=null?`Subscription "${i}" must not select an introspection top level field.`:"Anonymous Subscription must not select an introspection top level field.",{nodes:c}))}}}}}});var Ai=h(fo=>{"use strict";Object.defineProperty(fo,"__esModule",{value:!0});fo.groupBy=Uv;function Uv(e,t){let r=new Map;for(let n of e){let i=t(n),s=r.get(i);s===void 0?r.set(i,[n]):s.push(n)}return r}});var yo=h(mo=>{"use strict";Object.defineProperty(mo,"__esModule",{value:!0});mo.UniqueArgumentDefinitionNamesRule=Bv;var Kv=Ai(),$v=q();function Bv(e){return{DirectiveDefinition(n){var i;let s=(i=n.arguments)!==null&&i!==void 0?i:[];return r(`@${n.name.value}`,s)},InterfaceTypeDefinition:t,InterfaceTypeExtension:t,ObjectTypeDefinition:t,ObjectTypeExtension:t};function t(n){var i;let s=n.name.value,a=(i=n.fields)!==null&&i!==void 0?i:[];for(let u of a){var o;let c=u.name.value,l=(o=u.arguments)!==null&&o!==void 0?o:[];r(`${s}.${c}`,l)}return!1}function r(n,i){let s=(0,Kv.groupBy)(i,a=>a.name.value);for(let[a,o]of s)o.length>1&&e.reportError(new $v.GraphQLError(`Argument "${n}(${a}:)" can only be defined once.`,{nodes:o.map(u=>u.name)}));return!1}}});var vo=h(ho=>{"use strict";Object.defineProperty(ho,"__esModule",{value:!0});ho.UniqueArgumentNamesRule=Xv;var Yv=Ai(),Jv=q();function Xv(e){return{Field:t,Directive:t};function t(r){var n;let i=(n=r.arguments)!==null&&n!==void 0?n:[],s=(0,Yv.groupBy)(i,a=>a.name.value);for(let[a,o]of s)o.length>1&&e.reportError(new Jv.GraphQLError(`There can be only one argument named "${a}".`,{nodes:o.map(u=>u.name)}))}}});var bo=h(To=>{"use strict";Object.defineProperty(To,"__esModule",{value:!0});To.UniqueDirectiveNamesRule=Hv;var $l=q();function Hv(e){let t=Object.create(null),r=e.getSchema();return{DirectiveDefinition(n){let i=n.name.value;if(r!=null&&r.getDirective(i)){e.reportError(new $l.GraphQLError(`Directive "@${i}" already exists in the schema. It cannot be redefined.`,{nodes:n.name}));return}return t[i]?e.reportError(new $l.GraphQLError(`There can be only one directive named "@${i}".`,{nodes:[t[i],n.name]})):t[i]=n.name,!1}}}});var _o=h(Eo=>{"use strict";Object.defineProperty(Eo,"__esModule",{value:!0});Eo.UniqueDirectivesPerLocationRule=Zv;var zv=q(),go=H(),Bl=tr(),Wv=Ve();function Zv(e){let t=Object.create(null),r=e.getSchema(),n=r?r.getDirectives():Wv.specifiedDirectives;for(let o of n)t[o.name]=!o.isRepeatable;let i=e.getDocument().definitions;for(let o of i)o.kind===go.Kind.DIRECTIVE_DEFINITION&&(t[o.name.value]=!o.repeatable);let s=Object.create(null),a=Object.create(null);return{enter(o){if(!("directives"in o)||!o.directives)return;let u;if(o.kind===go.Kind.SCHEMA_DEFINITION||o.kind===go.Kind.SCHEMA_EXTENSION)u=s;else if((0,Bl.isTypeDefinitionNode)(o)||(0,Bl.isTypeExtensionNode)(o)){let c=o.name.value;u=a[c],u===void 0&&(a[c]=u=Object.create(null))}else u=Object.create(null);for(let c of o.directives){let l=c.name.value;t[l]&&(u[l]?e.reportError(new zv.GraphQLError(`The directive "@${l}" can only be used once at this location.`,{nodes:[u[l],c]})):u[l]=c)}}}}});var Oo=h(No=>{"use strict";Object.defineProperty(No,"__esModule",{value:!0});No.UniqueEnumValueNamesRule=tT;var Yl=q(),eT=Y();function tT(e){let t=e.getSchema(),r=t?t.getTypeMap():Object.create(null),n=Object.create(null);return{EnumTypeDefinition:i,EnumTypeExtension:i};function i(s){var a;let o=s.name.value;n[o]||(n[o]=Object.create(null));let u=(a=s.values)!==null&&a!==void 0?a:[],c=n[o];for(let l of u){let f=l.name.value,d=r[o];(0,eT.isEnumType)(d)&&d.getValue(f)?e.reportError(new Yl.GraphQLError(`Enum value "${o}.${f}" already exists in the schema. It cannot also be defined in this type extension.`,{nodes:l.name})):c[f]?e.reportError(new Yl.GraphQLError(`Enum value "${o}.${f}" can only be defined once.`,{nodes:[c[f],l.name]})):c[f]=l.name}return!1}}});var So=h(Lo=>{"use strict";Object.defineProperty(Lo,"__esModule",{value:!0});Lo.UniqueFieldDefinitionNamesRule=rT;var Jl=q(),Io=Y();function rT(e){let t=e.getSchema(),r=t?t.getTypeMap():Object.create(null),n=Object.create(null);return{InputObjectTypeDefinition:i,InputObjectTypeExtension:i,InterfaceTypeDefinition:i,InterfaceTypeExtension:i,ObjectTypeDefinition:i,ObjectTypeExtension:i};function i(s){var a;let o=s.name.value;n[o]||(n[o]=Object.create(null));let u=(a=s.fields)!==null&&a!==void 0?a:[],c=n[o];for(let l of u){let f=l.name.value;nT(r[o],f)?e.reportError(new Jl.GraphQLError(`Field "${o}.${f}" already exists in the schema. It cannot also be defined in this type extension.`,{nodes:l.name})):c[f]?e.reportError(new Jl.GraphQLError(`Field "${o}.${f}" can only be defined once.`,{nodes:[c[f],l.name]})):c[f]=l.name}return!1}}function nT(e,t){return(0,Io.isObjectType)(e)||(0,Io.isInterfaceType)(e)||(0,Io.isInputObjectType)(e)?e.getFields()[t]!=null:!1}});var Ao=h(Do=>{"use strict";Object.defineProperty(Do,"__esModule",{value:!0});Do.UniqueFragmentNamesRule=sT;var iT=q();function sT(e){let t=Object.create(null);return{OperationDefinition:()=>!1,FragmentDefinition(r){let n=r.name.value;return t[n]?e.reportError(new iT.GraphQLError(`There can be only one fragment named "${n}".`,{nodes:[t[n],r.name]})):t[n]=r.name,!1}}}});var jo=h(Ro=>{"use strict";Object.defineProperty(Ro,"__esModule",{value:!0});Ro.UniqueInputFieldNamesRule=uT;var aT=je(),oT=q();function uT(e){let t=[],r=Object.create(null);return{ObjectValue:{enter(){t.push(r),r=Object.create(null)},leave(){let n=t.pop();n||(0,aT.invariant)(!1),r=n}},ObjectField(n){let i=n.name.value;r[i]?e.reportError(new oT.GraphQLError(`There can be only one input field named "${i}".`,{nodes:[r[i],n.name]})):r[i]=n.name}}}});var wo=h(Po=>{"use strict";Object.defineProperty(Po,"__esModule",{value:!0});Po.UniqueOperationNamesRule=lT;var cT=q();function lT(e){let t=Object.create(null);return{OperationDefinition(r){let n=r.name;return n&&(t[n.value]?e.reportError(new cT.GraphQLError(`There can be only one operation named "${n.value}".`,{nodes:[t[n.value],n]})):t[n.value]=n),!1},FragmentDefinition:()=>!1}}});var ko=h(Fo=>{"use strict";Object.defineProperty(Fo,"__esModule",{value:!0});Fo.UniqueOperationTypesRule=pT;var Xl=q();function pT(e){let t=e.getSchema(),r=Object.create(null),n=t?{query:t.getQueryType(),mutation:t.getMutationType(),subscription:t.getSubscriptionType()}:{};return{SchemaDefinition:i,SchemaExtension:i};function i(s){var a;let o=(a=s.operationTypes)!==null&&a!==void 0?a:[];for(let u of o){let c=u.operation,l=r[c];n[c]?e.reportError(new Xl.GraphQLError(`Type for ${c} already defined in the schema. It cannot be redefined.`,{nodes:u})):l?e.reportError(new Xl.GraphQLError(`There can be only one ${c} type in schema.`,{nodes:[l,u]})):r[c]=u}return!1}}});var xo=h(qo=>{"use strict";Object.defineProperty(qo,"__esModule",{value:!0});qo.UniqueTypeNamesRule=dT;var Hl=q();function dT(e){let t=Object.create(null),r=e.getSchema();return{ScalarTypeDefinition:n,ObjectTypeDefinition:n,InterfaceTypeDefinition:n,UnionTypeDefinition:n,EnumTypeDefinition:n,InputObjectTypeDefinition:n};function n(i){let s=i.name.value;if(r!=null&&r.getType(s)){e.reportError(new Hl.GraphQLError(`Type "${s}" already exists in the schema. It cannot also be defined in this type definition.`,{nodes:i.name}));return}return t[s]?e.reportError(new Hl.GraphQLError(`There can be only one type named "${s}".`,{nodes:[t[s],i.name]})):t[s]=i.name,!1}}});var Mo=h(Vo=>{"use strict";Object.defineProperty(Vo,"__esModule",{value:!0});Vo.UniqueVariableNamesRule=yT;var fT=Ai(),mT=q();function yT(e){return{OperationDefinition(t){var r;let n=(r=t.variableDefinitions)!==null&&r!==void 0?r:[],i=(0,fT.groupBy)(n,s=>s.variable.name.value);for(let[s,a]of i)a.length>1&&e.reportError(new mT.GraphQLError(`There can be only one variable named "$${s}".`,{nodes:a.map(o=>o.variable.name)}))}}}});var Co=h(Go=>{"use strict";Object.defineProperty(Go,"__esModule",{value:!0});Go.ValuesOfCorrectTypeRule=bT;var hT=wt(),vn=re(),vT=Ft(),TT=kt(),sr=q(),Ri=Ce(),vt=Y();function bT(e){return{ListValue(t){let r=(0,vt.getNullableType)(e.getParentInputType());if(!(0,vt.isListType)(r))return ir(e,t),!1},ObjectValue(t){let r=(0,vt.getNamedType)(e.getInputType());if(!(0,vt.isInputObjectType)(r))return ir(e,t),!1;let n=(0,vT.keyMap)(t.fields,i=>i.name.value);for(let i of Object.values(r.getFields()))if(!n[i.name]&&(0,vt.isRequiredInputField)(i)){let a=(0,vn.inspect)(i.type);e.reportError(new sr.GraphQLError(`Field "${r.name}.${i.name}" of required type "${a}" was not provided.`,{nodes:t}))}},ObjectField(t){let r=(0,vt.getNamedType)(e.getParentInputType());if(!e.getInputType()&&(0,vt.isInputObjectType)(r)){let i=(0,TT.suggestionList)(t.name.value,Object.keys(r.getFields()));e.reportError(new sr.GraphQLError(`Field "${t.name.value}" is not defined by type "${r.name}".`+(0,hT.didYouMean)(i),{nodes:t}))}},NullValue(t){let r=e.getInputType();(0,vt.isNonNullType)(r)&&e.reportError(new sr.GraphQLError(`Expected value of type "${(0,vn.inspect)(r)}", found ${(0,Ri.print)(t)}.`,{nodes:t}))},EnumValue:t=>ir(e,t),IntValue:t=>ir(e,t),FloatValue:t=>ir(e,t),StringValue:t=>ir(e,t),BooleanValue:t=>ir(e,t)}}function ir(e,t){let r=e.getInputType();if(!r)return;let n=(0,vt.getNamedType)(r);if(!(0,vt.isLeafType)(n)){let i=(0,vn.inspect)(r);e.reportError(new sr.GraphQLError(`Expected value of type "${i}", found ${(0,Ri.print)(t)}.`,{nodes:t}));return}try{if(n.parseLiteral(t,void 0)===void 0){let s=(0,vn.inspect)(r);e.reportError(new sr.GraphQLError(`Expected value of type "${s}", found ${(0,Ri.print)(t)}.`,{nodes:t}))}}catch(i){let s=(0,vn.inspect)(r);i instanceof sr.GraphQLError?e.reportError(i):e.reportError(new sr.GraphQLError(`Expected value of type "${s}", found ${(0,Ri.print)(t)}; `+i.message,{nodes:t,originalError:i}))}}});var Uo=h(Qo=>{"use strict";Object.defineProperty(Qo,"__esModule",{value:!0});Qo.VariablesAreInputTypesRule=OT;var gT=q(),ET=Ce(),_T=Y(),NT=ht();function OT(e){return{VariableDefinition(t){let r=(0,NT.typeFromAST)(e.getSchema(),t.type);if(r!==void 0&&!(0,_T.isInputType)(r)){let n=t.variable.name.value,i=(0,ET.print)(t.type);e.reportError(new gT.GraphQLError(`Variable "$${n}" cannot be non-input type "${i}".`,{nodes:t.type}))}}}}});var $o=h(Ko=>{"use strict";Object.defineProperty(Ko,"__esModule",{value:!0});Ko.VariablesInAllowedPositionRule=DT;var zl=re(),IT=q(),LT=H(),Wl=Y(),Zl=zr(),ST=ht();function DT(e){let t=Object.create(null);return{OperationDefinition:{enter(){t=Object.create(null)},leave(r){let n=e.getRecursiveVariableUsages(r);for(let{node:i,type:s,defaultValue:a}of n){let o=i.name.value,u=t[o];if(u&&s){let c=e.getSchema(),l=(0,ST.typeFromAST)(c,u.type);if(l&&!AT(c,l,u.defaultValue,s,a)){let f=(0,zl.inspect)(l),d=(0,zl.inspect)(s);e.reportError(new IT.GraphQLError(`Variable "$${o}" of type "${f}" used in position expecting type "${d}".`,{nodes:[u,i]}))}}}}},VariableDefinition(r){t[r.variable.name.value]=r}}}function AT(e,t,r,n,i){if((0,Wl.isNonNullType)(n)&&!(0,Wl.isNonNullType)(t)){if(!(r!=null&&r.kind!==LT.Kind.NULL)&&!(i!==void 0))return!1;let o=n.ofType;return(0,Zl.isTypeSubTypeOf)(e,t,o)}return(0,Zl.isTypeSubTypeOf)(e,t,n)}});var Bo=h(Dr=>{"use strict";Object.defineProperty(Dr,"__esModule",{value:!0});Dr.specifiedSDLRules=Dr.specifiedRules=void 0;var RT=pa(),jT=fa(),PT=ya(),ep=ha(),tp=ga(),wT=_a(),rp=Ia(),FT=Sa(),kT=Aa(),qT=ja(),xT=wa(),VT=ka(),MT=xa(),GT=Ya(),CT=Ha(),QT=Wa(),np=eo(),UT=ro(),KT=po(),$T=yo(),ip=vo(),BT=bo(),sp=_o(),YT=Oo(),JT=So(),XT=Ao(),ap=jo(),HT=wo(),zT=ko(),WT=xo(),ZT=Mo(),eb=Co(),tb=Uo(),rb=$o(),nb=Object.freeze([RT.ExecutableDefinitionsRule,HT.UniqueOperationNamesRule,FT.LoneAnonymousOperationRule,KT.SingleFieldSubscriptionsRule,rp.KnownTypeNamesRule,PT.FragmentsOnCompositeTypesRule,tb.VariablesAreInputTypesRule,UT.ScalarLeafsRule,jT.FieldsOnCorrectTypeRule,XT.UniqueFragmentNamesRule,wT.KnownFragmentNamesRule,VT.NoUnusedFragmentsRule,CT.PossibleFragmentSpreadsRule,qT.NoFragmentCyclesRule,ZT.UniqueVariableNamesRule,xT.NoUndefinedVariablesRule,MT.NoUnusedVariablesRule,tp.KnownDirectivesRule,sp.UniqueDirectivesPerLocationRule,ep.KnownArgumentNamesRule,ip.UniqueArgumentNamesRule,eb.ValuesOfCorrectTypeRule,np.ProvidedRequiredArgumentsRule,rb.VariablesInAllowedPositionRule,GT.OverlappingFieldsCanBeMergedRule,ap.UniqueInputFieldNamesRule]);Dr.specifiedRules=nb;var ib=Object.freeze([kT.LoneSchemaDefinitionRule,zT.UniqueOperationTypesRule,WT.UniqueTypeNamesRule,YT.UniqueEnumValueNamesRule,JT.UniqueFieldDefinitionNamesRule,$T.UniqueArgumentDefinitionNamesRule,BT.UniqueDirectiveNamesRule,rp.KnownTypeNamesRule,tp.KnownDirectivesRule,sp.UniqueDirectivesPerLocationRule,QT.PossibleTypeExtensionsRule,ep.KnownArgumentNamesOnDirectivesRule,ip.UniqueArgumentNamesRule,ap.UniqueInputFieldNamesRule,np.ProvidedRequiredArgumentsOnDirectivesRule]);Dr.specifiedSDLRules=ib});var Xo=h(Gt=>{"use strict";Object.defineProperty(Gt,"__esModule",{value:!0});Gt.ValidationContext=Gt.SDLValidationContext=Gt.ASTValidationContext=void 0;var op=H(),sb=Yt(),up=vi(),Tn=class{constructor(t,r){this._ast=t,this._fragments=void 0,this._fragmentSpreads=new Map,this._recursivelyReferencedFragments=new Map,this._onError=r}get[Symbol.toStringTag](){return"ASTValidationContext"}reportError(t){this._onError(t)}getDocument(){return this._ast}getFragment(t){let r;if(this._fragments)r=this._fragments;else{r=Object.create(null);for(let n of this.getDocument().definitions)n.kind===op.Kind.FRAGMENT_DEFINITION&&(r[n.name.value]=n);this._fragments=r}return r[t]}getFragmentSpreads(t){let r=this._fragmentSpreads.get(t);if(!r){r=[];let n=[t],i;for(;i=n.pop();)for(let s of i.selections)s.kind===op.Kind.FRAGMENT_SPREAD?r.push(s):s.selectionSet&&n.push(s.selectionSet);this._fragmentSpreads.set(t,r)}return r}getRecursivelyReferencedFragments(t){let r=this._recursivelyReferencedFragments.get(t);if(!r){r=[];let n=Object.create(null),i=[t.selectionSet],s;for(;s=i.pop();)for(let a of this.getFragmentSpreads(s)){let o=a.name.value;if(n[o]!==!0){n[o]=!0;let u=this.getFragment(o);u&&(r.push(u),i.push(u.selectionSet))}}this._recursivelyReferencedFragments.set(t,r)}return r}};Gt.ASTValidationContext=Tn;var Yo=class extends Tn{constructor(t,r,n){super(t,n),this._schema=r}get[Symbol.toStringTag](){return"SDLValidationContext"}getSchema(){return this._schema}};Gt.SDLValidationContext=Yo;var Jo=class extends Tn{constructor(t,r,n,i){super(r,i),this._schema=t,this._typeInfo=n,this._variableUsages=new Map,this._recursiveVariableUsages=new Map}get[Symbol.toStringTag](){return"ValidationContext"}getSchema(){return this._schema}getVariableUsages(t){let r=this._variableUsages.get(t);if(!r){let n=[],i=new up.TypeInfo(this._schema);(0,sb.visit)(t,(0,up.visitWithTypeInfo)(i,{VariableDefinition:()=>!1,Variable(s){n.push({node:s,type:i.getInputType(),defaultValue:i.getDefaultValue()})}})),r=n,this._variableUsages.set(t,r)}return r}getRecursiveVariableUsages(t){let r=this._recursiveVariableUsages.get(t);if(!r){r=this.getVariableUsages(t);for(let n of this.getRecursivelyReferencedFragments(t))r=r.concat(this.getVariableUsages(n));this._recursiveVariableUsages.set(t,r)}return r}getType(){return this._typeInfo.getType()}getParentType(){return this._typeInfo.getParentType()}getInputType(){return this._typeInfo.getInputType()}getParentInputType(){return this._typeInfo.getParentInputType()}getFieldDef(){return this._typeInfo.getFieldDef()}getDirective(){return this._typeInfo.getDirective()}getArgument(){return this._typeInfo.getArgument()}getEnumValue(){return this._typeInfo.getEnumValue()}};Gt.ValidationContext=Jo});var bn=h(Ar=>{"use strict";Object.defineProperty(Ar,"__esModule",{value:!0});Ar.assertValidSDL=lb;Ar.assertValidSDLExtension=pb;Ar.validate=cb;Ar.validateSDL=Ho;var ab=Fe(),ob=q(),ji=Yt(),ub=un(),cp=vi(),lp=Bo(),pp=Xo();function cb(e,t,r=lp.specifiedRules,n,i=new cp.TypeInfo(e)){var s;let a=(s=n?.maxErrors)!==null&&s!==void 0?s:100;t||(0,ab.devAssert)(!1,"Must provide document."),(0,ub.assertValidSchema)(e);let o=Object.freeze({}),u=[],c=new pp.ValidationContext(e,t,i,f=>{if(u.length>=a)throw u.push(new ob.GraphQLError("Too many validation errors, error limit reached. Validation aborted.")),o;u.push(f)}),l=(0,ji.visitInParallel)(r.map(f=>f(c)));try{(0,ji.visit)(t,(0,cp.visitWithTypeInfo)(i,l))}catch(f){if(f!==o)throw f}return u}function Ho(e,t,r=lp.specifiedSDLRules){let n=[],i=new pp.SDLValidationContext(e,t,a=>{n.push(a)}),s=r.map(a=>a(i));return(0,ji.visit)(e,(0,ji.visitInParallel)(s)),n}function lb(e){let t=Ho(e);if(t.length!==0)throw new Error(t.map(r=>r.message).join(` + +`))}function pb(e,t){let r=Ho(e,t);if(r.length!==0)throw new Error(r.map(n=>n.message).join(` + +`))}});var dp=h(zo=>{"use strict";Object.defineProperty(zo,"__esModule",{value:!0});zo.memoize3=db;function db(e){let t;return function(n,i,s){t===void 0&&(t=new WeakMap);let a=t.get(n);a===void 0&&(a=new WeakMap,t.set(n,a));let o=a.get(i);o===void 0&&(o=new WeakMap,a.set(i,o));let u=o.get(s);return u===void 0&&(u=e(n,i,s),o.set(s,u)),u}}});var fp=h(Wo=>{"use strict";Object.defineProperty(Wo,"__esModule",{value:!0});Wo.promiseForObject=fb;function fb(e){return Promise.all(Object.values(e)).then(t=>{let r=Object.create(null);for(let[n,i]of Object.keys(e).entries())r[i]=t[n];return r})}});var mp=h(Zo=>{"use strict";Object.defineProperty(Zo,"__esModule",{value:!0});Zo.promiseReduce=yb;var mb=wn();function yb(e,t,r){let n=r;for(let i of e)n=(0,mb.isPromise)(n)?n.then(s=>t(s,i)):t(n,i);return n}});var yp=h(tu=>{"use strict";Object.defineProperty(tu,"__esModule",{value:!0});tu.toError=vb;var hb=re();function vb(e){return e instanceof Error?e:new eu(e)}var eu=class extends Error{constructor(t){super("Unexpected error value: "+(0,hb.inspect)(t)),this.name="NonErrorThrown",this.thrownValue=t}}});var Pi=h(ru=>{"use strict";Object.defineProperty(ru,"__esModule",{value:!0});ru.locatedError=gb;var Tb=yp(),bb=q();function gb(e,t,r){var n;let i=(0,Tb.toError)(e);return Eb(i)?i:new bb.GraphQLError(i.message,{nodes:(n=i.nodes)!==null&&n!==void 0?n:t,source:i.source,positions:i.positions,path:r,originalError:i})}function Eb(e){return Array.isArray(e.path)}});var En=h(He=>{"use strict";Object.defineProperty(He,"__esModule",{value:!0});He.assertValidExecutionArguments=Np;He.buildExecutionContext=Op;He.buildResolveInfo=Lp;He.defaultTypeResolver=He.defaultFieldResolver=void 0;He.execute=_p;He.executeSync=Db;He.getFieldDef=Dp;var iu=Fe(),ar=re(),_b=je(),Nb=fi(),ou=pt(),ut=wn(),Ob=dp(),or=pn(),hp=fp(),Ib=mp(),Xe=q(),Fi=Pi(),nu=dt(),vp=H(),Ct=Y(),Rr=Ye(),Lb=un(),gp=Di(),Ep=Sr(),Sb=(0,Ob.memoize3)((e,t,r)=>(0,gp.collectSubfields)(e.schema,e.fragments,e.variableValues,t,r));function _p(e){arguments.length<2||(0,iu.devAssert)(!1,"graphql@16 dropped long-deprecated support for positional arguments, please pass an object instead.");let{schema:t,document:r,variableValues:n,rootValue:i}=e;Np(t,r,n);let s=Op(e);if(!("schema"in s))return{errors:s};try{let{operation:a}=s,o=Ab(s,a,i);return(0,ut.isPromise)(o)?o.then(u=>wi(u,s.errors),u=>(s.errors.push(u),wi(null,s.errors))):wi(o,s.errors)}catch(a){return s.errors.push(a),wi(null,s.errors)}}function Db(e){let t=_p(e);if((0,ut.isPromise)(t))throw new Error("GraphQL execution failed to complete synchronously.");return t}function wi(e,t){return t.length===0?{data:e}:{errors:t,data:e}}function Np(e,t,r){t||(0,iu.devAssert)(!1,"Must provide document."),(0,Lb.assertValidSchema)(e),r==null||(0,ou.isObjectLike)(r)||(0,iu.devAssert)(!1,"Variables must be provided as an Object where each property is a variable value. Perhaps look to see if an unparsed JSON string was provided.")}function Op(e){var t,r;let{schema:n,document:i,rootValue:s,contextValue:a,variableValues:o,operationName:u,fieldResolver:c,typeResolver:l,subscribeFieldResolver:f}=e,d,y=Object.create(null);for(let E of i.definitions)switch(E.kind){case vp.Kind.OPERATION_DEFINITION:if(u==null){if(d!==void 0)return[new Xe.GraphQLError("Must provide operation name if query contains multiple operations.")];d=E}else((t=E.name)===null||t===void 0?void 0:t.value)===u&&(d=E);break;case vp.Kind.FRAGMENT_DEFINITION:y[E.name.value]=E;break;default:}if(!d)return u!=null?[new Xe.GraphQLError(`Unknown operation named "${u}".`)]:[new Xe.GraphQLError("Must provide an operation.")];let T=(r=d.variableDefinitions)!==null&&r!==void 0?r:[],I=(0,Ep.getVariableValues)(n,T,o??{},{maxErrors:50});return I.errors?I.errors:{schema:n,fragments:y,rootValue:s,contextValue:a,operation:d,variableValues:I.coerced,fieldResolver:c??au,typeResolver:l??Sp,subscribeFieldResolver:f??au,errors:[]}}function Ab(e,t,r){let n=e.schema.getRootType(t.operation);if(n==null)throw new Xe.GraphQLError(`Schema is not configured to execute ${t.operation} operation.`,{nodes:t});let i=(0,gp.collectFields)(e.schema,e.fragments,e.variableValues,n,t.selectionSet),s=void 0;switch(t.operation){case nu.OperationTypeNode.QUERY:return ki(e,n,r,s,i);case nu.OperationTypeNode.MUTATION:return Rb(e,n,r,s,i);case nu.OperationTypeNode.SUBSCRIPTION:return ki(e,n,r,s,i)}}function Rb(e,t,r,n,i){return(0,Ib.promiseReduce)(i.entries(),(s,[a,o])=>{let u=(0,or.addPath)(n,a,t.name),c=Ip(e,t,r,o,u);return c===void 0?s:(0,ut.isPromise)(c)?c.then(l=>(s[a]=l,s)):(s[a]=c,s)},Object.create(null))}function ki(e,t,r,n,i){let s=Object.create(null),a=!1;try{for(let[o,u]of i.entries()){let c=(0,or.addPath)(n,o,t.name),l=Ip(e,t,r,u,c);l!==void 0&&(s[o]=l,(0,ut.isPromise)(l)&&(a=!0))}}catch(o){if(a)return(0,hp.promiseForObject)(s).finally(()=>{throw o});throw o}return a?(0,hp.promiseForObject)(s):s}function Ip(e,t,r,n,i){var s;let a=Dp(e.schema,t,n[0]);if(!a)return;let o=a.type,u=(s=a.resolve)!==null&&s!==void 0?s:e.fieldResolver,c=Lp(e,a,n,t,i);try{let l=(0,Ep.getArgumentValues)(a,n[0],e.variableValues),f=e.contextValue,d=u(r,l,f,c),y;return(0,ut.isPromise)(d)?y=d.then(T=>gn(e,o,n,c,i,T)):y=gn(e,o,n,c,i,d),(0,ut.isPromise)(y)?y.then(void 0,T=>{let I=(0,Fi.locatedError)(T,n,(0,or.pathToArray)(i));return qi(I,o,e)}):y}catch(l){let f=(0,Fi.locatedError)(l,n,(0,or.pathToArray)(i));return qi(f,o,e)}}function Lp(e,t,r,n,i){return{fieldName:t.name,fieldNodes:r,returnType:t.type,parentType:n,path:i,schema:e.schema,fragments:e.fragments,rootValue:e.rootValue,operation:e.operation,variableValues:e.variableValues}}function qi(e,t,r){if((0,Ct.isNonNullType)(t))throw e;return r.errors.push(e),null}function gn(e,t,r,n,i,s){if(s instanceof Error)throw s;if((0,Ct.isNonNullType)(t)){let a=gn(e,t.ofType,r,n,i,s);if(a===null)throw new Error(`Cannot return null for non-nullable field ${n.parentType.name}.${n.fieldName}.`);return a}if(s==null)return null;if((0,Ct.isListType)(t))return jb(e,t,r,n,i,s);if((0,Ct.isLeafType)(t))return Pb(t,s);if((0,Ct.isAbstractType)(t))return wb(e,t,r,n,i,s);if((0,Ct.isObjectType)(t))return su(e,t,r,n,i,s);(0,_b.invariant)(!1,"Cannot complete value of unexpected output type: "+(0,ar.inspect)(t))}function jb(e,t,r,n,i,s){if(!(0,Nb.isIterableObject)(s))throw new Xe.GraphQLError(`Expected Iterable, but did not find one for field "${n.parentType.name}.${n.fieldName}".`);let a=t.ofType,o=!1,u=Array.from(s,(c,l)=>{let f=(0,or.addPath)(i,l,void 0);try{let d;return(0,ut.isPromise)(c)?d=c.then(y=>gn(e,a,r,n,f,y)):d=gn(e,a,r,n,f,c),(0,ut.isPromise)(d)?(o=!0,d.then(void 0,y=>{let T=(0,Fi.locatedError)(y,r,(0,or.pathToArray)(f));return qi(T,a,e)})):d}catch(d){let y=(0,Fi.locatedError)(d,r,(0,or.pathToArray)(f));return qi(y,a,e)}});return o?Promise.all(u):u}function Pb(e,t){let r=e.serialize(t);if(r==null)throw new Error(`Expected \`${(0,ar.inspect)(e)}.serialize(${(0,ar.inspect)(t)})\` to return non-nullable value, returned: ${(0,ar.inspect)(r)}`);return r}function wb(e,t,r,n,i,s){var a;let o=(a=t.resolveType)!==null&&a!==void 0?a:e.typeResolver,u=e.contextValue,c=o(s,u,n,t);return(0,ut.isPromise)(c)?c.then(l=>su(e,Tp(l,e,t,r,n,s),r,n,i,s)):su(e,Tp(c,e,t,r,n,s),r,n,i,s)}function Tp(e,t,r,n,i,s){if(e==null)throw new Xe.GraphQLError(`Abstract type "${r.name}" must resolve to an Object type at runtime for field "${i.parentType.name}.${i.fieldName}". Either the "${r.name}" type should provide a "resolveType" function or each possible type should provide an "isTypeOf" function.`,n);if((0,Ct.isObjectType)(e))throw new Xe.GraphQLError("Support for returning GraphQLObjectType from resolveType was removed in graphql-js@16.0.0 please return type name instead.");if(typeof e!="string")throw new Xe.GraphQLError(`Abstract type "${r.name}" must resolve to an Object type at runtime for field "${i.parentType.name}.${i.fieldName}" with value ${(0,ar.inspect)(s)}, received "${(0,ar.inspect)(e)}".`);let a=t.schema.getType(e);if(a==null)throw new Xe.GraphQLError(`Abstract type "${r.name}" was resolved to a type "${e}" that does not exist inside the schema.`,{nodes:n});if(!(0,Ct.isObjectType)(a))throw new Xe.GraphQLError(`Abstract type "${r.name}" was resolved to a non-object type "${e}".`,{nodes:n});if(!t.schema.isSubType(r,a))throw new Xe.GraphQLError(`Runtime Object type "${a.name}" is not a possible type for "${r.name}".`,{nodes:n});return a}function su(e,t,r,n,i,s){let a=Sb(e,t,r);if(t.isTypeOf){let o=t.isTypeOf(s,e.contextValue,n);if((0,ut.isPromise)(o))return o.then(u=>{if(!u)throw bp(t,s,r);return ki(e,t,s,i,a)});if(!o)throw bp(t,s,r)}return ki(e,t,s,i,a)}function bp(e,t,r){return new Xe.GraphQLError(`Expected value of type "${e.name}" but got: ${(0,ar.inspect)(t)}.`,{nodes:r})}var Sp=function(e,t,r,n){if((0,ou.isObjectLike)(e)&&typeof e.__typename=="string")return e.__typename;let i=r.schema.getPossibleTypes(n),s=[];for(let a=0;a{for(let o=0;o{"use strict";Object.defineProperty(xi,"__esModule",{value:!0});xi.graphql=Gb;xi.graphqlSync=Cb;var Fb=Fe(),kb=wn(),qb=vr(),xb=un(),Vb=bn(),Mb=En();function Gb(e){return new Promise(t=>t(Ap(e)))}function Cb(e){let t=Ap(e);if((0,kb.isPromise)(t))throw new Error("GraphQL execution failed to complete synchronously.");return t}function Ap(e){arguments.length<2||(0,Fb.devAssert)(!1,"graphql@16 dropped long-deprecated support for positional arguments, please pass an object instead.");let{schema:t,source:r,rootValue:n,contextValue:i,variableValues:s,operationName:a,fieldResolver:o,typeResolver:u}=e,c=(0,xb.validateSchema)(t);if(c.length>0)return{errors:c};let l;try{l=(0,qb.parse)(r)}catch(d){return{errors:[d]}}let f=(0,Vb.validate)(t,l);return f.length>0?{errors:f}:(0,Mb.execute)({schema:t,document:l,rootValue:n,contextValue:i,variableValues:s,operationName:a,fieldResolver:o,typeResolver:u})}});var wp=h(b=>{"use strict";Object.defineProperty(b,"__esModule",{value:!0});Object.defineProperty(b,"DEFAULT_DEPRECATION_REASON",{enumerable:!0,get:function(){return Tt.DEFAULT_DEPRECATION_REASON}});Object.defineProperty(b,"GRAPHQL_MAX_INT",{enumerable:!0,get:function(){return Dt.GRAPHQL_MAX_INT}});Object.defineProperty(b,"GRAPHQL_MIN_INT",{enumerable:!0,get:function(){return Dt.GRAPHQL_MIN_INT}});Object.defineProperty(b,"GraphQLBoolean",{enumerable:!0,get:function(){return Dt.GraphQLBoolean}});Object.defineProperty(b,"GraphQLDeprecatedDirective",{enumerable:!0,get:function(){return Tt.GraphQLDeprecatedDirective}});Object.defineProperty(b,"GraphQLDirective",{enumerable:!0,get:function(){return Tt.GraphQLDirective}});Object.defineProperty(b,"GraphQLEnumType",{enumerable:!0,get:function(){return x.GraphQLEnumType}});Object.defineProperty(b,"GraphQLFloat",{enumerable:!0,get:function(){return Dt.GraphQLFloat}});Object.defineProperty(b,"GraphQLID",{enumerable:!0,get:function(){return Dt.GraphQLID}});Object.defineProperty(b,"GraphQLIncludeDirective",{enumerable:!0,get:function(){return Tt.GraphQLIncludeDirective}});Object.defineProperty(b,"GraphQLInputObjectType",{enumerable:!0,get:function(){return x.GraphQLInputObjectType}});Object.defineProperty(b,"GraphQLInt",{enumerable:!0,get:function(){return Dt.GraphQLInt}});Object.defineProperty(b,"GraphQLInterfaceType",{enumerable:!0,get:function(){return x.GraphQLInterfaceType}});Object.defineProperty(b,"GraphQLList",{enumerable:!0,get:function(){return x.GraphQLList}});Object.defineProperty(b,"GraphQLNonNull",{enumerable:!0,get:function(){return x.GraphQLNonNull}});Object.defineProperty(b,"GraphQLObjectType",{enumerable:!0,get:function(){return x.GraphQLObjectType}});Object.defineProperty(b,"GraphQLScalarType",{enumerable:!0,get:function(){return x.GraphQLScalarType}});Object.defineProperty(b,"GraphQLSchema",{enumerable:!0,get:function(){return uu.GraphQLSchema}});Object.defineProperty(b,"GraphQLSkipDirective",{enumerable:!0,get:function(){return Tt.GraphQLSkipDirective}});Object.defineProperty(b,"GraphQLSpecifiedByDirective",{enumerable:!0,get:function(){return Tt.GraphQLSpecifiedByDirective}});Object.defineProperty(b,"GraphQLString",{enumerable:!0,get:function(){return Dt.GraphQLString}});Object.defineProperty(b,"GraphQLUnionType",{enumerable:!0,get:function(){return x.GraphQLUnionType}});Object.defineProperty(b,"SchemaMetaFieldDef",{enumerable:!0,get:function(){return Ge.SchemaMetaFieldDef}});Object.defineProperty(b,"TypeKind",{enumerable:!0,get:function(){return Ge.TypeKind}});Object.defineProperty(b,"TypeMetaFieldDef",{enumerable:!0,get:function(){return Ge.TypeMetaFieldDef}});Object.defineProperty(b,"TypeNameMetaFieldDef",{enumerable:!0,get:function(){return Ge.TypeNameMetaFieldDef}});Object.defineProperty(b,"__Directive",{enumerable:!0,get:function(){return Ge.__Directive}});Object.defineProperty(b,"__DirectiveLocation",{enumerable:!0,get:function(){return Ge.__DirectiveLocation}});Object.defineProperty(b,"__EnumValue",{enumerable:!0,get:function(){return Ge.__EnumValue}});Object.defineProperty(b,"__Field",{enumerable:!0,get:function(){return Ge.__Field}});Object.defineProperty(b,"__InputValue",{enumerable:!0,get:function(){return Ge.__InputValue}});Object.defineProperty(b,"__Schema",{enumerable:!0,get:function(){return Ge.__Schema}});Object.defineProperty(b,"__Type",{enumerable:!0,get:function(){return Ge.__Type}});Object.defineProperty(b,"__TypeKind",{enumerable:!0,get:function(){return Ge.__TypeKind}});Object.defineProperty(b,"assertAbstractType",{enumerable:!0,get:function(){return x.assertAbstractType}});Object.defineProperty(b,"assertCompositeType",{enumerable:!0,get:function(){return x.assertCompositeType}});Object.defineProperty(b,"assertDirective",{enumerable:!0,get:function(){return Tt.assertDirective}});Object.defineProperty(b,"assertEnumType",{enumerable:!0,get:function(){return x.assertEnumType}});Object.defineProperty(b,"assertEnumValueName",{enumerable:!0,get:function(){return Pp.assertEnumValueName}});Object.defineProperty(b,"assertInputObjectType",{enumerable:!0,get:function(){return x.assertInputObjectType}});Object.defineProperty(b,"assertInputType",{enumerable:!0,get:function(){return x.assertInputType}});Object.defineProperty(b,"assertInterfaceType",{enumerable:!0,get:function(){return x.assertInterfaceType}});Object.defineProperty(b,"assertLeafType",{enumerable:!0,get:function(){return x.assertLeafType}});Object.defineProperty(b,"assertListType",{enumerable:!0,get:function(){return x.assertListType}});Object.defineProperty(b,"assertName",{enumerable:!0,get:function(){return Pp.assertName}});Object.defineProperty(b,"assertNamedType",{enumerable:!0,get:function(){return x.assertNamedType}});Object.defineProperty(b,"assertNonNullType",{enumerable:!0,get:function(){return x.assertNonNullType}});Object.defineProperty(b,"assertNullableType",{enumerable:!0,get:function(){return x.assertNullableType}});Object.defineProperty(b,"assertObjectType",{enumerable:!0,get:function(){return x.assertObjectType}});Object.defineProperty(b,"assertOutputType",{enumerable:!0,get:function(){return x.assertOutputType}});Object.defineProperty(b,"assertScalarType",{enumerable:!0,get:function(){return x.assertScalarType}});Object.defineProperty(b,"assertSchema",{enumerable:!0,get:function(){return uu.assertSchema}});Object.defineProperty(b,"assertType",{enumerable:!0,get:function(){return x.assertType}});Object.defineProperty(b,"assertUnionType",{enumerable:!0,get:function(){return x.assertUnionType}});Object.defineProperty(b,"assertValidSchema",{enumerable:!0,get:function(){return jp.assertValidSchema}});Object.defineProperty(b,"assertWrappingType",{enumerable:!0,get:function(){return x.assertWrappingType}});Object.defineProperty(b,"getNamedType",{enumerable:!0,get:function(){return x.getNamedType}});Object.defineProperty(b,"getNullableType",{enumerable:!0,get:function(){return x.getNullableType}});Object.defineProperty(b,"introspectionTypes",{enumerable:!0,get:function(){return Ge.introspectionTypes}});Object.defineProperty(b,"isAbstractType",{enumerable:!0,get:function(){return x.isAbstractType}});Object.defineProperty(b,"isCompositeType",{enumerable:!0,get:function(){return x.isCompositeType}});Object.defineProperty(b,"isDirective",{enumerable:!0,get:function(){return Tt.isDirective}});Object.defineProperty(b,"isEnumType",{enumerable:!0,get:function(){return x.isEnumType}});Object.defineProperty(b,"isInputObjectType",{enumerable:!0,get:function(){return x.isInputObjectType}});Object.defineProperty(b,"isInputType",{enumerable:!0,get:function(){return x.isInputType}});Object.defineProperty(b,"isInterfaceType",{enumerable:!0,get:function(){return x.isInterfaceType}});Object.defineProperty(b,"isIntrospectionType",{enumerable:!0,get:function(){return Ge.isIntrospectionType}});Object.defineProperty(b,"isLeafType",{enumerable:!0,get:function(){return x.isLeafType}});Object.defineProperty(b,"isListType",{enumerable:!0,get:function(){return x.isListType}});Object.defineProperty(b,"isNamedType",{enumerable:!0,get:function(){return x.isNamedType}});Object.defineProperty(b,"isNonNullType",{enumerable:!0,get:function(){return x.isNonNullType}});Object.defineProperty(b,"isNullableType",{enumerable:!0,get:function(){return x.isNullableType}});Object.defineProperty(b,"isObjectType",{enumerable:!0,get:function(){return x.isObjectType}});Object.defineProperty(b,"isOutputType",{enumerable:!0,get:function(){return x.isOutputType}});Object.defineProperty(b,"isRequiredArgument",{enumerable:!0,get:function(){return x.isRequiredArgument}});Object.defineProperty(b,"isRequiredInputField",{enumerable:!0,get:function(){return x.isRequiredInputField}});Object.defineProperty(b,"isScalarType",{enumerable:!0,get:function(){return x.isScalarType}});Object.defineProperty(b,"isSchema",{enumerable:!0,get:function(){return uu.isSchema}});Object.defineProperty(b,"isSpecifiedDirective",{enumerable:!0,get:function(){return Tt.isSpecifiedDirective}});Object.defineProperty(b,"isSpecifiedScalarType",{enumerable:!0,get:function(){return Dt.isSpecifiedScalarType}});Object.defineProperty(b,"isType",{enumerable:!0,get:function(){return x.isType}});Object.defineProperty(b,"isUnionType",{enumerable:!0,get:function(){return x.isUnionType}});Object.defineProperty(b,"isWrappingType",{enumerable:!0,get:function(){return x.isWrappingType}});Object.defineProperty(b,"resolveObjMapThunk",{enumerable:!0,get:function(){return x.resolveObjMapThunk}});Object.defineProperty(b,"resolveReadonlyArrayThunk",{enumerable:!0,get:function(){return x.resolveReadonlyArrayThunk}});Object.defineProperty(b,"specifiedDirectives",{enumerable:!0,get:function(){return Tt.specifiedDirectives}});Object.defineProperty(b,"specifiedScalarTypes",{enumerable:!0,get:function(){return Dt.specifiedScalarTypes}});Object.defineProperty(b,"validateSchema",{enumerable:!0,get:function(){return jp.validateSchema}});var uu=Zt(),x=Y(),Tt=Ve(),Dt=yt(),Ge=Ye(),jp=un(),Pp=Br()});var kp=h(z=>{"use strict";Object.defineProperty(z,"__esModule",{value:!0});Object.defineProperty(z,"BREAK",{enumerable:!0,get:function(){return _n.BREAK}});Object.defineProperty(z,"DirectiveLocation",{enumerable:!0,get:function(){return Jb.DirectiveLocation}});Object.defineProperty(z,"Kind",{enumerable:!0,get:function(){return Kb.Kind}});Object.defineProperty(z,"Lexer",{enumerable:!0,get:function(){return Bb.Lexer}});Object.defineProperty(z,"Location",{enumerable:!0,get:function(){return cu.Location}});Object.defineProperty(z,"OperationTypeNode",{enumerable:!0,get:function(){return cu.OperationTypeNode}});Object.defineProperty(z,"Source",{enumerable:!0,get:function(){return Qb.Source}});Object.defineProperty(z,"Token",{enumerable:!0,get:function(){return cu.Token}});Object.defineProperty(z,"TokenKind",{enumerable:!0,get:function(){return $b.TokenKind}});Object.defineProperty(z,"getEnterLeaveForKind",{enumerable:!0,get:function(){return _n.getEnterLeaveForKind}});Object.defineProperty(z,"getLocation",{enumerable:!0,get:function(){return Ub.getLocation}});Object.defineProperty(z,"getVisitFn",{enumerable:!0,get:function(){return _n.getVisitFn}});Object.defineProperty(z,"isConstValueNode",{enumerable:!0,get:function(){return bt.isConstValueNode}});Object.defineProperty(z,"isDefinitionNode",{enumerable:!0,get:function(){return bt.isDefinitionNode}});Object.defineProperty(z,"isExecutableDefinitionNode",{enumerable:!0,get:function(){return bt.isExecutableDefinitionNode}});Object.defineProperty(z,"isSelectionNode",{enumerable:!0,get:function(){return bt.isSelectionNode}});Object.defineProperty(z,"isTypeDefinitionNode",{enumerable:!0,get:function(){return bt.isTypeDefinitionNode}});Object.defineProperty(z,"isTypeExtensionNode",{enumerable:!0,get:function(){return bt.isTypeExtensionNode}});Object.defineProperty(z,"isTypeNode",{enumerable:!0,get:function(){return bt.isTypeNode}});Object.defineProperty(z,"isTypeSystemDefinitionNode",{enumerable:!0,get:function(){return bt.isTypeSystemDefinitionNode}});Object.defineProperty(z,"isTypeSystemExtensionNode",{enumerable:!0,get:function(){return bt.isTypeSystemExtensionNode}});Object.defineProperty(z,"isValueNode",{enumerable:!0,get:function(){return bt.isValueNode}});Object.defineProperty(z,"parse",{enumerable:!0,get:function(){return Vi.parse}});Object.defineProperty(z,"parseConstValue",{enumerable:!0,get:function(){return Vi.parseConstValue}});Object.defineProperty(z,"parseType",{enumerable:!0,get:function(){return Vi.parseType}});Object.defineProperty(z,"parseValue",{enumerable:!0,get:function(){return Vi.parseValue}});Object.defineProperty(z,"print",{enumerable:!0,get:function(){return Yb.print}});Object.defineProperty(z,"printLocation",{enumerable:!0,get:function(){return Fp.printLocation}});Object.defineProperty(z,"printSourceLocation",{enumerable:!0,get:function(){return Fp.printSourceLocation}});Object.defineProperty(z,"visit",{enumerable:!0,get:function(){return _n.visit}});Object.defineProperty(z,"visitInParallel",{enumerable:!0,get:function(){return _n.visitInParallel}});var Qb=Un(),Ub=Fn(),Fp=os(),Kb=H(),$b=Vr(),Bb=Mn(),Vi=vr(),Yb=Ce(),_n=Yt(),cu=dt(),bt=tr(),Jb=yr()});var qp=h(lu=>{"use strict";Object.defineProperty(lu,"__esModule",{value:!0});lu.isAsyncIterable=Xb;function Xb(e){return typeof e?.[Symbol.asyncIterator]=="function"}});var xp=h(pu=>{"use strict";Object.defineProperty(pu,"__esModule",{value:!0});pu.mapAsyncIterator=Hb;function Hb(e,t){let r=e[Symbol.asyncIterator]();async function n(i){if(i.done)return i;try{return{value:await t(i.value),done:!1}}catch(s){if(typeof r.return=="function")try{await r.return()}catch{}throw s}}return{async next(){return n(await r.next())},async return(){return typeof r.return=="function"?n(await r.return()):{value:void 0,done:!0}},async throw(i){if(typeof r.throw=="function")return n(await r.throw(i));throw i},[Symbol.asyncIterator](){return this}}}});var Cp=h(Mi=>{"use strict";Object.defineProperty(Mi,"__esModule",{value:!0});Mi.createSourceEventStream=Gp;Mi.subscribe=ng;var zb=Fe(),Wb=re(),Mp=qp(),Vp=pn(),du=q(),Zb=Pi(),eg=Di(),Nn=En(),tg=xp(),rg=Sr();async function ng(e){arguments.length<2||(0,zb.devAssert)(!1,"graphql@16 dropped long-deprecated support for positional arguments, please pass an object instead.");let t=await Gp(e);if(!(0,Mp.isAsyncIterable)(t))return t;let r=n=>(0,Nn.execute)({...e,rootValue:n});return(0,tg.mapAsyncIterator)(t,r)}function ig(e){let t=e[0];return t&&"document"in t?t:{schema:t,document:e[1],rootValue:e[2],contextValue:e[3],variableValues:e[4],operationName:e[5],subscribeFieldResolver:e[6]}}async function Gp(...e){let t=ig(e),{schema:r,document:n,variableValues:i}=t;(0,Nn.assertValidExecutionArguments)(r,n,i);let s=(0,Nn.buildExecutionContext)(t);if(!("schema"in s))return{errors:s};try{let a=await sg(s);if(!(0,Mp.isAsyncIterable)(a))throw new Error(`Subscription field must return Async Iterable. Received: ${(0,Wb.inspect)(a)}.`);return a}catch(a){if(a instanceof du.GraphQLError)return{errors:[a]};throw a}}async function sg(e){let{schema:t,fragments:r,operation:n,variableValues:i,rootValue:s}=e,a=t.getSubscriptionType();if(a==null)throw new du.GraphQLError("Schema is not configured to execute subscription operation.",{nodes:n});let o=(0,eg.collectFields)(t,r,i,a,n.selectionSet),[u,c]=[...o.entries()][0],l=(0,Nn.getFieldDef)(t,a,c[0]);if(!l){let T=c[0].name.value;throw new du.GraphQLError(`The subscription field "${T}" is not defined.`,{nodes:c})}let f=(0,Vp.addPath)(void 0,u,a.name),d=(0,Nn.buildResolveInfo)(e,l,c,a,f);try{var y;let T=(0,rg.getArgumentValues)(l,c[0],i),I=e.contextValue,L=await((y=l.subscribe)!==null&&y!==void 0?y:e.subscribeFieldResolver)(s,T,I,d);if(L instanceof Error)throw L;return L}catch(T){throw(0,Zb.locatedError)(T,c,(0,Vp.pathToArray)(f))}}});var Up=h(ze=>{"use strict";Object.defineProperty(ze,"__esModule",{value:!0});Object.defineProperty(ze,"createSourceEventStream",{enumerable:!0,get:function(){return Qp.createSourceEventStream}});Object.defineProperty(ze,"defaultFieldResolver",{enumerable:!0,get:function(){return Gi.defaultFieldResolver}});Object.defineProperty(ze,"defaultTypeResolver",{enumerable:!0,get:function(){return Gi.defaultTypeResolver}});Object.defineProperty(ze,"execute",{enumerable:!0,get:function(){return Gi.execute}});Object.defineProperty(ze,"executeSync",{enumerable:!0,get:function(){return Gi.executeSync}});Object.defineProperty(ze,"getArgumentValues",{enumerable:!0,get:function(){return fu.getArgumentValues}});Object.defineProperty(ze,"getDirectiveValues",{enumerable:!0,get:function(){return fu.getDirectiveValues}});Object.defineProperty(ze,"getVariableValues",{enumerable:!0,get:function(){return fu.getVariableValues}});Object.defineProperty(ze,"responsePathAsArray",{enumerable:!0,get:function(){return ag.pathToArray}});Object.defineProperty(ze,"subscribe",{enumerable:!0,get:function(){return Qp.subscribe}});var ag=pn(),Gi=En(),Qp=Cp(),fu=Sr()});var Kp=h(hu=>{"use strict";Object.defineProperty(hu,"__esModule",{value:!0});hu.NoDeprecatedCustomRule=og;var mu=je(),On=q(),yu=Y();function og(e){return{Field(t){let r=e.getFieldDef(),n=r?.deprecationReason;if(r&&n!=null){let i=e.getParentType();i!=null||(0,mu.invariant)(!1),e.reportError(new On.GraphQLError(`The field ${i.name}.${r.name} is deprecated. ${n}`,{nodes:t}))}},Argument(t){let r=e.getArgument(),n=r?.deprecationReason;if(r&&n!=null){let i=e.getDirective();if(i!=null)e.reportError(new On.GraphQLError(`Directive "@${i.name}" argument "${r.name}" is deprecated. ${n}`,{nodes:t}));else{let s=e.getParentType(),a=e.getFieldDef();s!=null&&a!=null||(0,mu.invariant)(!1),e.reportError(new On.GraphQLError(`Field "${s.name}.${a.name}" argument "${r.name}" is deprecated. ${n}`,{nodes:t}))}}},ObjectField(t){let r=(0,yu.getNamedType)(e.getParentInputType());if((0,yu.isInputObjectType)(r)){let n=r.getFields()[t.name.value],i=n?.deprecationReason;i!=null&&e.reportError(new On.GraphQLError(`The input field ${r.name}.${n.name} is deprecated. ${i}`,{nodes:t}))}},EnumValue(t){let r=e.getEnumValue(),n=r?.deprecationReason;if(r&&n!=null){let i=(0,yu.getNamedType)(e.getInputType());i!=null||(0,mu.invariant)(!1),e.reportError(new On.GraphQLError(`The enum value "${i.name}.${r.name}" is deprecated. ${n}`,{nodes:t}))}}}}});var $p=h(vu=>{"use strict";Object.defineProperty(vu,"__esModule",{value:!0});vu.NoSchemaIntrospectionCustomRule=pg;var ug=q(),cg=Y(),lg=Ye();function pg(e){return{Field(t){let r=(0,cg.getNamedType)(e.getType());r&&(0,lg.isIntrospectionType)(r)&&e.reportError(new ug.GraphQLError(`GraphQL introspection has been disabled, but the requested query contained the field "${t.name.value}".`,{nodes:t}))}}}});var Bp=h(Q=>{"use strict";Object.defineProperty(Q,"__esModule",{value:!0});Object.defineProperty(Q,"ExecutableDefinitionsRule",{enumerable:!0,get:function(){return yg.ExecutableDefinitionsRule}});Object.defineProperty(Q,"FieldsOnCorrectTypeRule",{enumerable:!0,get:function(){return hg.FieldsOnCorrectTypeRule}});Object.defineProperty(Q,"FragmentsOnCompositeTypesRule",{enumerable:!0,get:function(){return vg.FragmentsOnCompositeTypesRule}});Object.defineProperty(Q,"KnownArgumentNamesRule",{enumerable:!0,get:function(){return Tg.KnownArgumentNamesRule}});Object.defineProperty(Q,"KnownDirectivesRule",{enumerable:!0,get:function(){return bg.KnownDirectivesRule}});Object.defineProperty(Q,"KnownFragmentNamesRule",{enumerable:!0,get:function(){return gg.KnownFragmentNamesRule}});Object.defineProperty(Q,"KnownTypeNamesRule",{enumerable:!0,get:function(){return Eg.KnownTypeNamesRule}});Object.defineProperty(Q,"LoneAnonymousOperationRule",{enumerable:!0,get:function(){return _g.LoneAnonymousOperationRule}});Object.defineProperty(Q,"LoneSchemaDefinitionRule",{enumerable:!0,get:function(){return Cg.LoneSchemaDefinitionRule}});Object.defineProperty(Q,"NoDeprecatedCustomRule",{enumerable:!0,get:function(){return Xg.NoDeprecatedCustomRule}});Object.defineProperty(Q,"NoFragmentCyclesRule",{enumerable:!0,get:function(){return Ng.NoFragmentCyclesRule}});Object.defineProperty(Q,"NoSchemaIntrospectionCustomRule",{enumerable:!0,get:function(){return Hg.NoSchemaIntrospectionCustomRule}});Object.defineProperty(Q,"NoUndefinedVariablesRule",{enumerable:!0,get:function(){return Og.NoUndefinedVariablesRule}});Object.defineProperty(Q,"NoUnusedFragmentsRule",{enumerable:!0,get:function(){return Ig.NoUnusedFragmentsRule}});Object.defineProperty(Q,"NoUnusedVariablesRule",{enumerable:!0,get:function(){return Lg.NoUnusedVariablesRule}});Object.defineProperty(Q,"OverlappingFieldsCanBeMergedRule",{enumerable:!0,get:function(){return Sg.OverlappingFieldsCanBeMergedRule}});Object.defineProperty(Q,"PossibleFragmentSpreadsRule",{enumerable:!0,get:function(){return Dg.PossibleFragmentSpreadsRule}});Object.defineProperty(Q,"PossibleTypeExtensionsRule",{enumerable:!0,get:function(){return Jg.PossibleTypeExtensionsRule}});Object.defineProperty(Q,"ProvidedRequiredArgumentsRule",{enumerable:!0,get:function(){return Ag.ProvidedRequiredArgumentsRule}});Object.defineProperty(Q,"ScalarLeafsRule",{enumerable:!0,get:function(){return Rg.ScalarLeafsRule}});Object.defineProperty(Q,"SingleFieldSubscriptionsRule",{enumerable:!0,get:function(){return jg.SingleFieldSubscriptionsRule}});Object.defineProperty(Q,"UniqueArgumentDefinitionNamesRule",{enumerable:!0,get:function(){return Bg.UniqueArgumentDefinitionNamesRule}});Object.defineProperty(Q,"UniqueArgumentNamesRule",{enumerable:!0,get:function(){return Pg.UniqueArgumentNamesRule}});Object.defineProperty(Q,"UniqueDirectiveNamesRule",{enumerable:!0,get:function(){return Yg.UniqueDirectiveNamesRule}});Object.defineProperty(Q,"UniqueDirectivesPerLocationRule",{enumerable:!0,get:function(){return wg.UniqueDirectivesPerLocationRule}});Object.defineProperty(Q,"UniqueEnumValueNamesRule",{enumerable:!0,get:function(){return Kg.UniqueEnumValueNamesRule}});Object.defineProperty(Q,"UniqueFieldDefinitionNamesRule",{enumerable:!0,get:function(){return $g.UniqueFieldDefinitionNamesRule}});Object.defineProperty(Q,"UniqueFragmentNamesRule",{enumerable:!0,get:function(){return Fg.UniqueFragmentNamesRule}});Object.defineProperty(Q,"UniqueInputFieldNamesRule",{enumerable:!0,get:function(){return kg.UniqueInputFieldNamesRule}});Object.defineProperty(Q,"UniqueOperationNamesRule",{enumerable:!0,get:function(){return qg.UniqueOperationNamesRule}});Object.defineProperty(Q,"UniqueOperationTypesRule",{enumerable:!0,get:function(){return Qg.UniqueOperationTypesRule}});Object.defineProperty(Q,"UniqueTypeNamesRule",{enumerable:!0,get:function(){return Ug.UniqueTypeNamesRule}});Object.defineProperty(Q,"UniqueVariableNamesRule",{enumerable:!0,get:function(){return xg.UniqueVariableNamesRule}});Object.defineProperty(Q,"ValidationContext",{enumerable:!0,get:function(){return fg.ValidationContext}});Object.defineProperty(Q,"ValuesOfCorrectTypeRule",{enumerable:!0,get:function(){return Vg.ValuesOfCorrectTypeRule}});Object.defineProperty(Q,"VariablesAreInputTypesRule",{enumerable:!0,get:function(){return Mg.VariablesAreInputTypesRule}});Object.defineProperty(Q,"VariablesInAllowedPositionRule",{enumerable:!0,get:function(){return Gg.VariablesInAllowedPositionRule}});Object.defineProperty(Q,"specifiedRules",{enumerable:!0,get:function(){return mg.specifiedRules}});Object.defineProperty(Q,"validate",{enumerable:!0,get:function(){return dg.validate}});var dg=bn(),fg=Xo(),mg=Bo(),yg=pa(),hg=fa(),vg=ya(),Tg=ha(),bg=ga(),gg=_a(),Eg=Ia(),_g=Sa(),Ng=ja(),Og=wa(),Ig=ka(),Lg=xa(),Sg=Ya(),Dg=Ha(),Ag=eo(),Rg=ro(),jg=po(),Pg=vo(),wg=_o(),Fg=Ao(),kg=jo(),qg=wo(),xg=Mo(),Vg=Co(),Mg=Uo(),Gg=$o(),Cg=Aa(),Qg=ko(),Ug=xo(),Kg=Oo(),$g=So(),Bg=yo(),Yg=bo(),Jg=Wa(),Xg=Kp(),Hg=$p()});var Yp=h(ur=>{"use strict";Object.defineProperty(ur,"__esModule",{value:!0});Object.defineProperty(ur,"GraphQLError",{enumerable:!0,get:function(){return Tu.GraphQLError}});Object.defineProperty(ur,"formatError",{enumerable:!0,get:function(){return Tu.formatError}});Object.defineProperty(ur,"locatedError",{enumerable:!0,get:function(){return Wg.locatedError}});Object.defineProperty(ur,"printError",{enumerable:!0,get:function(){return Tu.printError}});Object.defineProperty(ur,"syntaxError",{enumerable:!0,get:function(){return zg.syntaxError}});var Tu=q(),zg=qn(),Wg=Pi()});var gu=h(bu=>{"use strict";Object.defineProperty(bu,"__esModule",{value:!0});bu.getIntrospectionQuery=Zg;function Zg(e){let t={descriptions:!0,specifiedByUrl:!1,directiveIsRepeatable:!1,schemaDescription:!1,inputValueDeprecation:!1,...e},r=t.descriptions?"description":"",n=t.specifiedByUrl?"specifiedByURL":"",i=t.directiveIsRepeatable?"isRepeatable":"",s=t.schemaDescription?r:"";function a(o){return t.inputValueDeprecation?o:""}return` + query IntrospectionQuery { + __schema { + ${s} + queryType { name } + mutationType { name } + subscriptionType { name } + types { + ...FullType + } + directives { + name + ${r} + ${i} + locations + args${a("(includeDeprecated: true)")} { + ...InputValue + } + } + } + } + + fragment FullType on __Type { + kind + name + ${r} + ${n} + fields(includeDeprecated: true) { + name + ${r} + args${a("(includeDeprecated: true)")} { + ...InputValue + } + type { + ...TypeRef + } + isDeprecated + deprecationReason + } + inputFields${a("(includeDeprecated: true)")} { + ...InputValue + } + interfaces { + ...TypeRef + } + enumValues(includeDeprecated: true) { + name + ${r} + isDeprecated + deprecationReason + } + possibleTypes { + ...TypeRef + } + } + + fragment InputValue on __InputValue { + name + ${r} + type { ...TypeRef } + defaultValue + ${a("isDeprecated")} + ${a("deprecationReason")} + } + + fragment TypeRef on __Type { + kind + name + ofType { + kind + name + ofType { + kind + name + ofType { + kind + name + ofType { + kind + name + ofType { + kind + name + ofType { + kind + name + ofType { + kind + name + ofType { + kind + name + ofType { + kind + name + } + } + } + } + } + } + } + } + } + } + `}});var Jp=h(Eu=>{"use strict";Object.defineProperty(Eu,"__esModule",{value:!0});Eu.getOperationAST=tE;var eE=H();function tE(e,t){let r=null;for(let i of e.definitions)if(i.kind===eE.Kind.OPERATION_DEFINITION){var n;if(t==null){if(r)return null;r=i}else if(((n=i.name)===null||n===void 0?void 0:n.value)===t)return i}return r}});var Xp=h(_u=>{"use strict";Object.defineProperty(_u,"__esModule",{value:!0});_u.getOperationRootType=rE;var Ci=q();function rE(e,t){if(t.operation==="query"){let r=e.getQueryType();if(!r)throw new Ci.GraphQLError("Schema does not define the required query root type.",{nodes:t});return r}if(t.operation==="mutation"){let r=e.getMutationType();if(!r)throw new Ci.GraphQLError("Schema is not configured for mutations.",{nodes:t});return r}if(t.operation==="subscription"){let r=e.getSubscriptionType();if(!r)throw new Ci.GraphQLError("Schema is not configured for subscriptions.",{nodes:t});return r}throw new Ci.GraphQLError("Can only have query, mutation and subscription operations.",{nodes:t})}});var Hp=h(Nu=>{"use strict";Object.defineProperty(Nu,"__esModule",{value:!0});Nu.introspectionFromSchema=oE;var nE=je(),iE=vr(),sE=En(),aE=gu();function oE(e,t){let r={specifiedByUrl:!0,directiveIsRepeatable:!0,schemaDescription:!0,inputValueDeprecation:!0,...t},n=(0,iE.parse)((0,aE.getIntrospectionQuery)(r)),i=(0,sE.executeSync)({schema:e,document:n});return!i.errors&&i.data||(0,nE.invariant)(!1),i.data}});var Wp=h(Ou=>{"use strict";Object.defineProperty(Ou,"__esModule",{value:!0});Ou.buildClientSchema=mE;var uE=Fe(),Qe=re(),zp=pt(),Qi=Kr(),cE=vr(),Ue=Y(),lE=Ve(),gt=Ye(),pE=yt(),dE=Zt(),fE=yn();function mE(e,t){(0,zp.isObjectLike)(e)&&(0,zp.isObjectLike)(e.__schema)||(0,uE.devAssert)(!1,`Invalid or incomplete introspection result. Ensure that you are passing "data" property of introspection response and no "errors" was returned alongside: ${(0,Qe.inspect)(e)}.`);let r=e.__schema,n=(0,Qi.keyValMap)(r.types,m=>m.name,m=>d(m));for(let m of[...pE.specifiedScalarTypes,...gt.introspectionTypes])n[m.name]&&(n[m.name]=m);let i=r.queryType?l(r.queryType):null,s=r.mutationType?l(r.mutationType):null,a=r.subscriptionType?l(r.subscriptionType):null,o=r.directives?r.directives.map(V):[];return new dE.GraphQLSchema({description:r.description,query:i,mutation:s,subscription:a,types:Object.values(n),directives:o,assumeValid:t?.assumeValid});function u(m){if(m.kind===gt.TypeKind.LIST){let _=m.ofType;if(!_)throw new Error("Decorated type deeper than introspection query.");return new Ue.GraphQLList(u(_))}if(m.kind===gt.TypeKind.NON_NULL){let _=m.ofType;if(!_)throw new Error("Decorated type deeper than introspection query.");let oe=u(_);return new Ue.GraphQLNonNull((0,Ue.assertNullableType)(oe))}return c(m)}function c(m){let _=m.name;if(!_)throw new Error(`Unknown type reference: ${(0,Qe.inspect)(m)}.`);let oe=n[_];if(!oe)throw new Error(`Invalid or incomplete schema, unknown type: ${_}. Ensure that a full introspection query is used in order to build a client schema.`);return oe}function l(m){return(0,Ue.assertObjectType)(c(m))}function f(m){return(0,Ue.assertInterfaceType)(c(m))}function d(m){if(m!=null&&m.name!=null&&m.kind!=null)switch(m.kind){case gt.TypeKind.SCALAR:return y(m);case gt.TypeKind.OBJECT:return I(m);case gt.TypeKind.INTERFACE:return E(m);case gt.TypeKind.UNION:return L(m);case gt.TypeKind.ENUM:return F(m);case gt.TypeKind.INPUT_OBJECT:return k(m)}let _=(0,Qe.inspect)(m);throw new Error(`Invalid or incomplete introspection result. Ensure that a full introspection query is used in order to build a client schema: ${_}.`)}function y(m){return new Ue.GraphQLScalarType({name:m.name,description:m.description,specifiedByURL:m.specifiedByURL})}function T(m){if(m.interfaces===null&&m.kind===gt.TypeKind.INTERFACE)return[];if(!m.interfaces){let _=(0,Qe.inspect)(m);throw new Error(`Introspection result missing interfaces: ${_}.`)}return m.interfaces.map(f)}function I(m){return new Ue.GraphQLObjectType({name:m.name,description:m.description,interfaces:()=>T(m),fields:()=>P(m)})}function E(m){return new Ue.GraphQLInterfaceType({name:m.name,description:m.description,interfaces:()=>T(m),fields:()=>P(m)})}function L(m){if(!m.possibleTypes){let _=(0,Qe.inspect)(m);throw new Error(`Introspection result missing possibleTypes: ${_}.`)}return new Ue.GraphQLUnionType({name:m.name,description:m.description,types:()=>m.possibleTypes.map(l)})}function F(m){if(!m.enumValues){let _=(0,Qe.inspect)(m);throw new Error(`Introspection result missing enumValues: ${_}.`)}return new Ue.GraphQLEnumType({name:m.name,description:m.description,values:(0,Qi.keyValMap)(m.enumValues,_=>_.name,_=>({description:_.description,deprecationReason:_.deprecationReason}))})}function k(m){if(!m.inputFields){let _=(0,Qe.inspect)(m);throw new Error(`Introspection result missing inputFields: ${_}.`)}return new Ue.GraphQLInputObjectType({name:m.name,description:m.description,fields:()=>C(m.inputFields)})}function P(m){if(!m.fields)throw new Error(`Introspection result missing fields: ${(0,Qe.inspect)(m)}.`);return(0,Qi.keyValMap)(m.fields,_=>_.name,G)}function G(m){let _=u(m.type);if(!(0,Ue.isOutputType)(_)){let oe=(0,Qe.inspect)(_);throw new Error(`Introspection must provide output type for fields, but received: ${oe}.`)}if(!m.args){let oe=(0,Qe.inspect)(m);throw new Error(`Introspection result missing field args: ${oe}.`)}return{description:m.description,deprecationReason:m.deprecationReason,type:_,args:C(m.args)}}function C(m){return(0,Qi.keyValMap)(m,_=>_.name,w)}function w(m){let _=u(m.type);if(!(0,Ue.isInputType)(_)){let $=(0,Qe.inspect)(_);throw new Error(`Introspection must provide input type for arguments, but received: ${$}.`)}let oe=m.defaultValue!=null?(0,fE.valueFromAST)((0,cE.parseValue)(m.defaultValue),_):void 0;return{description:m.description,type:_,defaultValue:oe,deprecationReason:m.deprecationReason}}function V(m){if(!m.args){let _=(0,Qe.inspect)(m);throw new Error(`Introspection result missing directive args: ${_}.`)}if(!m.locations){let _=(0,Qe.inspect)(m);throw new Error(`Introspection result missing directive locations: ${_}.`)}return new lE.GraphQLDirective({name:m.name,description:m.description,isRepeatable:m.isRepeatable,locations:m.locations.slice(),args:C(m.args)})}}});var Iu=h($i=>{"use strict";Object.defineProperty($i,"__esModule",{value:!0});$i.extendSchema=gE;$i.extendSchemaImpl=od;var yE=Fe(),hE=re(),vE=je(),TE=Ft(),In=Ss(),We=H(),Zp=tr(),pe=Y(),Ki=Ve(),id=Ye(),sd=yt(),ed=Zt(),bE=bn(),ad=Sr(),td=yn();function gE(e,t,r){(0,ed.assertSchema)(e),t!=null&&t.kind===We.Kind.DOCUMENT||(0,yE.devAssert)(!1,"Must provide valid Document AST."),r?.assumeValid!==!0&&r?.assumeValidSDL!==!0&&(0,bE.assertValidSDLExtension)(t,e);let n=e.toConfig(),i=od(n,t,r);return n===i?e:new ed.GraphQLSchema(i)}function od(e,t,r){var n,i,s,a;let o=[],u=Object.create(null),c=[],l,f=[];for(let v of t.definitions)if(v.kind===We.Kind.SCHEMA_DEFINITION)l=v;else if(v.kind===We.Kind.SCHEMA_EXTENSION)f.push(v);else if((0,Zp.isTypeDefinitionNode)(v))o.push(v);else if((0,Zp.isTypeExtensionNode)(v)){let S=v.name.value,N=u[S];u[S]=N?N.concat([v]):[v]}else v.kind===We.Kind.DIRECTIVE_DEFINITION&&c.push(v);if(Object.keys(u).length===0&&o.length===0&&c.length===0&&f.length===0&&l==null)return e;let d=Object.create(null);for(let v of e.types)d[v.name]=F(v);for(let v of o){var y;let S=v.name.value;d[S]=(y=rd[S])!==null&&y!==void 0?y:jt(v)}let T={query:e.query&&E(e.query),mutation:e.mutation&&E(e.mutation),subscription:e.subscription&&E(e.subscription),...l&&oe([l]),...oe(f)};return{description:(n=l)===null||n===void 0||(i=n.description)===null||i===void 0?void 0:i.value,...T,types:Object.values(d),directives:[...e.directives.map(L),...c.map(ye)],extensions:Object.create(null),astNode:(s=l)!==null&&s!==void 0?s:e.astNode,extensionASTNodes:e.extensionASTNodes.concat(f),assumeValid:(a=r?.assumeValid)!==null&&a!==void 0?a:!1};function I(v){return(0,pe.isListType)(v)?new pe.GraphQLList(I(v.ofType)):(0,pe.isNonNullType)(v)?new pe.GraphQLNonNull(I(v.ofType)):E(v)}function E(v){return d[v.name]}function L(v){let S=v.toConfig();return new Ki.GraphQLDirective({...S,args:(0,In.mapValue)(S.args,_)})}function F(v){if((0,id.isIntrospectionType)(v)||(0,sd.isSpecifiedScalarType)(v))return v;if((0,pe.isScalarType)(v))return G(v);if((0,pe.isObjectType)(v))return C(v);if((0,pe.isInterfaceType)(v))return w(v);if((0,pe.isUnionType)(v))return V(v);if((0,pe.isEnumType)(v))return P(v);if((0,pe.isInputObjectType)(v))return k(v);(0,vE.invariant)(!1,"Unexpected type: "+(0,hE.inspect)(v))}function k(v){var S;let N=v.toConfig(),D=(S=u[N.name])!==null&&S!==void 0?S:[];return new pe.GraphQLInputObjectType({...N,fields:()=>({...(0,In.mapValue)(N.fields,ie=>({...ie,type:I(ie.type)})),...Le(D)}),extensionASTNodes:N.extensionASTNodes.concat(D)})}function P(v){var S;let N=v.toConfig(),D=(S=u[v.name])!==null&&S!==void 0?S:[];return new pe.GraphQLEnumType({...N,values:{...N.values,...xe(D)},extensionASTNodes:N.extensionASTNodes.concat(D)})}function G(v){var S;let N=v.toConfig(),D=(S=u[N.name])!==null&&S!==void 0?S:[],ie=N.specifiedByURL;for(let ce of D){var be;ie=(be=nd(ce))!==null&&be!==void 0?be:ie}return new pe.GraphQLScalarType({...N,specifiedByURL:ie,extensionASTNodes:N.extensionASTNodes.concat(D)})}function C(v){var S;let N=v.toConfig(),D=(S=u[N.name])!==null&&S!==void 0?S:[];return new pe.GraphQLObjectType({...N,interfaces:()=>[...v.getInterfaces().map(E),...Ae(D)],fields:()=>({...(0,In.mapValue)(N.fields,m),...Te(D)}),extensionASTNodes:N.extensionASTNodes.concat(D)})}function w(v){var S;let N=v.toConfig(),D=(S=u[N.name])!==null&&S!==void 0?S:[];return new pe.GraphQLInterfaceType({...N,interfaces:()=>[...v.getInterfaces().map(E),...Ae(D)],fields:()=>({...(0,In.mapValue)(N.fields,m),...Te(D)}),extensionASTNodes:N.extensionASTNodes.concat(D)})}function V(v){var S;let N=v.toConfig(),D=(S=u[N.name])!==null&&S!==void 0?S:[];return new pe.GraphQLUnionType({...N,types:()=>[...v.getTypes().map(E),...Nt(D)],extensionASTNodes:N.extensionASTNodes.concat(D)})}function m(v){return{...v,type:I(v.type),args:v.args&&(0,In.mapValue)(v.args,_)}}function _(v){return{...v,type:I(v.type)}}function oe(v){let S={};for(let D of v){var N;let ie=(N=D.operationTypes)!==null&&N!==void 0?N:[];for(let be of ie)S[be.operation]=$(be.type)}return S}function $(v){var S;let N=v.name.value,D=(S=rd[N])!==null&&S!==void 0?S:d[N];if(D===void 0)throw new Error(`Unknown type: "${N}".`);return D}function X(v){return v.kind===We.Kind.LIST_TYPE?new pe.GraphQLList(X(v.type)):v.kind===We.Kind.NON_NULL_TYPE?new pe.GraphQLNonNull(X(v.type)):$(v)}function ye(v){var S;return new Ki.GraphQLDirective({name:v.name.value,description:(S=v.description)===null||S===void 0?void 0:S.value,locations:v.locations.map(({value:N})=>N),isRepeatable:v.repeatable,args:Ie(v.arguments),astNode:v})}function Te(v){let S=Object.create(null);for(let ie of v){var N;let be=(N=ie.fields)!==null&&N!==void 0?N:[];for(let ce of be){var D;S[ce.name.value]={type:X(ce.type),description:(D=ce.description)===null||D===void 0?void 0:D.value,args:Ie(ce.arguments),deprecationReason:Ui(ce),astNode:ce}}}return S}function Ie(v){let S=v??[],N=Object.create(null);for(let ie of S){var D;let be=X(ie.type);N[ie.name.value]={type:be,description:(D=ie.description)===null||D===void 0?void 0:D.value,defaultValue:(0,td.valueFromAST)(ie.defaultValue,be),deprecationReason:Ui(ie),astNode:ie}}return N}function Le(v){let S=Object.create(null);for(let ie of v){var N;let be=(N=ie.fields)!==null&&N!==void 0?N:[];for(let ce of be){var D;let pr=X(ce.type);S[ce.name.value]={type:pr,description:(D=ce.description)===null||D===void 0?void 0:D.value,defaultValue:(0,td.valueFromAST)(ce.defaultValue,pr),deprecationReason:Ui(ce),astNode:ce}}}return S}function xe(v){let S=Object.create(null);for(let ie of v){var N;let be=(N=ie.values)!==null&&N!==void 0?N:[];for(let ce of be){var D;S[ce.name.value]={description:(D=ce.description)===null||D===void 0?void 0:D.value,deprecationReason:Ui(ce),astNode:ce}}}return S}function Ae(v){return v.flatMap(S=>{var N,D;return(N=(D=S.interfaces)===null||D===void 0?void 0:D.map($))!==null&&N!==void 0?N:[]})}function Nt(v){return v.flatMap(S=>{var N,D;return(N=(D=S.types)===null||D===void 0?void 0:D.map($))!==null&&N!==void 0?N:[]})}function jt(v){var S;let N=v.name.value,D=(S=u[N])!==null&&S!==void 0?S:[];switch(v.kind){case We.Kind.OBJECT_TYPE_DEFINITION:{var ie;let tt=[v,...D];return new pe.GraphQLObjectType({name:N,description:(ie=v.description)===null||ie===void 0?void 0:ie.value,interfaces:()=>Ae(tt),fields:()=>Te(tt),astNode:v,extensionASTNodes:D})}case We.Kind.INTERFACE_TYPE_DEFINITION:{var be;let tt=[v,...D];return new pe.GraphQLInterfaceType({name:N,description:(be=v.description)===null||be===void 0?void 0:be.value,interfaces:()=>Ae(tt),fields:()=>Te(tt),astNode:v,extensionASTNodes:D})}case We.Kind.ENUM_TYPE_DEFINITION:{var ce;let tt=[v,...D];return new pe.GraphQLEnumType({name:N,description:(ce=v.description)===null||ce===void 0?void 0:ce.value,values:xe(tt),astNode:v,extensionASTNodes:D})}case We.Kind.UNION_TYPE_DEFINITION:{var pr;let tt=[v,...D];return new pe.GraphQLUnionType({name:N,description:(pr=v.description)===null||pr===void 0?void 0:pr.value,types:()=>Nt(tt),astNode:v,extensionASTNodes:D})}case We.Kind.SCALAR_TYPE_DEFINITION:{var es;return new pe.GraphQLScalarType({name:N,description:(es=v.description)===null||es===void 0?void 0:es.value,specifiedByURL:nd(v),astNode:v,extensionASTNodes:D})}case We.Kind.INPUT_OBJECT_TYPE_DEFINITION:{var ts;let tt=[v,...D];return new pe.GraphQLInputObjectType({name:N,description:(ts=v.description)===null||ts===void 0?void 0:ts.value,fields:()=>Le(tt),astNode:v,extensionASTNodes:D})}}}}var rd=(0,TE.keyMap)([...sd.specifiedScalarTypes,...id.introspectionTypes],e=>e.name);function Ui(e){let t=(0,ad.getDirectiveValues)(Ki.GraphQLDeprecatedDirective,e);return t?.reason}function nd(e){let t=(0,ad.getDirectiveValues)(Ki.GraphQLSpecifiedByDirective,e);return t?.url}});var cd=h(Bi=>{"use strict";Object.defineProperty(Bi,"__esModule",{value:!0});Bi.buildASTSchema=ud;Bi.buildSchema=DE;var EE=Fe(),_E=H(),NE=vr(),OE=Ve(),IE=Zt(),LE=bn(),SE=Iu();function ud(e,t){e!=null&&e.kind===_E.Kind.DOCUMENT||(0,EE.devAssert)(!1,"Must provide valid Document AST."),t?.assumeValid!==!0&&t?.assumeValidSDL!==!0&&(0,LE.assertValidSDL)(e);let r={description:void 0,types:[],directives:[],extensions:Object.create(null),extensionASTNodes:[],assumeValid:!1},n=(0,SE.extendSchemaImpl)(r,e,t);if(n.astNode==null)for(let s of n.types)switch(s.name){case"Query":n.query=s;break;case"Mutation":n.mutation=s;break;case"Subscription":n.subscription=s;break}let i=[...n.directives,...OE.specifiedDirectives.filter(s=>n.directives.every(a=>a.name!==s.name))];return new IE.GraphQLSchema({...n,directives:i})}function DE(e,t){let r=(0,NE.parse)(e,{noLocation:t?.noLocation,allowLegacyFragmentVariables:t?.allowLegacyFragmentVariables});return ud(r,{assumeValidSDL:t?.assumeValidSDL,assumeValid:t?.assumeValid})}});var dd=h(Su=>{"use strict";Object.defineProperty(Su,"__esModule",{value:!0});Su.lexicographicSortSchema=kE;var AE=re(),RE=je(),jE=Kr(),ld=$r(),ke=Y(),PE=Ve(),wE=Ye(),FE=Zt();function kE(e){let t=e.toConfig(),r=(0,jE.keyValMap)(Lu(t.types),d=>d.name,f);return new FE.GraphQLSchema({...t,types:Object.values(r),directives:Lu(t.directives).map(a),query:s(t.query),mutation:s(t.mutation),subscription:s(t.subscription)});function n(d){return(0,ke.isListType)(d)?new ke.GraphQLList(n(d.ofType)):(0,ke.isNonNullType)(d)?new ke.GraphQLNonNull(n(d.ofType)):i(d)}function i(d){return r[d.name]}function s(d){return d&&i(d)}function a(d){let y=d.toConfig();return new PE.GraphQLDirective({...y,locations:pd(y.locations,T=>T),args:o(y.args)})}function o(d){return Yi(d,y=>({...y,type:n(y.type)}))}function u(d){return Yi(d,y=>({...y,type:n(y.type),args:y.args&&o(y.args)}))}function c(d){return Yi(d,y=>({...y,type:n(y.type)}))}function l(d){return Lu(d).map(i)}function f(d){if((0,ke.isScalarType)(d)||(0,wE.isIntrospectionType)(d))return d;if((0,ke.isObjectType)(d)){let y=d.toConfig();return new ke.GraphQLObjectType({...y,interfaces:()=>l(y.interfaces),fields:()=>u(y.fields)})}if((0,ke.isInterfaceType)(d)){let y=d.toConfig();return new ke.GraphQLInterfaceType({...y,interfaces:()=>l(y.interfaces),fields:()=>u(y.fields)})}if((0,ke.isUnionType)(d)){let y=d.toConfig();return new ke.GraphQLUnionType({...y,types:()=>l(y.types)})}if((0,ke.isEnumType)(d)){let y=d.toConfig();return new ke.GraphQLEnumType({...y,values:Yi(y.values,T=>T)})}if((0,ke.isInputObjectType)(d)){let y=d.toConfig();return new ke.GraphQLInputObjectType({...y,fields:()=>c(y.fields)})}(0,RE.invariant)(!1,"Unexpected type: "+(0,AE.inspect)(d))}}function Yi(e,t){let r=Object.create(null);for(let n of Object.keys(e).sort(ld.naturalCompare))r[n]=t(e[n]);return r}function Lu(e){return pd(e,t=>t.name)}function pd(e,t){return e.slice().sort((r,n)=>{let i=t(r),s=t(n);return(0,ld.naturalCompare)(i,s)})}});var bd=h(Ln=>{"use strict";Object.defineProperty(Ln,"__esModule",{value:!0});Ln.printIntrospectionSchema=QE;Ln.printSchema=CE;Ln.printType=yd;var qE=re(),xE=je(),VE=qr(),Au=H(),Ji=Ce(),jr=Y(),Ru=Ve(),fd=Ye(),ME=yt(),GE=sn();function CE(e){return md(e,t=>!(0,Ru.isSpecifiedDirective)(t),UE)}function QE(e){return md(e,Ru.isSpecifiedDirective,fd.isIntrospectionType)}function UE(e){return!(0,ME.isSpecifiedScalarType)(e)&&!(0,fd.isIntrospectionType)(e)}function md(e,t,r){let n=e.getDirectives().filter(t),i=Object.values(e.getTypeMap()).filter(r);return[KE(e),...n.map(s=>WE(s)),...i.map(s=>yd(s))].filter(Boolean).join(` + +`)}function KE(e){if(e.description==null&&$E(e))return;let t=[],r=e.getQueryType();r&&t.push(` query: ${r.name}`);let n=e.getMutationType();n&&t.push(` mutation: ${n.name}`);let i=e.getSubscriptionType();return i&&t.push(` subscription: ${i.name}`),Ze(e)+`schema { +${t.join(` +`)} +}`}function $E(e){let t=e.getQueryType();if(t&&t.name!=="Query")return!1;let r=e.getMutationType();if(r&&r.name!=="Mutation")return!1;let n=e.getSubscriptionType();return!(n&&n.name!=="Subscription")}function yd(e){if((0,jr.isScalarType)(e))return BE(e);if((0,jr.isObjectType)(e))return YE(e);if((0,jr.isInterfaceType)(e))return JE(e);if((0,jr.isUnionType)(e))return XE(e);if((0,jr.isEnumType)(e))return HE(e);if((0,jr.isInputObjectType)(e))return zE(e);(0,xE.invariant)(!1,"Unexpected type: "+(0,qE.inspect)(e))}function BE(e){return Ze(e)+`scalar ${e.name}`+ZE(e)}function hd(e){let t=e.getInterfaces();return t.length?" implements "+t.map(r=>r.name).join(" & "):""}function YE(e){return Ze(e)+`type ${e.name}`+hd(e)+vd(e)}function JE(e){return Ze(e)+`interface ${e.name}`+hd(e)+vd(e)}function XE(e){let t=e.getTypes(),r=t.length?" = "+t.join(" | "):"";return Ze(e)+"union "+e.name+r}function HE(e){let t=e.getValues().map((r,n)=>Ze(r," ",!n)+" "+r.name+Pu(r.deprecationReason));return Ze(e)+`enum ${e.name}`+ju(t)}function zE(e){let t=Object.values(e.getFields()).map((r,n)=>Ze(r," ",!n)+" "+Du(r));return Ze(e)+`input ${e.name}`+ju(t)}function vd(e){let t=Object.values(e.getFields()).map((r,n)=>Ze(r," ",!n)+" "+r.name+Td(r.args," ")+": "+String(r.type)+Pu(r.deprecationReason));return ju(t)}function ju(e){return e.length!==0?` { +`+e.join(` +`)+` +}`:""}function Td(e,t=""){return e.length===0?"":e.every(r=>!r.description)?"("+e.map(Du).join(", ")+")":`( +`+e.map((r,n)=>Ze(r," "+t,!n)+" "+t+Du(r)).join(` +`)+` +`+t+")"}function Du(e){let t=(0,GE.astFromValue)(e.defaultValue,e.type),r=e.name+": "+String(e.type);return t&&(r+=` = ${(0,Ji.print)(t)}`),r+Pu(e.deprecationReason)}function WE(e){return Ze(e)+"directive @"+e.name+Td(e.args)+(e.isRepeatable?" repeatable":"")+" on "+e.locations.join(" | ")}function Pu(e){return e==null?"":e!==Ru.DEFAULT_DEPRECATION_REASON?` @deprecated(reason: ${(0,Ji.print)({kind:Au.Kind.STRING,value:e})})`:" @deprecated"}function ZE(e){return e.specifiedByURL==null?"":` @specifiedBy(url: ${(0,Ji.print)({kind:Au.Kind.STRING,value:e.specifiedByURL})})`}function Ze(e,t="",r=!0){let{description:n}=e;if(n==null)return"";let i=(0,Ji.print)({kind:Au.Kind.STRING,value:n,block:(0,VE.isPrintableAsBlockString)(n)});return(t&&!r?` +`+t:t)+i.replace(/\n/g,` +`+t)+` +`}});var gd=h(wu=>{"use strict";Object.defineProperty(wu,"__esModule",{value:!0});wu.concatAST=t_;var e_=H();function t_(e){let t=[];for(let r of e)t.push(...r.definitions);return{kind:e_.Kind.DOCUMENT,definitions:t}}});var Nd=h(Fu=>{"use strict";Object.defineProperty(Fu,"__esModule",{value:!0});Fu.separateOperations=n_;var Xi=H(),r_=Yt();function n_(e){let t=[],r=Object.create(null);for(let i of e.definitions)switch(i.kind){case Xi.Kind.OPERATION_DEFINITION:t.push(i);break;case Xi.Kind.FRAGMENT_DEFINITION:r[i.name.value]=Ed(i.selectionSet);break;default:}let n=Object.create(null);for(let i of t){let s=new Set;for(let o of Ed(i.selectionSet))_d(s,r,o);let a=i.name?i.name.value:"";n[a]={kind:Xi.Kind.DOCUMENT,definitions:e.definitions.filter(o=>o===i||o.kind===Xi.Kind.FRAGMENT_DEFINITION&&s.has(o.name.value))}}return n}function _d(e,t,r){if(!e.has(r)){e.add(r);let n=t[r];if(n!==void 0)for(let i of n)_d(e,t,i)}}function Ed(e){let t=[];return(0,r_.visit)(e,{FragmentSpread(r){t.push(r.name.value)}}),t}});var Ld=h(qu=>{"use strict";Object.defineProperty(qu,"__esModule",{value:!0});qu.stripIgnoredCharacters=s_;var i_=qr(),Od=Mn(),Id=Un(),ku=Vr();function s_(e){let t=(0,Id.isSource)(e)?e:new Id.Source(e),r=t.body,n=new Od.Lexer(t),i="",s=!1;for(;n.advance().kind!==ku.TokenKind.EOF;){let a=n.token,o=a.kind,u=!(0,Od.isPunctuatorTokenKind)(a.kind);s&&(u||a.kind===ku.TokenKind.SPREAD)&&(i+=" ");let c=r.slice(a.start,a.end);o===ku.TokenKind.BLOCK_STRING?i+=(0,i_.printBlockString)(a.value,{minimize:!0}):i+=c,s=u}return i}});var Dd=h(Hi=>{"use strict";Object.defineProperty(Hi,"__esModule",{value:!0});Hi.assertValidName=c_;Hi.isValidNameError=Sd;var a_=Fe(),o_=q(),u_=Br();function c_(e){let t=Sd(e);if(t)throw t;return e}function Sd(e){if(typeof e=="string"||(0,a_.devAssert)(!1,"Expected name to be a string."),e.startsWith("__"))return new o_.GraphQLError(`Name "${e}" must not begin with "__", which is reserved by GraphQL introspection.`);try{(0,u_.assertName)(e)}catch(t){return t}}});var qd=h(Et=>{"use strict";Object.defineProperty(Et,"__esModule",{value:!0});Et.DangerousChangeType=Et.BreakingChangeType=void 0;Et.findBreakingChanges=y_;Et.findDangerousChanges=h_;var l_=re(),Fd=je(),Ad=Ft(),p_=Ce(),Z=Y(),d_=yt(),f_=sn(),m_=Ga(),me;Et.BreakingChangeType=me;(function(e){e.TYPE_REMOVED="TYPE_REMOVED",e.TYPE_CHANGED_KIND="TYPE_CHANGED_KIND",e.TYPE_REMOVED_FROM_UNION="TYPE_REMOVED_FROM_UNION",e.VALUE_REMOVED_FROM_ENUM="VALUE_REMOVED_FROM_ENUM",e.REQUIRED_INPUT_FIELD_ADDED="REQUIRED_INPUT_FIELD_ADDED",e.IMPLEMENTED_INTERFACE_REMOVED="IMPLEMENTED_INTERFACE_REMOVED",e.FIELD_REMOVED="FIELD_REMOVED",e.FIELD_CHANGED_KIND="FIELD_CHANGED_KIND",e.REQUIRED_ARG_ADDED="REQUIRED_ARG_ADDED",e.ARG_REMOVED="ARG_REMOVED",e.ARG_CHANGED_KIND="ARG_CHANGED_KIND",e.DIRECTIVE_REMOVED="DIRECTIVE_REMOVED",e.DIRECTIVE_ARG_REMOVED="DIRECTIVE_ARG_REMOVED",e.REQUIRED_DIRECTIVE_ARG_ADDED="REQUIRED_DIRECTIVE_ARG_ADDED",e.DIRECTIVE_REPEATABLE_REMOVED="DIRECTIVE_REPEATABLE_REMOVED",e.DIRECTIVE_LOCATION_REMOVED="DIRECTIVE_LOCATION_REMOVED"})(me||(Et.BreakingChangeType=me={}));var ct;Et.DangerousChangeType=ct;(function(e){e.VALUE_ADDED_TO_ENUM="VALUE_ADDED_TO_ENUM",e.TYPE_ADDED_TO_UNION="TYPE_ADDED_TO_UNION",e.OPTIONAL_INPUT_FIELD_ADDED="OPTIONAL_INPUT_FIELD_ADDED",e.OPTIONAL_ARG_ADDED="OPTIONAL_ARG_ADDED",e.IMPLEMENTED_INTERFACE_ADDED="IMPLEMENTED_INTERFACE_ADDED",e.ARG_DEFAULT_VALUE_CHANGE="ARG_DEFAULT_VALUE_CHANGE"})(ct||(Et.DangerousChangeType=ct={}));function y_(e,t){return kd(e,t).filter(r=>r.type in me)}function h_(e,t){return kd(e,t).filter(r=>r.type in ct)}function kd(e,t){return[...T_(e,t),...v_(e,t)]}function v_(e,t){let r=[],n=At(e.getDirectives(),t.getDirectives());for(let i of n.removed)r.push({type:me.DIRECTIVE_REMOVED,description:`${i.name} was removed.`});for(let[i,s]of n.persisted){let a=At(i.args,s.args);for(let o of a.added)(0,Z.isRequiredArgument)(o)&&r.push({type:me.REQUIRED_DIRECTIVE_ARG_ADDED,description:`A required arg ${o.name} on directive ${i.name} was added.`});for(let o of a.removed)r.push({type:me.DIRECTIVE_ARG_REMOVED,description:`${o.name} was removed from ${i.name}.`});i.isRepeatable&&!s.isRepeatable&&r.push({type:me.DIRECTIVE_REPEATABLE_REMOVED,description:`Repeatable flag was removed from ${i.name}.`});for(let o of i.locations)s.locations.includes(o)||r.push({type:me.DIRECTIVE_LOCATION_REMOVED,description:`${o} was removed from ${i.name}.`})}return r}function T_(e,t){let r=[],n=At(Object.values(e.getTypeMap()),Object.values(t.getTypeMap()));for(let i of n.removed)r.push({type:me.TYPE_REMOVED,description:(0,d_.isSpecifiedScalarType)(i)?`Standard scalar ${i.name} was removed because it is not referenced anymore.`:`${i.name} was removed.`});for(let[i,s]of n.persisted)(0,Z.isEnumType)(i)&&(0,Z.isEnumType)(s)?r.push(...E_(i,s)):(0,Z.isUnionType)(i)&&(0,Z.isUnionType)(s)?r.push(...g_(i,s)):(0,Z.isInputObjectType)(i)&&(0,Z.isInputObjectType)(s)?r.push(...b_(i,s)):(0,Z.isObjectType)(i)&&(0,Z.isObjectType)(s)?r.push(...jd(i,s),...Rd(i,s)):(0,Z.isInterfaceType)(i)&&(0,Z.isInterfaceType)(s)?r.push(...jd(i,s),...Rd(i,s)):i.constructor!==s.constructor&&r.push({type:me.TYPE_CHANGED_KIND,description:`${i.name} changed from ${Pd(i)} to ${Pd(s)}.`});return r}function b_(e,t){let r=[],n=At(Object.values(e.getFields()),Object.values(t.getFields()));for(let i of n.added)(0,Z.isRequiredInputField)(i)?r.push({type:me.REQUIRED_INPUT_FIELD_ADDED,description:`A required field ${i.name} on input type ${e.name} was added.`}):r.push({type:ct.OPTIONAL_INPUT_FIELD_ADDED,description:`An optional field ${i.name} on input type ${e.name} was added.`});for(let i of n.removed)r.push({type:me.FIELD_REMOVED,description:`${e.name}.${i.name} was removed.`});for(let[i,s]of n.persisted)Dn(i.type,s.type)||r.push({type:me.FIELD_CHANGED_KIND,description:`${e.name}.${i.name} changed type from ${String(i.type)} to ${String(s.type)}.`});return r}function g_(e,t){let r=[],n=At(e.getTypes(),t.getTypes());for(let i of n.added)r.push({type:ct.TYPE_ADDED_TO_UNION,description:`${i.name} was added to union type ${e.name}.`});for(let i of n.removed)r.push({type:me.TYPE_REMOVED_FROM_UNION,description:`${i.name} was removed from union type ${e.name}.`});return r}function E_(e,t){let r=[],n=At(e.getValues(),t.getValues());for(let i of n.added)r.push({type:ct.VALUE_ADDED_TO_ENUM,description:`${i.name} was added to enum type ${e.name}.`});for(let i of n.removed)r.push({type:me.VALUE_REMOVED_FROM_ENUM,description:`${i.name} was removed from enum type ${e.name}.`});return r}function Rd(e,t){let r=[],n=At(e.getInterfaces(),t.getInterfaces());for(let i of n.added)r.push({type:ct.IMPLEMENTED_INTERFACE_ADDED,description:`${i.name} added to interfaces implemented by ${e.name}.`});for(let i of n.removed)r.push({type:me.IMPLEMENTED_INTERFACE_REMOVED,description:`${e.name} no longer implements interface ${i.name}.`});return r}function jd(e,t){let r=[],n=At(Object.values(e.getFields()),Object.values(t.getFields()));for(let i of n.removed)r.push({type:me.FIELD_REMOVED,description:`${e.name}.${i.name} was removed.`});for(let[i,s]of n.persisted)r.push(...__(e,i,s)),Sn(i.type,s.type)||r.push({type:me.FIELD_CHANGED_KIND,description:`${e.name}.${i.name} changed type from ${String(i.type)} to ${String(s.type)}.`});return r}function __(e,t,r){let n=[],i=At(t.args,r.args);for(let s of i.removed)n.push({type:me.ARG_REMOVED,description:`${e.name}.${t.name} arg ${s.name} was removed.`});for(let[s,a]of i.persisted)if(!Dn(s.type,a.type))n.push({type:me.ARG_CHANGED_KIND,description:`${e.name}.${t.name} arg ${s.name} has changed type from ${String(s.type)} to ${String(a.type)}.`});else if(s.defaultValue!==void 0)if(a.defaultValue===void 0)n.push({type:ct.ARG_DEFAULT_VALUE_CHANGE,description:`${e.name}.${t.name} arg ${s.name} defaultValue was removed.`});else{let u=wd(s.defaultValue,s.type),c=wd(a.defaultValue,a.type);u!==c&&n.push({type:ct.ARG_DEFAULT_VALUE_CHANGE,description:`${e.name}.${t.name} arg ${s.name} has changed defaultValue from ${u} to ${c}.`})}for(let s of i.added)(0,Z.isRequiredArgument)(s)?n.push({type:me.REQUIRED_ARG_ADDED,description:`A required arg ${s.name} on ${e.name}.${t.name} was added.`}):n.push({type:ct.OPTIONAL_ARG_ADDED,description:`An optional arg ${s.name} on ${e.name}.${t.name} was added.`});return n}function Sn(e,t){return(0,Z.isListType)(e)?(0,Z.isListType)(t)&&Sn(e.ofType,t.ofType)||(0,Z.isNonNullType)(t)&&Sn(e,t.ofType):(0,Z.isNonNullType)(e)?(0,Z.isNonNullType)(t)&&Sn(e.ofType,t.ofType):(0,Z.isNamedType)(t)&&e.name===t.name||(0,Z.isNonNullType)(t)&&Sn(e,t.ofType)}function Dn(e,t){return(0,Z.isListType)(e)?(0,Z.isListType)(t)&&Dn(e.ofType,t.ofType):(0,Z.isNonNullType)(e)?(0,Z.isNonNullType)(t)&&Dn(e.ofType,t.ofType)||!(0,Z.isNonNullType)(t)&&Dn(e.ofType,t):(0,Z.isNamedType)(t)&&e.name===t.name}function Pd(e){if((0,Z.isScalarType)(e))return"a Scalar type";if((0,Z.isObjectType)(e))return"an Object type";if((0,Z.isInterfaceType)(e))return"an Interface type";if((0,Z.isUnionType)(e))return"a Union type";if((0,Z.isEnumType)(e))return"an Enum type";if((0,Z.isInputObjectType)(e))return"an Input type";(0,Fd.invariant)(!1,"Unexpected type: "+(0,l_.inspect)(e))}function wd(e,t){let r=(0,f_.astFromValue)(e,t);return r!=null||(0,Fd.invariant)(!1),(0,p_.print)((0,m_.sortValueNode)(r))}function At(e,t){let r=[],n=[],i=[],s=(0,Ad.keyMap)(e,({name:o})=>o),a=(0,Ad.keyMap)(t,({name:o})=>o);for(let o of e){let u=a[o.name];u===void 0?n.push(o):i.push([o,u])}for(let o of t)s[o.name]===void 0&&r.push(o);return{added:r,persisted:i,removed:n}}});var Gd=h(W=>{"use strict";Object.defineProperty(W,"__esModule",{value:!0});Object.defineProperty(W,"BreakingChangeType",{enumerable:!0,get:function(){return zi.BreakingChangeType}});Object.defineProperty(W,"DangerousChangeType",{enumerable:!0,get:function(){return zi.DangerousChangeType}});Object.defineProperty(W,"TypeInfo",{enumerable:!0,get:function(){return Vd.TypeInfo}});Object.defineProperty(W,"assertValidName",{enumerable:!0,get:function(){return Md.assertValidName}});Object.defineProperty(W,"astFromValue",{enumerable:!0,get:function(){return w_.astFromValue}});Object.defineProperty(W,"buildASTSchema",{enumerable:!0,get:function(){return xd.buildASTSchema}});Object.defineProperty(W,"buildClientSchema",{enumerable:!0,get:function(){return S_.buildClientSchema}});Object.defineProperty(W,"buildSchema",{enumerable:!0,get:function(){return xd.buildSchema}});Object.defineProperty(W,"coerceInputValue",{enumerable:!0,get:function(){return F_.coerceInputValue}});Object.defineProperty(W,"concatAST",{enumerable:!0,get:function(){return k_.concatAST}});Object.defineProperty(W,"doTypesOverlap",{enumerable:!0,get:function(){return Vu.doTypesOverlap}});Object.defineProperty(W,"extendSchema",{enumerable:!0,get:function(){return D_.extendSchema}});Object.defineProperty(W,"findBreakingChanges",{enumerable:!0,get:function(){return zi.findBreakingChanges}});Object.defineProperty(W,"findDangerousChanges",{enumerable:!0,get:function(){return zi.findDangerousChanges}});Object.defineProperty(W,"getIntrospectionQuery",{enumerable:!0,get:function(){return N_.getIntrospectionQuery}});Object.defineProperty(W,"getOperationAST",{enumerable:!0,get:function(){return O_.getOperationAST}});Object.defineProperty(W,"getOperationRootType",{enumerable:!0,get:function(){return I_.getOperationRootType}});Object.defineProperty(W,"introspectionFromSchema",{enumerable:!0,get:function(){return L_.introspectionFromSchema}});Object.defineProperty(W,"isEqualType",{enumerable:!0,get:function(){return Vu.isEqualType}});Object.defineProperty(W,"isTypeSubTypeOf",{enumerable:!0,get:function(){return Vu.isTypeSubTypeOf}});Object.defineProperty(W,"isValidNameError",{enumerable:!0,get:function(){return Md.isValidNameError}});Object.defineProperty(W,"lexicographicSortSchema",{enumerable:!0,get:function(){return A_.lexicographicSortSchema}});Object.defineProperty(W,"printIntrospectionSchema",{enumerable:!0,get:function(){return xu.printIntrospectionSchema}});Object.defineProperty(W,"printSchema",{enumerable:!0,get:function(){return xu.printSchema}});Object.defineProperty(W,"printType",{enumerable:!0,get:function(){return xu.printType}});Object.defineProperty(W,"separateOperations",{enumerable:!0,get:function(){return q_.separateOperations}});Object.defineProperty(W,"stripIgnoredCharacters",{enumerable:!0,get:function(){return x_.stripIgnoredCharacters}});Object.defineProperty(W,"typeFromAST",{enumerable:!0,get:function(){return R_.typeFromAST}});Object.defineProperty(W,"valueFromAST",{enumerable:!0,get:function(){return j_.valueFromAST}});Object.defineProperty(W,"valueFromASTUntyped",{enumerable:!0,get:function(){return P_.valueFromASTUntyped}});Object.defineProperty(W,"visitWithTypeInfo",{enumerable:!0,get:function(){return Vd.visitWithTypeInfo}});var N_=gu(),O_=Jp(),I_=Xp(),L_=Hp(),S_=Wp(),xd=cd(),D_=Iu(),A_=dd(),xu=bd(),R_=ht(),j_=yn(),P_=Vs(),w_=sn(),Vd=vi(),F_=ao(),k_=gd(),q_=Nd(),x_=Ld(),Vu=zr(),Md=Dd(),zi=qd()});var Ud=h(p=>{"use strict";Object.defineProperty(p,"__esModule",{value:!0});Object.defineProperty(p,"BREAK",{enumerable:!0,get:function(){return ee.BREAK}});Object.defineProperty(p,"BreakingChangeType",{enumerable:!0,get:function(){return te.BreakingChangeType}});Object.defineProperty(p,"DEFAULT_DEPRECATION_REASON",{enumerable:!0,get:function(){return g.DEFAULT_DEPRECATION_REASON}});Object.defineProperty(p,"DangerousChangeType",{enumerable:!0,get:function(){return te.DangerousChangeType}});Object.defineProperty(p,"DirectiveLocation",{enumerable:!0,get:function(){return ee.DirectiveLocation}});Object.defineProperty(p,"ExecutableDefinitionsRule",{enumerable:!0,get:function(){return K.ExecutableDefinitionsRule}});Object.defineProperty(p,"FieldsOnCorrectTypeRule",{enumerable:!0,get:function(){return K.FieldsOnCorrectTypeRule}});Object.defineProperty(p,"FragmentsOnCompositeTypesRule",{enumerable:!0,get:function(){return K.FragmentsOnCompositeTypesRule}});Object.defineProperty(p,"GRAPHQL_MAX_INT",{enumerable:!0,get:function(){return g.GRAPHQL_MAX_INT}});Object.defineProperty(p,"GRAPHQL_MIN_INT",{enumerable:!0,get:function(){return g.GRAPHQL_MIN_INT}});Object.defineProperty(p,"GraphQLBoolean",{enumerable:!0,get:function(){return g.GraphQLBoolean}});Object.defineProperty(p,"GraphQLDeprecatedDirective",{enumerable:!0,get:function(){return g.GraphQLDeprecatedDirective}});Object.defineProperty(p,"GraphQLDirective",{enumerable:!0,get:function(){return g.GraphQLDirective}});Object.defineProperty(p,"GraphQLEnumType",{enumerable:!0,get:function(){return g.GraphQLEnumType}});Object.defineProperty(p,"GraphQLError",{enumerable:!0,get:function(){return An.GraphQLError}});Object.defineProperty(p,"GraphQLFloat",{enumerable:!0,get:function(){return g.GraphQLFloat}});Object.defineProperty(p,"GraphQLID",{enumerable:!0,get:function(){return g.GraphQLID}});Object.defineProperty(p,"GraphQLIncludeDirective",{enumerable:!0,get:function(){return g.GraphQLIncludeDirective}});Object.defineProperty(p,"GraphQLInputObjectType",{enumerable:!0,get:function(){return g.GraphQLInputObjectType}});Object.defineProperty(p,"GraphQLInt",{enumerable:!0,get:function(){return g.GraphQLInt}});Object.defineProperty(p,"GraphQLInterfaceType",{enumerable:!0,get:function(){return g.GraphQLInterfaceType}});Object.defineProperty(p,"GraphQLList",{enumerable:!0,get:function(){return g.GraphQLList}});Object.defineProperty(p,"GraphQLNonNull",{enumerable:!0,get:function(){return g.GraphQLNonNull}});Object.defineProperty(p,"GraphQLObjectType",{enumerable:!0,get:function(){return g.GraphQLObjectType}});Object.defineProperty(p,"GraphQLScalarType",{enumerable:!0,get:function(){return g.GraphQLScalarType}});Object.defineProperty(p,"GraphQLSchema",{enumerable:!0,get:function(){return g.GraphQLSchema}});Object.defineProperty(p,"GraphQLSkipDirective",{enumerable:!0,get:function(){return g.GraphQLSkipDirective}});Object.defineProperty(p,"GraphQLSpecifiedByDirective",{enumerable:!0,get:function(){return g.GraphQLSpecifiedByDirective}});Object.defineProperty(p,"GraphQLString",{enumerable:!0,get:function(){return g.GraphQLString}});Object.defineProperty(p,"GraphQLUnionType",{enumerable:!0,get:function(){return g.GraphQLUnionType}});Object.defineProperty(p,"Kind",{enumerable:!0,get:function(){return ee.Kind}});Object.defineProperty(p,"KnownArgumentNamesRule",{enumerable:!0,get:function(){return K.KnownArgumentNamesRule}});Object.defineProperty(p,"KnownDirectivesRule",{enumerable:!0,get:function(){return K.KnownDirectivesRule}});Object.defineProperty(p,"KnownFragmentNamesRule",{enumerable:!0,get:function(){return K.KnownFragmentNamesRule}});Object.defineProperty(p,"KnownTypeNamesRule",{enumerable:!0,get:function(){return K.KnownTypeNamesRule}});Object.defineProperty(p,"Lexer",{enumerable:!0,get:function(){return ee.Lexer}});Object.defineProperty(p,"Location",{enumerable:!0,get:function(){return ee.Location}});Object.defineProperty(p,"LoneAnonymousOperationRule",{enumerable:!0,get:function(){return K.LoneAnonymousOperationRule}});Object.defineProperty(p,"LoneSchemaDefinitionRule",{enumerable:!0,get:function(){return K.LoneSchemaDefinitionRule}});Object.defineProperty(p,"NoDeprecatedCustomRule",{enumerable:!0,get:function(){return K.NoDeprecatedCustomRule}});Object.defineProperty(p,"NoFragmentCyclesRule",{enumerable:!0,get:function(){return K.NoFragmentCyclesRule}});Object.defineProperty(p,"NoSchemaIntrospectionCustomRule",{enumerable:!0,get:function(){return K.NoSchemaIntrospectionCustomRule}});Object.defineProperty(p,"NoUndefinedVariablesRule",{enumerable:!0,get:function(){return K.NoUndefinedVariablesRule}});Object.defineProperty(p,"NoUnusedFragmentsRule",{enumerable:!0,get:function(){return K.NoUnusedFragmentsRule}});Object.defineProperty(p,"NoUnusedVariablesRule",{enumerable:!0,get:function(){return K.NoUnusedVariablesRule}});Object.defineProperty(p,"OperationTypeNode",{enumerable:!0,get:function(){return ee.OperationTypeNode}});Object.defineProperty(p,"OverlappingFieldsCanBeMergedRule",{enumerable:!0,get:function(){return K.OverlappingFieldsCanBeMergedRule}});Object.defineProperty(p,"PossibleFragmentSpreadsRule",{enumerable:!0,get:function(){return K.PossibleFragmentSpreadsRule}});Object.defineProperty(p,"PossibleTypeExtensionsRule",{enumerable:!0,get:function(){return K.PossibleTypeExtensionsRule}});Object.defineProperty(p,"ProvidedRequiredArgumentsRule",{enumerable:!0,get:function(){return K.ProvidedRequiredArgumentsRule}});Object.defineProperty(p,"ScalarLeafsRule",{enumerable:!0,get:function(){return K.ScalarLeafsRule}});Object.defineProperty(p,"SchemaMetaFieldDef",{enumerable:!0,get:function(){return g.SchemaMetaFieldDef}});Object.defineProperty(p,"SingleFieldSubscriptionsRule",{enumerable:!0,get:function(){return K.SingleFieldSubscriptionsRule}});Object.defineProperty(p,"Source",{enumerable:!0,get:function(){return ee.Source}});Object.defineProperty(p,"Token",{enumerable:!0,get:function(){return ee.Token}});Object.defineProperty(p,"TokenKind",{enumerable:!0,get:function(){return ee.TokenKind}});Object.defineProperty(p,"TypeInfo",{enumerable:!0,get:function(){return te.TypeInfo}});Object.defineProperty(p,"TypeKind",{enumerable:!0,get:function(){return g.TypeKind}});Object.defineProperty(p,"TypeMetaFieldDef",{enumerable:!0,get:function(){return g.TypeMetaFieldDef}});Object.defineProperty(p,"TypeNameMetaFieldDef",{enumerable:!0,get:function(){return g.TypeNameMetaFieldDef}});Object.defineProperty(p,"UniqueArgumentDefinitionNamesRule",{enumerable:!0,get:function(){return K.UniqueArgumentDefinitionNamesRule}});Object.defineProperty(p,"UniqueArgumentNamesRule",{enumerable:!0,get:function(){return K.UniqueArgumentNamesRule}});Object.defineProperty(p,"UniqueDirectiveNamesRule",{enumerable:!0,get:function(){return K.UniqueDirectiveNamesRule}});Object.defineProperty(p,"UniqueDirectivesPerLocationRule",{enumerable:!0,get:function(){return K.UniqueDirectivesPerLocationRule}});Object.defineProperty(p,"UniqueEnumValueNamesRule",{enumerable:!0,get:function(){return K.UniqueEnumValueNamesRule}});Object.defineProperty(p,"UniqueFieldDefinitionNamesRule",{enumerable:!0,get:function(){return K.UniqueFieldDefinitionNamesRule}});Object.defineProperty(p,"UniqueFragmentNamesRule",{enumerable:!0,get:function(){return K.UniqueFragmentNamesRule}});Object.defineProperty(p,"UniqueInputFieldNamesRule",{enumerable:!0,get:function(){return K.UniqueInputFieldNamesRule}});Object.defineProperty(p,"UniqueOperationNamesRule",{enumerable:!0,get:function(){return K.UniqueOperationNamesRule}});Object.defineProperty(p,"UniqueOperationTypesRule",{enumerable:!0,get:function(){return K.UniqueOperationTypesRule}});Object.defineProperty(p,"UniqueTypeNamesRule",{enumerable:!0,get:function(){return K.UniqueTypeNamesRule}});Object.defineProperty(p,"UniqueVariableNamesRule",{enumerable:!0,get:function(){return K.UniqueVariableNamesRule}});Object.defineProperty(p,"ValidationContext",{enumerable:!0,get:function(){return K.ValidationContext}});Object.defineProperty(p,"ValuesOfCorrectTypeRule",{enumerable:!0,get:function(){return K.ValuesOfCorrectTypeRule}});Object.defineProperty(p,"VariablesAreInputTypesRule",{enumerable:!0,get:function(){return K.VariablesAreInputTypesRule}});Object.defineProperty(p,"VariablesInAllowedPositionRule",{enumerable:!0,get:function(){return K.VariablesInAllowedPositionRule}});Object.defineProperty(p,"__Directive",{enumerable:!0,get:function(){return g.__Directive}});Object.defineProperty(p,"__DirectiveLocation",{enumerable:!0,get:function(){return g.__DirectiveLocation}});Object.defineProperty(p,"__EnumValue",{enumerable:!0,get:function(){return g.__EnumValue}});Object.defineProperty(p,"__Field",{enumerable:!0,get:function(){return g.__Field}});Object.defineProperty(p,"__InputValue",{enumerable:!0,get:function(){return g.__InputValue}});Object.defineProperty(p,"__Schema",{enumerable:!0,get:function(){return g.__Schema}});Object.defineProperty(p,"__Type",{enumerable:!0,get:function(){return g.__Type}});Object.defineProperty(p,"__TypeKind",{enumerable:!0,get:function(){return g.__TypeKind}});Object.defineProperty(p,"assertAbstractType",{enumerable:!0,get:function(){return g.assertAbstractType}});Object.defineProperty(p,"assertCompositeType",{enumerable:!0,get:function(){return g.assertCompositeType}});Object.defineProperty(p,"assertDirective",{enumerable:!0,get:function(){return g.assertDirective}});Object.defineProperty(p,"assertEnumType",{enumerable:!0,get:function(){return g.assertEnumType}});Object.defineProperty(p,"assertEnumValueName",{enumerable:!0,get:function(){return g.assertEnumValueName}});Object.defineProperty(p,"assertInputObjectType",{enumerable:!0,get:function(){return g.assertInputObjectType}});Object.defineProperty(p,"assertInputType",{enumerable:!0,get:function(){return g.assertInputType}});Object.defineProperty(p,"assertInterfaceType",{enumerable:!0,get:function(){return g.assertInterfaceType}});Object.defineProperty(p,"assertLeafType",{enumerable:!0,get:function(){return g.assertLeafType}});Object.defineProperty(p,"assertListType",{enumerable:!0,get:function(){return g.assertListType}});Object.defineProperty(p,"assertName",{enumerable:!0,get:function(){return g.assertName}});Object.defineProperty(p,"assertNamedType",{enumerable:!0,get:function(){return g.assertNamedType}});Object.defineProperty(p,"assertNonNullType",{enumerable:!0,get:function(){return g.assertNonNullType}});Object.defineProperty(p,"assertNullableType",{enumerable:!0,get:function(){return g.assertNullableType}});Object.defineProperty(p,"assertObjectType",{enumerable:!0,get:function(){return g.assertObjectType}});Object.defineProperty(p,"assertOutputType",{enumerable:!0,get:function(){return g.assertOutputType}});Object.defineProperty(p,"assertScalarType",{enumerable:!0,get:function(){return g.assertScalarType}});Object.defineProperty(p,"assertSchema",{enumerable:!0,get:function(){return g.assertSchema}});Object.defineProperty(p,"assertType",{enumerable:!0,get:function(){return g.assertType}});Object.defineProperty(p,"assertUnionType",{enumerable:!0,get:function(){return g.assertUnionType}});Object.defineProperty(p,"assertValidName",{enumerable:!0,get:function(){return te.assertValidName}});Object.defineProperty(p,"assertValidSchema",{enumerable:!0,get:function(){return g.assertValidSchema}});Object.defineProperty(p,"assertWrappingType",{enumerable:!0,get:function(){return g.assertWrappingType}});Object.defineProperty(p,"astFromValue",{enumerable:!0,get:function(){return te.astFromValue}});Object.defineProperty(p,"buildASTSchema",{enumerable:!0,get:function(){return te.buildASTSchema}});Object.defineProperty(p,"buildClientSchema",{enumerable:!0,get:function(){return te.buildClientSchema}});Object.defineProperty(p,"buildSchema",{enumerable:!0,get:function(){return te.buildSchema}});Object.defineProperty(p,"coerceInputValue",{enumerable:!0,get:function(){return te.coerceInputValue}});Object.defineProperty(p,"concatAST",{enumerable:!0,get:function(){return te.concatAST}});Object.defineProperty(p,"createSourceEventStream",{enumerable:!0,get:function(){return _t.createSourceEventStream}});Object.defineProperty(p,"defaultFieldResolver",{enumerable:!0,get:function(){return _t.defaultFieldResolver}});Object.defineProperty(p,"defaultTypeResolver",{enumerable:!0,get:function(){return _t.defaultTypeResolver}});Object.defineProperty(p,"doTypesOverlap",{enumerable:!0,get:function(){return te.doTypesOverlap}});Object.defineProperty(p,"execute",{enumerable:!0,get:function(){return _t.execute}});Object.defineProperty(p,"executeSync",{enumerable:!0,get:function(){return _t.executeSync}});Object.defineProperty(p,"extendSchema",{enumerable:!0,get:function(){return te.extendSchema}});Object.defineProperty(p,"findBreakingChanges",{enumerable:!0,get:function(){return te.findBreakingChanges}});Object.defineProperty(p,"findDangerousChanges",{enumerable:!0,get:function(){return te.findDangerousChanges}});Object.defineProperty(p,"formatError",{enumerable:!0,get:function(){return An.formatError}});Object.defineProperty(p,"getArgumentValues",{enumerable:!0,get:function(){return _t.getArgumentValues}});Object.defineProperty(p,"getDirectiveValues",{enumerable:!0,get:function(){return _t.getDirectiveValues}});Object.defineProperty(p,"getEnterLeaveForKind",{enumerable:!0,get:function(){return ee.getEnterLeaveForKind}});Object.defineProperty(p,"getIntrospectionQuery",{enumerable:!0,get:function(){return te.getIntrospectionQuery}});Object.defineProperty(p,"getLocation",{enumerable:!0,get:function(){return ee.getLocation}});Object.defineProperty(p,"getNamedType",{enumerable:!0,get:function(){return g.getNamedType}});Object.defineProperty(p,"getNullableType",{enumerable:!0,get:function(){return g.getNullableType}});Object.defineProperty(p,"getOperationAST",{enumerable:!0,get:function(){return te.getOperationAST}});Object.defineProperty(p,"getOperationRootType",{enumerable:!0,get:function(){return te.getOperationRootType}});Object.defineProperty(p,"getVariableValues",{enumerable:!0,get:function(){return _t.getVariableValues}});Object.defineProperty(p,"getVisitFn",{enumerable:!0,get:function(){return ee.getVisitFn}});Object.defineProperty(p,"graphql",{enumerable:!0,get:function(){return Qd.graphql}});Object.defineProperty(p,"graphqlSync",{enumerable:!0,get:function(){return Qd.graphqlSync}});Object.defineProperty(p,"introspectionFromSchema",{enumerable:!0,get:function(){return te.introspectionFromSchema}});Object.defineProperty(p,"introspectionTypes",{enumerable:!0,get:function(){return g.introspectionTypes}});Object.defineProperty(p,"isAbstractType",{enumerable:!0,get:function(){return g.isAbstractType}});Object.defineProperty(p,"isCompositeType",{enumerable:!0,get:function(){return g.isCompositeType}});Object.defineProperty(p,"isConstValueNode",{enumerable:!0,get:function(){return ee.isConstValueNode}});Object.defineProperty(p,"isDefinitionNode",{enumerable:!0,get:function(){return ee.isDefinitionNode}});Object.defineProperty(p,"isDirective",{enumerable:!0,get:function(){return g.isDirective}});Object.defineProperty(p,"isEnumType",{enumerable:!0,get:function(){return g.isEnumType}});Object.defineProperty(p,"isEqualType",{enumerable:!0,get:function(){return te.isEqualType}});Object.defineProperty(p,"isExecutableDefinitionNode",{enumerable:!0,get:function(){return ee.isExecutableDefinitionNode}});Object.defineProperty(p,"isInputObjectType",{enumerable:!0,get:function(){return g.isInputObjectType}});Object.defineProperty(p,"isInputType",{enumerable:!0,get:function(){return g.isInputType}});Object.defineProperty(p,"isInterfaceType",{enumerable:!0,get:function(){return g.isInterfaceType}});Object.defineProperty(p,"isIntrospectionType",{enumerable:!0,get:function(){return g.isIntrospectionType}});Object.defineProperty(p,"isLeafType",{enumerable:!0,get:function(){return g.isLeafType}});Object.defineProperty(p,"isListType",{enumerable:!0,get:function(){return g.isListType}});Object.defineProperty(p,"isNamedType",{enumerable:!0,get:function(){return g.isNamedType}});Object.defineProperty(p,"isNonNullType",{enumerable:!0,get:function(){return g.isNonNullType}});Object.defineProperty(p,"isNullableType",{enumerable:!0,get:function(){return g.isNullableType}});Object.defineProperty(p,"isObjectType",{enumerable:!0,get:function(){return g.isObjectType}});Object.defineProperty(p,"isOutputType",{enumerable:!0,get:function(){return g.isOutputType}});Object.defineProperty(p,"isRequiredArgument",{enumerable:!0,get:function(){return g.isRequiredArgument}});Object.defineProperty(p,"isRequiredInputField",{enumerable:!0,get:function(){return g.isRequiredInputField}});Object.defineProperty(p,"isScalarType",{enumerable:!0,get:function(){return g.isScalarType}});Object.defineProperty(p,"isSchema",{enumerable:!0,get:function(){return g.isSchema}});Object.defineProperty(p,"isSelectionNode",{enumerable:!0,get:function(){return ee.isSelectionNode}});Object.defineProperty(p,"isSpecifiedDirective",{enumerable:!0,get:function(){return g.isSpecifiedDirective}});Object.defineProperty(p,"isSpecifiedScalarType",{enumerable:!0,get:function(){return g.isSpecifiedScalarType}});Object.defineProperty(p,"isType",{enumerable:!0,get:function(){return g.isType}});Object.defineProperty(p,"isTypeDefinitionNode",{enumerable:!0,get:function(){return ee.isTypeDefinitionNode}});Object.defineProperty(p,"isTypeExtensionNode",{enumerable:!0,get:function(){return ee.isTypeExtensionNode}});Object.defineProperty(p,"isTypeNode",{enumerable:!0,get:function(){return ee.isTypeNode}});Object.defineProperty(p,"isTypeSubTypeOf",{enumerable:!0,get:function(){return te.isTypeSubTypeOf}});Object.defineProperty(p,"isTypeSystemDefinitionNode",{enumerable:!0,get:function(){return ee.isTypeSystemDefinitionNode}});Object.defineProperty(p,"isTypeSystemExtensionNode",{enumerable:!0,get:function(){return ee.isTypeSystemExtensionNode}});Object.defineProperty(p,"isUnionType",{enumerable:!0,get:function(){return g.isUnionType}});Object.defineProperty(p,"isValidNameError",{enumerable:!0,get:function(){return te.isValidNameError}});Object.defineProperty(p,"isValueNode",{enumerable:!0,get:function(){return ee.isValueNode}});Object.defineProperty(p,"isWrappingType",{enumerable:!0,get:function(){return g.isWrappingType}});Object.defineProperty(p,"lexicographicSortSchema",{enumerable:!0,get:function(){return te.lexicographicSortSchema}});Object.defineProperty(p,"locatedError",{enumerable:!0,get:function(){return An.locatedError}});Object.defineProperty(p,"parse",{enumerable:!0,get:function(){return ee.parse}});Object.defineProperty(p,"parseConstValue",{enumerable:!0,get:function(){return ee.parseConstValue}});Object.defineProperty(p,"parseType",{enumerable:!0,get:function(){return ee.parseType}});Object.defineProperty(p,"parseValue",{enumerable:!0,get:function(){return ee.parseValue}});Object.defineProperty(p,"print",{enumerable:!0,get:function(){return ee.print}});Object.defineProperty(p,"printError",{enumerable:!0,get:function(){return An.printError}});Object.defineProperty(p,"printIntrospectionSchema",{enumerable:!0,get:function(){return te.printIntrospectionSchema}});Object.defineProperty(p,"printLocation",{enumerable:!0,get:function(){return ee.printLocation}});Object.defineProperty(p,"printSchema",{enumerable:!0,get:function(){return te.printSchema}});Object.defineProperty(p,"printSourceLocation",{enumerable:!0,get:function(){return ee.printSourceLocation}});Object.defineProperty(p,"printType",{enumerable:!0,get:function(){return te.printType}});Object.defineProperty(p,"resolveObjMapThunk",{enumerable:!0,get:function(){return g.resolveObjMapThunk}});Object.defineProperty(p,"resolveReadonlyArrayThunk",{enumerable:!0,get:function(){return g.resolveReadonlyArrayThunk}});Object.defineProperty(p,"responsePathAsArray",{enumerable:!0,get:function(){return _t.responsePathAsArray}});Object.defineProperty(p,"separateOperations",{enumerable:!0,get:function(){return te.separateOperations}});Object.defineProperty(p,"specifiedDirectives",{enumerable:!0,get:function(){return g.specifiedDirectives}});Object.defineProperty(p,"specifiedRules",{enumerable:!0,get:function(){return K.specifiedRules}});Object.defineProperty(p,"specifiedScalarTypes",{enumerable:!0,get:function(){return g.specifiedScalarTypes}});Object.defineProperty(p,"stripIgnoredCharacters",{enumerable:!0,get:function(){return te.stripIgnoredCharacters}});Object.defineProperty(p,"subscribe",{enumerable:!0,get:function(){return _t.subscribe}});Object.defineProperty(p,"syntaxError",{enumerable:!0,get:function(){return An.syntaxError}});Object.defineProperty(p,"typeFromAST",{enumerable:!0,get:function(){return te.typeFromAST}});Object.defineProperty(p,"validate",{enumerable:!0,get:function(){return K.validate}});Object.defineProperty(p,"validateSchema",{enumerable:!0,get:function(){return g.validateSchema}});Object.defineProperty(p,"valueFromAST",{enumerable:!0,get:function(){return te.valueFromAST}});Object.defineProperty(p,"valueFromASTUntyped",{enumerable:!0,get:function(){return te.valueFromASTUntyped}});Object.defineProperty(p,"version",{enumerable:!0,get:function(){return Cd.version}});Object.defineProperty(p,"versionInfo",{enumerable:!0,get:function(){return Cd.versionInfo}});Object.defineProperty(p,"visit",{enumerable:!0,get:function(){return ee.visit}});Object.defineProperty(p,"visitInParallel",{enumerable:!0,get:function(){return ee.visitInParallel}});Object.defineProperty(p,"visitWithTypeInfo",{enumerable:!0,get:function(){return te.visitWithTypeInfo}});var Cd=Gu(),Qd=Rp(),g=wp(),ee=kp(),_t=Up(),K=Bp(),An=Yp(),te=Gd()});var Yd=h(Pr=>{"use strict";Object.defineProperty(Pr,"__esModule",{value:!0});Pr.isExecutionResult=Pr.createHandler=void 0;var Rn=Ud(),$d=Pn(),et=dr();function V_(e){let{validate:t=Rn.validate,execute:r=Rn.execute,subscribe:n=Rn.subscribe,schema:i,authenticate:s=function(E){var L;let F=E.headers.get(et.TOKEN_HEADER_KEY);if(F)return Array.isArray(F)?F.join(""):F;let k=new URL((L=E.url)!==null&&L!==void 0?L:"","http://localhost/").searchParams.get(et.TOKEN_QUERY_KEY);return k||"xxxxxxxx-xxxx-4xxx-yxxx-xxxxxxxxxxxx".replace(/[xy]/g,P=>{let G=Math.random()*16|0;return(P=="x"?G:G&3|8).toString(16)})},onConnect:a,context:o,onSubscribe:u,onOperation:c,onNext:l,onComplete:f}=e,d={};function y(I){let E={},L,F=(()=>{let P=[],G={done:!1,error:null,resolve:()=>{}};async function C(){clearInterval(L),typeof I=="string"&&delete d[I];for(let V of Object.values(E))(0,et.isAsyncGenerator)(V)&&await V.return(void 0)}let w=async function*(){for(;;){for(P.length||await new Promise(m=>G.resolve=m);P.length;)yield P.shift();if(G.error)throw G.error;if(G.done)return}}();return w.throw=async V=>(G.done||(G.done=!0,G.error=V,G.resolve(),await C()),{done:!0,value:void 0}),w.return=async()=>(G.done||(G.done=!0,G.resolve(),await C()),{done:!0,value:void 0}),{next(V){P.push(V),G.resolve()},iterator:w}})(),k=!1;return{get open(){return k},ops:E,subscribe(){return k=!0,F.next(`: + +`),L=setInterval(()=>F.next(`: + +`),12e3),F.iterator},from(P,G,C,w){(async()=>{if((0,et.isAsyncIterable)(C))for await(let V of C){let m=await l?.(P,G,V);m&&(V=m),F.next((0,et.print)({event:"next",data:w?{id:w,payload:V}:V}))}else{let V=await l?.(P,G,C);V&&(C=V),F.next((0,et.print)({event:"next",data:w?{id:w,payload:C}:C}))}F.next((0,et.print)({event:"complete",data:w?{id:w}:null})),await f?.(P,G),w?delete E[w]:await F.iterator.return()})().catch(F.iterator.throw)}}}async function T(I,E){let L,F=await u?.(I,E);if(cr(F))return F;if(Bd(F)||(0,et.isAsyncIterable)(F))return{ctx:typeof o=="function"?await o(I,E):o,perform(){return F}};if(F)L=F;else{if(!i)throw new Error("The GraphQL schema is not provided");let{operationName:G,variables:C}=E,w;try{w=(0,Rn.parse)(E.query)}catch(m){return[JSON.stringify({errors:[m instanceof Error?{message:m.message}:m]}),{status:400,statusText:"Bad Request",headers:{"content-type":"application/json; charset=utf-8"}}]}let V={operationName:G,document:w,variableValues:C,contextValue:typeof o=="function"?await o(I,E):o};L={...V,schema:typeof i=="function"?await i(I,V):i}}let k;try{let G=(0,Rn.getOperationAST)(L.document,L.operationName);if(!G)throw null;k=G.operation}catch{return[JSON.stringify({errors:[{message:"Unable to detect operation AST"}]}),{status:400,statusText:"Bad Request",headers:{"content-type":"application/json; charset=utf-8"}}]}if(k==="mutation"&&I.method==="GET")return[JSON.stringify({errors:[{message:"Cannot perform mutations over GET"}]}),{status:405,statusText:"Method Not Allowed",headers:{allow:"POST","content-type":"application/json; charset=utf-8"}}];let P=t(L.schema,L.document);return P.length?I.headers.get("accept")==="text/event-stream"?{ctx:L.contextValue,perform(){return{errors:P}}}:[JSON.stringify({errors:P}),{status:400,statusText:"Bad Request",headers:{"content-type":"application/json; charset=utf-8"}}]:{ctx:L.contextValue,async perform(){let G=await(k==="subscription"?n(L):r(L)),C=await c?.(L.contextValue,I,L,G);return C||G}}}return async function(E){var L,F,k;let P=await s(E);if(cr(P))return P;let G=E.headers.get("accept")||"*/*",C=typeof P=="string"?d[P]:null;if(G==="text/event-stream"){let $=await a?.(E);if(cr($))return $;if(!C){let X=await Kd(E);if(cr(X))return X;let ye=X,Te=y(null);Te.ops[""]=null;let Ie=await T(E,ye);if(cr(Ie))return Ie;let Le=await Ie.perform();return(0,et.isAsyncIterable)(Le)&&(Te.ops[""]=Le),Te.from(Ie.ctx,E,Le,null),[Te.subscribe(),{status:200,statusText:"OK",headers:{connection:"keep-alive","cache-control":"no-cache","content-encoding":"none","content-type":"text/event-stream; charset=utf-8"}}]}return C.open?[JSON.stringify({errors:[{message:"Stream already open"}]}),{status:409,statusText:"Conflict",headers:{"content-type":"application/json; charset=utf-8"}}]:[C.subscribe(),{status:200,statusText:"OK",headers:{connection:"keep-alive","cache-control":"no-cache","content-encoding":"none","content-type":"text/event-stream; charset=utf-8"}}]}if(typeof P!="string")return[null,{status:404,statusText:"Not Found"}];if(E.method==="PUT")return["*/*","text/plain"].includes(G)?C?[JSON.stringify({errors:[{message:"Stream already registered"}]}),{status:409,statusText:"Conflict",headers:{"content-type":"application/json; charset=utf-8"}}]:(d[P]=y(P),[P,{status:201,statusText:"Created",headers:{"content-type":"text/plain; charset=utf-8"}}]):[null,{status:406,statusText:"Not Acceptable"}];if(E.method==="DELETE"){if(!C)return[JSON.stringify({errors:[{message:"Stream not found"}]}),{status:404,statusText:"Not Found",headers:{"content-type":"application/json; charset=utf-8"}}];let $=new URL((L=E.url)!==null&&L!==void 0?L:"","http://localhost/").searchParams.get("operationId");if(!$)return[JSON.stringify({errors:[{message:"Operation ID is missing"}]}),{status:400,statusText:"Bad Request",headers:{"content-type":"application/json; charset=utf-8"}}];let X=C.ops[$];return(0,et.isAsyncGenerator)(X)&&X.return(void 0),delete C.ops[$],[null,{status:200,statusText:"OK"}]}else{if(E.method!=="GET"&&E.method!=="POST")return[null,{status:405,statusText:"Method Not Allowed",headers:{allow:"GET, POST, PUT, DELETE"}}];if(!C)return[JSON.stringify({errors:[{message:"Stream not found"}]}),{status:404,statusText:"Not Found",headers:{"content-type":"application/json; charset=utf-8"}}]}if(!["*/*","application/*","application/json"].includes(G))return[null,{status:406,statusText:"Not Acceptable"}];let w=await Kd(E);if(cr(w))return w;let V=w,m=String((k=(F=V.extensions)===null||F===void 0?void 0:F.operationId)!==null&&k!==void 0?k:"");if(!m)return[JSON.stringify({errors:[{message:"Operation ID is missing"}]}),{status:400,statusText:"Bad Request",headers:{"content-type":"application/json; charset=utf-8"}}];if(m in C.ops)return[JSON.stringify({errors:[{message:"Operation with ID already exists"}]}),{status:409,statusText:"Conflict",headers:{"content-type":"application/json; charset=utf-8"}}];C.ops[m]=null;let _=await T(E,V);if(cr(_))return _;if(!(m in C.ops))return[null,{status:204,statusText:"No Content"}];let oe=await _.perform();return!(m in C.ops)&&((0,et.isAsyncGenerator)(oe)&&oe.return(void 0),!(m in C.ops))?[null,{status:204,statusText:"No Content"}]:((0,et.isAsyncIterable)(oe)&&(C.ops[m]=oe),C.from(_.ctx,E,oe,m),[null,{status:202,statusText:"Accepted"}])}}Pr.createHandler=V_;async function Kd(e){var t,r,n;let i={};try{switch(!0){case e.method==="GET":{try{let[,s]=e.url.split("?"),a=new URLSearchParams(s);i.operationName=(t=a.get("operationName"))!==null&&t!==void 0?t:void 0,i.query=(r=a.get("query"))!==null&&r!==void 0?r:void 0;let o=a.get("variables");o&&(i.variables=JSON.parse(o));let u=a.get("extensions");u&&(i.extensions=JSON.parse(u))}catch{throw new Error("Unparsable URL")}break}case(e.method==="POST"&&((n=e.headers.get("content-type"))===null||n===void 0?void 0:n.includes("application/json"))):{if(!e.body)throw new Error("Missing body");let s=typeof e.body=="function"?await e.body():e.body,a=typeof s=="string"?JSON.parse(s):s;if(!(0,$d.isObject)(a))throw new Error("JSON body must be an object");i.operationName=a.operationName,i.query=a.query,i.variables=a.variables,i.extensions=a.extensions;break}default:return[null,{status:415,statusText:"Unsupported Media Type"}]}if(i.query==null)throw new Error("Missing query");if(typeof i.query!="string")throw new Error("Invalid query");if(i.variables!=null&&(typeof i.variables!="object"||Array.isArray(i.variables)))throw new Error("Invalid variables");if(i.extensions!=null&&(typeof i.extensions!="object"||Array.isArray(i.extensions)))throw new Error("Invalid extensions");return i}catch(s){return[JSON.stringify({errors:[s instanceof Error?{message:s.message}:s]}),{status:400,statusText:"Bad Request",headers:{"content-type":"application/json; charset=utf-8"}}]}}function cr(e){return Array.isArray(e)}function Bd(e){return(0,$d.isObject)(e)&&("data"in e||"data"in e&&e.data==null&&"errors"in e)}Pr.isExecutionResult=Bd});var Xd=h(Wi=>{"use strict";Object.defineProperty(Wi,"__esModule",{value:!0});Wi.createParser=void 0;var Jd=dr(),lr;(function(e){e[e.NewLine=10]="NewLine",e[e.CchunkiageReturn=13]="CchunkiageReturn",e[e.Space=32]="Space",e[e.Colon=58]="Colon"})(lr||(lr={}));function M_(){let e,t,r,n=!1,i={event:"",data:""},s=[],a=new TextDecoder;return function(u){if(e===void 0)e=u,t=0,r=-1;else{let f=new Uint8Array(e.length+u.length);f.set(e),f.set(u,e.length),e=f}let c=e.length,l=0;for(;t0){let d=e.subarray(l,f),y=a.decode(d.subarray(0,r)),T=r+(d[r+1]===lr.Space?2:1),I=a.decode(d.subarray(T));switch(y){case"event":i.event=I;break;case"data":i.data=i.data?i.data+` +`+I:I;break}}l=t,r=-1}if(l===c){e=void 0;let f=[...s];return s=[],f}else l!==0&&(e=e.subarray(l),t-=l)}}Wi.createParser=M_});var zd=h(lt=>{"use strict";var G_=lt&<.__createBinding||(Object.create?function(e,t,r,n){n===void 0&&(n=r);var i=Object.getOwnPropertyDescriptor(t,r);(!i||("get"in i?!t.__esModule:i.writable||i.configurable))&&(i={enumerable:!0,get:function(){return t[r]}}),Object.defineProperty(e,n,i)}:function(e,t,r,n){n===void 0&&(n=r),e[n]=t[r]}),C_=lt&<.__exportStar||function(e,t){for(var r in e)r!=="default"&&!Object.prototype.hasOwnProperty.call(t,r)&&G_(t,e,r)};Object.defineProperty(lt,"__esModule",{value:!0});lt.NetworkError=lt.createClient=void 0;var Q_=Xd(),U_=Pn(),K_=dr();C_(dr(),lt);function $_(e){let{singleConnection:t=!1,lazy:r=!0,lazyCloseTimeout:n=0,onNonLazyError:i=console.error,generateID:s=function(){return"xxxxxxxx-xxxx-4xxx-yxxx-xxxxxxxxxxxx".replace(/[xy]/g,V=>{let m=Math.random()*16|0;return(V=="x"?m:m&3|8).toString(16)})},retryAttempts:a=5,retry:o=async function(V){let m=1e3;for(let _=0;_setTimeout(_,m+Math.floor(Math.random()*2700+300)))},credentials:u="same-origin",referrer:c,referrerPolicy:l,onMessage:f,on:d}=e,y=e.fetchFn||fetch,T=e.abortControllerImpl||AbortController,I=(()=>{let w=!1,V=[];return{get disposed(){return w},onDispose(m){return w?(setTimeout(()=>m(),0),()=>{}):(V.push(m),()=>{V.splice(V.indexOf(m),1)})},dispose(){if(!w){w=!0;for(let m of[...V])m()}}}})(),E,L,F=0,k=null,P=0;async function G(){try{if(I.disposed)throw new Error("Client has been disposed");return await(L??(L=(async()=>{var w,V,m;if(k){if(await o(P),E.signal.aborted)throw new Error("Connection aborted by the client");P++}(w=d?.connecting)===null||w===void 0||w.call(d,!!k),E=new T;let _=I.onDispose(()=>E.abort());E.signal.addEventListener("abort",()=>{_(),L=void 0});let oe=typeof e.url=="function"?await e.url():e.url;if(E.signal.aborted)throw new Error("Connection aborted by the client");let $=typeof e.headers=="function"?await e.headers():(V=e.headers)!==null&&V!==void 0?V:{};if(E.signal.aborted)throw new Error("Connection aborted by the client");let X;try{X=await y(oe,{signal:E.signal,method:"PUT",credentials:u,referrer:c,referrerPolicy:l,headers:$})}catch(Ie){throw new qe(Ie)}if(X.status!==201)throw new qe(X);let ye=await X.text();$[K_.TOKEN_HEADER_KEY]=ye;let Te=await Hd({signal:E.signal,headers:$,credentials:u,referrer:c,referrerPolicy:l,url:oe,fetchFn:y,onMessage:Ie=>{var Le;(Le=d?.message)===null||Le===void 0||Le.call(d,Ie),f?.(Ie)}});return(m=d?.connected)===null||m===void 0||m.call(d,!!k),Te.waitForThrow().catch(()=>L=void 0),Te})()))}catch(w){throw L=void 0,w}}t&&!r&&(async()=>{for(F++;;)try{let{waitForThrow:w}=await G();await w()}catch(w){if(I.disposed)return;if(!(w instanceof qe)||(L=void 0,!a||P>=a))return i?.(w);k=w}})();function C(w,V,m){if(!t){let $=new T,X=I.onDispose(()=>{X(),$.abort()});return(async()=>{var ye,Te,Ie,Le,xe;let Ae=null,Nt=0;for(;;)try{if(Ae){if(await o(Nt),$.signal.aborted)throw new Error("Connection aborted by the client");Nt++}(ye=d?.connecting)===null||ye===void 0||ye.call(d,!!Ae),(Te=m?.connecting)===null||Te===void 0||Te.call(m,!!Ae);let jt=typeof e.url=="function"?await e.url():e.url;if($.signal.aborted)throw new Error("Connection aborted by the client");let v=typeof e.headers=="function"?await e.headers():(Ie=e.headers)!==null&&Ie!==void 0?Ie:{};if($.signal.aborted)throw new Error("Connection aborted by the client");let{getResults:S}=await Hd({signal:$.signal,headers:{...v,"content-type":"application/json; charset=utf-8"},credentials:u,referrer:c,referrerPolicy:l,url:jt,body:JSON.stringify(w),fetchFn:y,onMessage:N=>{var D,ie;(D=d?.message)===null||D===void 0||D.call(d,N),(ie=m?.message)===null||ie===void 0||ie.call(m,N),f?.(N)}});(Le=d?.connected)===null||Le===void 0||Le.call(d,!!Ae),(xe=m?.connected)===null||xe===void 0||xe.call(m,!!Ae);for await(let N of S())Ae=null,Nt=0,V.next(N);return $.abort()}catch(jt){if($.signal.aborted)return;if(!(jt instanceof qe)||!a||Nt>=a)throw jt;Ae=jt}})().then(()=>V.complete()).catch(ye=>V.error(ye)),()=>$.abort()}F++;let _=new T,oe=I.onDispose(()=>{oe(),_.abort()});return(async()=>{let $=s();w={...w,extensions:{...w.extensions,operationId:$}};let X=null;for(;;){X=null;try{let{url:ye,headers:Te,getResults:Ie}=await G(),Le;try{Le=await y(ye,{signal:_.signal,method:"POST",credentials:u,referrer:c,referrerPolicy:l,headers:{...Te,"content-type":"application/json; charset=utf-8"},body:JSON.stringify(w)})}catch(xe){throw new qe(xe)}if(Le.status!==202)throw new qe(Le);X=async()=>{let xe;try{let Ae=new T,Nt=I.onDispose(()=>{Nt(),Ae.abort()});xe=await y(ye+"?operationId="+$,{signal:Ae.signal,method:"DELETE",credentials:u,referrer:c,referrerPolicy:l,headers:Te})}catch(Ae){throw new qe(Ae)}if(xe.status!==200)throw new qe(xe)};for await(let xe of Ie({signal:_.signal,operationId:$}))k=null,P=0,V.next(xe);return X=null,_.abort()}catch(ye){if(_.signal.aborted)return await X?.();if(!(ye instanceof qe)||(r&&(L=void 0),!a||P>=a))throw _.abort(),ye;k=ye}finally{_.signal.aborted&&--F===0&&(isFinite(n)&&n>0?setTimeout(()=>{F||E.abort()},n):E.abort())}}})().then(()=>V.complete()).catch($=>V.error($)),()=>_.abort()}return{subscribe:C,iterate(w,V){let m=[],_={done:!1,error:null,resolve:()=>{}},oe=C(w,{next(X){m.push(X),_.resolve()},error(X){_.done=!0,_.error=X,_.resolve()},complete(){_.done=!0,_.resolve()}},V),$=async function*(){for(;;){for(m.length||await new Promise(ye=>_.resolve=ye);m.length;)yield m.shift();if(_.error)throw _.error;if(_.done)return}}();return $.throw=async X=>(_.done||(_.done=!0,_.error=X,_.resolve()),{done:!0,value:void 0}),$.return=async()=>(oe(),{done:!0,value:void 0}),$},dispose(){I.dispose()}}}lt.createClient=$_;var qe=class extends Error{constructor(t){let r,n;B_(t)?(n=t,r="Server responded with "+t.status+": "+t.statusText):t instanceof Error?r=t.message:r=String(t),super(r),this.name=this.constructor.name,this.response=n}};lt.NetworkError=qe;function B_(e){return(0,U_.isObject)(e)&&typeof e.ok=="boolean"&&typeof e.status=="number"&&typeof e.statusText=="string"}async function Hd(e){let{signal:t,url:r,credentials:n,headers:i,body:s,referrer:a,referrerPolicy:o,fetchFn:u,onMessage:c}=e,l={},f={},d;try{d=await u(r,{signal:t,method:s?"POST":"GET",credentials:n,referrer:a,referrerPolicy:o,headers:{...i,accept:"text/event-stream"},body:s})}catch(I){throw new qe(I)}if(!d.ok)throw new qe(d);if(!d.body)throw new Error("Missing response body");let y=null,T;return(async()=>{var I;try{let E=(0,Q_.createParser)();for await(let L of Y_(d.body)){if(typeof L=="string")throw y=new Error(`Unexpected string chunk "${L}"`);let F;try{F=E(L)}catch(k){throw y=k}if(F)for(let k of F){try{c?.(k)}catch(G){throw y=G}let P=k.data&&"id"in k.data?k.data.id:"";switch(P in f||(f[P]=[]),k.event){case"next":P?f[P].push(k.data.payload):f[P].push(k.data);break;case"complete":f[P].push("complete");break;default:throw y=new Error(`Unexpected message event "${k.event}"`)}(I=l[P])===null||I===void 0||I.proceed()}}if(Object.keys(l).length)throw new Error("Connection closed while having active streams")}catch(E){!y&&Object.keys(l).length?y=new qe(E):y=E,T?.(y)}finally{Object.values(l).forEach(({proceed:E})=>E())}})(),{url:r,headers:i,waitForThrow:()=>new Promise((I,E)=>{if(y)return E(y);T=E}),async*getResults(I){var E;let{signal:L,operationId:F=""}=I??{};try{for(;;){for(;!((E=f[F])===null||E===void 0)&&E.length;){let k=f[F].shift();if(k==="complete")return;yield k}if(y)throw y;if(L?.aborted)throw new Error("Getting results aborted by the client");await new Promise(k=>{let P=()=>{L?.removeEventListener("abort",P),delete l[F],k()};L?.addEventListener("abort",P),l[F]={proceed:P}})}}finally{delete f[F]}}}}function Y_(e){return typeof Object(e)[Symbol.asyncIterator]=="function"?(e=e,e[Symbol.asyncIterator]()):async function*(){let t=e.getReader(),r;do r=await t.read(),r.value!==void 0&&(yield r.value);while(!r.done)}()}});var Wd=h(Rt=>{"use strict";var J_=Rt&&Rt.__createBinding||(Object.create?function(e,t,r,n){n===void 0&&(n=r);var i=Object.getOwnPropertyDescriptor(t,r);(!i||("get"in i?!t.__esModule:i.writable||i.configurable))&&(i={enumerable:!0,get:function(){return t[r]}}),Object.defineProperty(e,n,i)}:function(e,t,r,n){n===void 0&&(n=r),e[n]=t[r]}),Mu=Rt&&Rt.__exportStar||function(e,t){for(var r in e)r!=="default"&&!Object.prototype.hasOwnProperty.call(t,r)&&J_(t,e,r)};Object.defineProperty(Rt,"__esModule",{value:!0});Mu(dr(),Rt);Mu(Yd(),Rt);Mu(zd(),Rt)});var z_=h(Zi=>{Object.defineProperty(Zi,"__esModule",{value:!0});Zi.createAsyncIterable=void 0;var X_=Wd();async function H_(){let e="http://127.0.0.1:4200/graphql/stream";return console.log("using url",e),(0,X_.createClient)({url:e}).iterate({query:"subscription { greetings }"})}Zi.createAsyncIterable=H_});return z_();})(); +return z.createAsyncIterable(); diff --git a/proto/catalog.proto b/proto/catalog.proto index a3fcd1ab2a63..fd7d77116c1a 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -263,6 +263,10 @@ message Function { // The zstd-compressed binary of the function. optional bytes compressed_binary = 17; bool always_retry_on_network_error = 16; + // The runtime used when javascript is used as the language. Could be "quickjs" or "deno". + optional string runtime = 18; + // The function type, which is used to execute the function. Could be "sync", "async", "generator" or "async_generator" + optional string function_type = 19; oneof kind { ScalarFunction scalar = 11; diff --git a/proto/expr.proto b/proto/expr.proto index 48fe88d1ff49..c8a9887c1663 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -543,6 +543,10 @@ message UserDefinedFunction { // - If `language` is `rust` or `wasm`, the zstd-compressed wasm binary. optional bytes compressed_binary = 10; bool always_retry_on_network_error = 9; + // The runtime used when javascript is used as the language. Could be "quickjs" or "deno". + optional string runtime = 11; + // The function type, which is used to execute the function. Could be "sync", "async", "generator" or "async_generator" + optional string function_type = 12; } // Additional information for user defined table functions. @@ -554,4 +558,6 @@ message UserDefinedTableFunction { optional string identifier = 6; optional string body = 7; optional bytes compressed_binary = 10; + optional string runtime = 11; + optional string function_type = 12; } diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index c2f49c017e36..5ed92b65609c 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -10,6 +10,7 @@ repository = { workspace = true } [features] rw-static-link = ["workspace-config/rw-static-link"] rw-dynamic-link = ["workspace-config/rw-dynamic-link"] +embedded-deno-udf = ["risingwave_expr/embedded-deno-udf"] embedded-python-udf = ["risingwave_expr/embedded-python-udf"] default = ["rw-static-link"] diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 2e9a2ef6c18a..65601d9c3a3e 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -257,7 +257,7 @@ mod test { } async fn new_avro_conf_from_local(file_name: &str) -> ConnectorResult { - let schema_path = "file://".to_owned() + &test_data_path(file_name); + let schema_path = format!("file://{}", test_data_path(file_name)); let info = StreamSourceInfo { row_schema_location: schema_path.clone(), use_schema_registry: false, diff --git a/src/connector/src/with_options_test.rs b/src/connector/src/with_options_test.rs index 83504f787004..155932ce9045 100644 --- a/src/connector/src/with_options_test.rs +++ b/src/connector/src/with_options_test.rs @@ -169,8 +169,10 @@ fn generate_with_options_yaml_inner(path: &Path) -> String { let struct_infos = flatten_nested_options(struct_infos); // Generate the output - "# THIS FILE IS AUTO_GENERATED. DO NOT EDIT\n\n".to_string() - + &serde_yaml::to_string(&struct_infos).unwrap() + format!( + "# THIS FILE IS AUTO_GENERATED. DO NOT EDIT\n\n{}", + serde_yaml::to_string(&struct_infos).unwrap() + ) } #[derive(Debug, Serialize, Clone)] diff --git a/src/expr/core/Cargo.toml b/src/expr/core/Cargo.toml index 30f6419e05e5..3f5ca590026d 100644 --- a/src/expr/core/Cargo.toml +++ b/src/expr/core/Cargo.toml @@ -16,6 +16,7 @@ ignored = ["workspace-hack", "ctor"] normal = ["workspace-hack", "ctor"] [features] +embedded-deno-udf = ["arrow-udf-js-deno"] embedded-python-udf = ["arrow-udf-python"] [dependencies] @@ -23,6 +24,7 @@ anyhow = "1" arrow-array = { workspace = true } arrow-schema = { workspace = true } arrow-udf-js = { workspace = true } +arrow-udf-js-deno = { workspace = true, optional = true } arrow-udf-python = { workspace = true, optional = true } arrow-udf-wasm = { workspace = true } async-trait = "0.1" @@ -39,6 +41,7 @@ easy-ext = "1" educe = "0.5" either = "1" enum-as-inner = "0.6" +futures = "0.3" futures-async-stream = { workspace = true } futures-util = "0.3" itertools = { workspace = true } diff --git a/src/expr/core/src/expr/expr_udf.rs b/src/expr/core/src/expr/expr_udf.rs index 4e63dc5950bb..ed7d597cce52 100644 --- a/src/expr/core/src/expr/expr_udf.rs +++ b/src/expr/core/src/expr/expr_udf.rs @@ -21,6 +21,8 @@ use std::time::Duration; use anyhow::{Context, Error}; use arrow_schema::{Field, Fields, Schema}; use arrow_udf_js::{CallMode as JsCallMode, Runtime as JsRuntime}; +#[cfg(feature = "embedded-deno-udf")] +use arrow_udf_js_deno::{CallMode as DenoCallMode, Runtime as DenoRuntime}; #[cfg(feature = "embedded-python-udf")] use arrow_udf_python::{CallMode as PythonCallMode, Runtime as PythonRuntime}; use arrow_udf_wasm::Runtime as WasmRuntime; @@ -73,6 +75,8 @@ pub enum UdfImpl { JavaScript(JsRuntime), #[cfg(feature = "embedded-python-udf")] Python(PythonRuntime), + #[cfg(feature = "embedded-deno-udf")] + Deno(Arc), } #[async_trait::async_trait] @@ -127,9 +131,11 @@ impl UserDefinedFunction { }; let language = match &self.imp { UdfImpl::Wasm(_) => "wasm", - UdfImpl::JavaScript(_) => "javascript", + UdfImpl::JavaScript(_) => "javascript(quickjs)", #[cfg(feature = "embedded-python-udf")] UdfImpl::Python(_) => "python", + #[cfg(feature = "embedded-deno-udf")] + UdfImpl::Deno(_) => "javascript(deno)", UdfImpl::External(_) => "external", }; let labels: &[&str; 4] = &[addr, language, &self.identifier, fragment_id.as_str()]; @@ -152,6 +158,11 @@ impl UserDefinedFunction { UdfImpl::JavaScript(runtime) => runtime.call(&self.identifier, &arrow_input), #[cfg(feature = "embedded-python-udf")] UdfImpl::Python(runtime) => runtime.call(&self.identifier, &arrow_input), + #[cfg(feature = "embedded-deno-udf")] + UdfImpl::Deno(runtime) => tokio::task::block_in_place(|| { + tokio::runtime::Handle::current() + .block_on(runtime.call(&self.identifier, arrow_input)) + }), UdfImpl::External(client) => { let disable_retry_count = self.disable_retry_count.load(Ordering::Relaxed); let result = if self.always_retry_on_network_error { @@ -240,6 +251,15 @@ impl Build for UserDefinedFunction { let return_type = DataType::from(prost.get_return_type().unwrap()); let udf = prost.get_rex_node().unwrap().as_udf().unwrap(); + #[cfg(not(feature = "embedded-deno-udf"))] + let runtime = "quickjs"; + + #[cfg(feature = "embedded-deno-udf")] + let runtime = match udf.runtime.as_deref() { + Some("deno") => "deno", + _ => "quickjs", + }; + let identifier = udf.get_identifier()?; let imp = match udf.language.as_str() { #[cfg(not(madsim))] @@ -250,7 +270,7 @@ impl Build for UserDefinedFunction { let runtime = get_or_create_wasm_runtime(&wasm_binary)?; UdfImpl::Wasm(runtime) } - "javascript" => { + "javascript" if runtime != "deno" => { let mut rt = JsRuntime::new()?; let body = format!( "export function {}({}) {{ {} }}", @@ -266,6 +286,48 @@ impl Build for UserDefinedFunction { )?; UdfImpl::JavaScript(rt) } + #[cfg(feature = "embedded-deno-udf")] + "javascript" if runtime == "deno" => { + let rt = DenoRuntime::new(); + let body = match udf.get_body() { + Ok(body) => body.clone(), + Err(_) => match udf.get_compressed_binary() { + Ok(compressed_binary) => { + let binary = zstd::stream::decode_all(compressed_binary.as_slice()) + .context("failed to decompress binary")?; + String::from_utf8(binary).context("failed to decode binary")? + } + Err(_) => { + bail!("UDF body or compressed binary is required for deno UDF"); + } + }, + }; + + let body = if matches!(udf.function_type.as_deref(), Some("async")) { + format!( + "export async function {}({}) {{ {} }}", + identifier, + udf.arg_names.join(","), + body + ) + } else { + format!( + "export function {}({}) {{ {} }}", + identifier, + udf.arg_names.join(","), + body + ) + }; + + futures::executor::block_on(rt.add_function( + identifier, + arrow_schema::DataType::try_from(&return_type)?, + DenoCallMode::CalledOnNullInput, + &body, + ))?; + + UdfImpl::Deno(rt) + } #[cfg(feature = "embedded-python-udf")] "python" if udf.body.is_some() => { let mut rt = PythonRuntime::builder().sandboxed(true).build()?; diff --git a/src/expr/core/src/table_function/user_defined.rs b/src/expr/core/src/table_function/user_defined.rs index 0d1d726c394c..b65ee5e77758 100644 --- a/src/expr/core/src/table_function/user_defined.rs +++ b/src/expr/core/src/table_function/user_defined.rs @@ -18,6 +18,8 @@ use anyhow::Context; use arrow_array::RecordBatch; use arrow_schema::{Field, Fields, Schema, SchemaRef}; use arrow_udf_js::{CallMode as JsCallMode, Runtime as JsRuntime}; +#[cfg(feature = "embedded-deno-udf")] +use arrow_udf_js_deno::{CallMode as DenoCallMode, Runtime as DenoRuntime}; #[cfg(feature = "embedded-python-udf")] use arrow_udf_python::{CallMode as PythonCallMode, Runtime as PythonRuntime}; use cfg_or_panic::cfg_or_panic; @@ -78,6 +80,13 @@ impl UdfImpl { yield res?; } } + #[cfg(feature = "embedded-deno-udf")] + UdfImpl::Deno(runtime) => { + let mut iter = runtime.call_table_function(identifier, input, 1024).await?; + while let Some(res) = iter.next().await { + yield res?; + } + } UdfImpl::Wasm(runtime) => { for res in runtime.call_table_function(identifier, &input)? { yield res?; @@ -188,6 +197,15 @@ pub fn new_user_defined(prost: &PbTableFunction, chunk_size: usize) -> Result "deno", + _ => "quickjs", + }; + let client = match udtf.language.as_str() { "wasm" | "rust" => { let compressed_wasm_binary = udtf.get_compressed_binary()?; @@ -196,7 +214,7 @@ pub fn new_user_defined(prost: &PbTableFunction, chunk_size: usize) -> Result { + "javascript" if runtime != "deno" => { let mut rt = JsRuntime::new()?; let body = format!( "export function* {}({}) {{ {} }}", @@ -212,6 +230,44 @@ pub fn new_user_defined(prost: &PbTableFunction, chunk_size: usize) -> Result { + let rt = DenoRuntime::new(); + let body = match udtf.get_body() { + Ok(body) => body.clone(), + Err(_) => match udtf.get_compressed_binary() { + Ok(compressed_binary) => { + let binary = zstd::stream::decode_all(compressed_binary.as_slice()) + .context("failed to decompress binary")?; + String::from_utf8(binary).context("failed to decode binary")? + } + Err(_) => { + bail!("UDF body or compressed binary is required for deno UDF"); + } + }, + }; + + let body = format!( + "export {} {}({}) {{ {} }}", + match udtf.function_type.as_deref() { + Some("async") => "async function", + Some("async_generator") => "async function*", + Some("sync") => "function", + _ => "function*", + }, + identifier, + udtf.arg_names.join(","), + body + ); + + futures::executor::block_on(rt.add_function( + identifier, + arrow_schema::DataType::try_from(&return_type)?, + DenoCallMode::CalledOnNullInput, + &body, + ))?; + UdfImpl::Deno(rt) + } #[cfg(feature = "embedded-python-udf")] "python" if udtf.body.is_some() => { let mut rt = PythonRuntime::builder().sandboxed(true).build()?; diff --git a/src/frontend/src/catalog/function_catalog.rs b/src/frontend/src/catalog/function_catalog.rs index 142bc222a59b..1a0e01db1b92 100644 --- a/src/frontend/src/catalog/function_catalog.rs +++ b/src/frontend/src/catalog/function_catalog.rs @@ -35,6 +35,8 @@ pub struct FunctionCatalog { pub link: Option, pub compressed_binary: Option>, pub always_retry_on_network_error: bool, + pub function_type: Option, + pub runtime: Option, } #[derive(Clone, Display, PartialEq, Eq, Hash, Debug)] @@ -72,6 +74,8 @@ impl From<&PbFunction> for FunctionCatalog { link: prost.link.clone(), compressed_binary: prost.compressed_binary.clone(), always_retry_on_network_error: prost.always_retry_on_network_error, + function_type: prost.function_type.clone(), + runtime: prost.runtime.clone(), } } } diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index c72c207c5378..4703112bc16d 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -80,6 +80,8 @@ impl TableFunction { identifier: c.identifier.clone(), body: c.body.clone(), compressed_binary: c.compressed_binary.clone(), + function_type: c.function_type.clone(), + runtime: c.runtime.clone(), }), } } diff --git a/src/frontend/src/expr/user_defined_function.rs b/src/frontend/src/expr/user_defined_function.rs index a9f7f148e8df..44abfa1859c4 100644 --- a/src/frontend/src/expr/user_defined_function.rs +++ b/src/frontend/src/expr/user_defined_function.rs @@ -60,6 +60,8 @@ impl UserDefinedFunction { link: udf.link.clone(), compressed_binary: udf.compressed_binary.clone(), always_retry_on_network_error: udf.always_retry_on_network_error, + function_type: udf.function_type.clone(), + runtime: udf.runtime.clone(), }; Ok(Self { @@ -96,6 +98,8 @@ impl Expr for UserDefinedFunction { body: self.catalog.body.clone(), compressed_binary: self.catalog.compressed_binary.clone(), always_retry_on_network_error: self.catalog.always_retry_on_network_error, + function_type: self.catalog.function_type.clone(), + runtime: self.catalog.runtime.clone(), })), } } diff --git a/src/frontend/src/handler/create_function.rs b/src/frontend/src/handler/create_function.rs index cb2a88c64cf3..428a5612e177 100644 --- a/src/frontend/src/handler/create_function.rs +++ b/src/frontend/src/handler/create_function.rs @@ -64,6 +64,21 @@ pub async fn handle_create_function( // correct protocol. None => "".to_string(), }; + + let rt = match params.runtime { + Some(runtime) => { + if language.as_str() == "javascript" { + runtime.to_string() + } else { + return Err(ErrorCode::InvalidParameterValue( + "runtime is only supported for javascript".to_string(), + ) + .into()); + } + } + None => "".to_string(), + }; + let return_type; let kind = match returns { Some(CreateFunctionReturns::Value(data_type)) => { @@ -122,6 +137,8 @@ pub async fn handle_create_function( let mut link = None; let mut body = None; let mut compressed_binary = None; + let mut function_type = None; + let mut runtime = None; match language.as_str() { "python" if params.using.is_none() => { @@ -177,7 +194,7 @@ pub async fn handle_create_function( } link = Some(l); } - "javascript" => { + "javascript" if rt.as_str() != "deno" => { identifier = function_name.to_string(); body = Some( params @@ -185,6 +202,46 @@ pub async fn handle_create_function( .ok_or_else(|| ErrorCode::InvalidParameterValue("AS must be specified".into()))? .into_string(), ); + runtime = Some("quickjs".to_string()); + } + "javascript" if rt.as_str() == "deno" => { + identifier = function_name.to_string(); + match (params.using, params.as_) { + (None, None) => { + return Err(ErrorCode::InvalidParameterValue( + "Either USING or AS must be specified".into(), + ) + .into()) + } + (None, Some(_as)) => body = Some(_as.into_string()), + (Some(CreateFunctionUsing::Link(link)), None) => { + let bytes = download_code_from_link(&link).await?; + compressed_binary = Some(zstd::stream::encode_all(bytes.as_slice(), 0)?); + } + (Some(CreateFunctionUsing::Base64(encoded)), None) => { + use base64::prelude::{Engine, BASE64_STANDARD}; + let bytes = BASE64_STANDARD + .decode(encoded) + .context("invalid base64 encoding")?; + compressed_binary = Some(zstd::stream::encode_all(bytes.as_slice(), 0)?); + } + (Some(_), Some(_)) => { + return Err(ErrorCode::InvalidParameterValue( + "Both USING and AS cannot be specified".into(), + ) + .into()) + } + }; + + function_type = match params.function_type { + Some(CreateFunctionType::Sync) => Some("sync".to_string()), + Some(CreateFunctionType::Async) => Some("async".to_string()), + Some(CreateFunctionType::Generator) => Some("generator".to_string()), + Some(CreateFunctionType::AsyncGenerator) => Some("async_generator".to_string()), + None => None, + }; + + runtime = Some("deno".to_string()); } "rust" => { if params.using.is_some() { @@ -282,6 +339,8 @@ pub async fn handle_create_function( always_retry_on_network_error: with_options .always_retry_on_network_error .unwrap_or_default(), + runtime, + function_type, }; let catalog_writer = session.catalog_writer()?; @@ -364,6 +423,19 @@ fn find_wasm_identifier_v2( Ok(identifier.into()) } +/// Download wasm binary from a link. +#[allow(clippy::unused_async)] +async fn download_code_from_link(link: &str) -> Result> { + // currently only local file system is supported + if let Some(path) = link.strip_prefix("fs://") { + let content = + std::fs::read(path).context("failed to read the code from local file system")?; + Ok(content) + } else { + Err(ErrorCode::InvalidParameterValue("only 'fs://' is supported".to_string()).into()) + } +} + /// Generate a function identifier in v0.1 format from the function signature. fn wasm_identifier_v1( name: &str, diff --git a/src/frontend/src/handler/create_sql_function.rs b/src/frontend/src/handler/create_sql_function.rs index f4c683b4d7a0..ae7b2730e25d 100644 --- a/src/frontend/src/handler/create_sql_function.rs +++ b/src/frontend/src/handler/create_sql_function.rs @@ -344,6 +344,8 @@ pub async fn handle_create_sql_function( link: None, owner: session.user_id(), always_retry_on_network_error: false, + runtime: None, + function_type: None, }; let catalog_writer = session.catalog_writer()?; diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index 724355358b02..7feaf8878869 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -8,6 +8,7 @@ mod m20240304_074901_subscription; mod m20240410_082733_with_version_column_migration; mod m20240410_154406_session_params; mod m20240417_062305_subscription_internal_table_name; +mod m20240418_142249_function_runtime; pub struct Migrator; @@ -21,6 +22,7 @@ impl MigratorTrait for Migrator { Box::new(m20240410_082733_with_version_column_migration::Migration), Box::new(m20240410_154406_session_params::Migration), Box::new(m20240417_062305_subscription_internal_table_name::Migration), + Box::new(m20240418_142249_function_runtime::Migration), ] } } diff --git a/src/meta/model_v2/migration/src/m20240418_142249_function_runtime.rs b/src/meta/model_v2/migration/src/m20240418_142249_function_runtime.rs new file mode 100644 index 000000000000..80dcfae266c8 --- /dev/null +++ b/src/meta/model_v2/migration/src/m20240418_142249_function_runtime.rs @@ -0,0 +1,61 @@ +use sea_orm_migration::prelude::{Table as MigrationTable, *}; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + assert!(manager.has_table(Function::Table.to_string()).await?); + + manager + .alter_table( + MigrationTable::alter() + .table(Function::Table) + .add_column(ColumnDef::new(Function::Runtime).string()) + .to_owned(), + ) + .await?; + + manager + .alter_table( + MigrationTable::alter() + .table(Function::Table) + .add_column(ColumnDef::new(Function::FunctionType).string()) + .to_owned(), + ) + .await?; + + Ok(()) + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + // drop tables cascade. + + manager + .alter_table( + MigrationTable::alter() + .table(Function::Table) + .drop_column(Alias::new(Function::Runtime.to_string())) + .to_owned(), + ) + .await?; + + manager + .alter_table( + MigrationTable::alter() + .table(Function::Table) + .drop_column(Alias::new(Function::FunctionType.to_string())) + .to_owned(), + ) + .await?; + Ok(()) + } +} + +#[derive(DeriveIden)] +enum Function { + Table, + Runtime, + FunctionType, +} diff --git a/src/meta/model_v2/src/function.rs b/src/meta/model_v2/src/function.rs index e0d79f3bc557..520df5cd2bf0 100644 --- a/src/meta/model_v2/src/function.rs +++ b/src/meta/model_v2/src/function.rs @@ -47,6 +47,8 @@ pub struct Model { pub compressed_binary: Option>, pub kind: FunctionKind, pub always_retry_on_network_error: bool, + pub runtime: Option, + pub function_type: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -104,6 +106,8 @@ impl From for ActiveModel { compressed_binary: Set(function.compressed_binary), kind: Set(function.kind.unwrap().into()), always_retry_on_network_error: Set(function.always_retry_on_network_error), + runtime: Set(function.runtime), + function_type: Set(function.function_type), } } } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index ff9845988839..a271ab85bb92 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -324,6 +324,8 @@ impl From> for PbFunction { compressed_binary: value.0.compressed_binary, kind: Some(value.0.kind.into()), always_retry_on_network_error: value.0.always_retry_on_network_error, + runtime: value.0.runtime, + function_type: value.0.function_type, } } } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 79c12c1b6cc6..5b587981ac67 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -2884,6 +2884,9 @@ impl fmt::Display for TableColumnDef { pub struct CreateFunctionBody { /// LANGUAGE lang_name pub language: Option, + + pub runtime: Option, + /// IMMUTABLE | STABLE | VOLATILE pub behavior: Option, /// AS 'definition' @@ -2894,6 +2897,8 @@ pub struct CreateFunctionBody { pub return_: Option, /// USING ... pub using: Option, + + pub function_type: Option, } impl fmt::Display for CreateFunctionBody { @@ -2901,6 +2906,11 @@ impl fmt::Display for CreateFunctionBody { if let Some(language) = &self.language { write!(f, " LANGUAGE {language}")?; } + + if let Some(runtime) = &self.runtime { + write!(f, " RUNTIME {runtime}")?; + } + if let Some(behavior) = &self.behavior { write!(f, " {behavior}")?; } @@ -2913,6 +2923,9 @@ impl fmt::Display for CreateFunctionBody { if let Some(using) = &self.using { write!(f, " {using}")?; } + if let Some(function_type) = &self.function_type { + write!(f, " {function_type}")?; + } Ok(()) } } @@ -2987,6 +3000,42 @@ impl fmt::Display for CreateFunctionUsing { } } +#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub enum FunctionRuntime { + QuickJs, + Deno, +} + +impl fmt::Display for FunctionRuntime { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + FunctionRuntime::QuickJs => write!(f, "quickjs"), + FunctionRuntime::Deno => write!(f, "deno"), + } + } +} + +#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub enum CreateFunctionType { + Sync, + Async, + Generator, + AsyncGenerator, +} + +impl fmt::Display for CreateFunctionType { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + CreateFunctionType::Sync => write!(f, "SYNC"), + CreateFunctionType::Async => write!(f, "ASYNC"), + CreateFunctionType::Generator => write!(f, "SYNC GENERATOR"), + CreateFunctionType::AsyncGenerator => write!(f, "ASYNC GENERATOR"), + } + } +} + #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub enum SetVariableValue { @@ -3206,6 +3255,8 @@ mod tests { as_: Some(FunctionDefinition::SingleQuotedDef("SELECT 1".to_string())), return_: None, using: None, + runtime: None, + function_type: None, }, with_options: CreateFunctionWithOptions { always_retry_on_network_error: None, @@ -3227,6 +3278,8 @@ mod tests { as_: Some(FunctionDefinition::SingleQuotedDef("SELECT 1".to_string())), return_: None, using: None, + runtime: None, + function_type: None, }, with_options: CreateFunctionWithOptions { always_retry_on_network_error: Some(true), @@ -3236,5 +3289,29 @@ mod tests { "CREATE FUNCTION foo(INT) RETURNS INT LANGUAGE python IMMUTABLE AS 'SELECT 1' WITH ( ALWAYS_RETRY_NETWORK_ERRORS = true )", format!("{}", create_function) ); + + let create_function = Statement::CreateFunction { + temporary: false, + or_replace: false, + name: ObjectName(vec![Ident::new_unchecked("foo")]), + args: Some(vec![OperateFunctionArg::unnamed(DataType::Int)]), + returns: Some(CreateFunctionReturns::Value(DataType::Int)), + params: CreateFunctionBody { + language: Some(Ident::new_unchecked("javascript")), + behavior: None, + as_: Some(FunctionDefinition::SingleQuotedDef("SELECT 1".to_string())), + return_: None, + using: None, + runtime: Some(FunctionRuntime::Deno), + function_type: Some(CreateFunctionType::AsyncGenerator), + }, + with_options: CreateFunctionWithOptions { + always_retry_on_network_error: None, + }, + }; + assert_eq!( + "CREATE FUNCTION foo(INT) RETURNS INT LANGUAGE javascript RUNTIME deno AS 'SELECT 1' ASYNC GENERATOR", + format!("{}", create_function) + ); } } diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 5d282427d453..630162964bce 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -89,6 +89,7 @@ define_keywords!( ASC, ASENSITIVE, ASYMMETRIC, + ASYNC, AT, ATOMIC, AUTHORIZATION, @@ -246,6 +247,7 @@ define_keywords!( FUNCTION, FUNCTIONS, FUSION, + GENERATOR, GET, GLOBAL, GRANT, @@ -438,6 +440,7 @@ define_keywords!( ROWID, ROWS, ROW_NUMBER, + RUNTIME, SAVEPOINT, SCALAR, SCHEMA, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 92009ecb016e..b4459a1f7c76 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -2440,6 +2440,9 @@ impl Parser { } else if self.parse_keyword(Keyword::LANGUAGE) { ensure_not_set(&body.language, "LANGUAGE")?; body.language = Some(self.parse_identifier()?); + } else if self.parse_keyword(Keyword::RUNTIME) { + ensure_not_set(&body.runtime, "RUNTIME")?; + body.runtime = Some(self.parse_function_runtime()?); } else if self.parse_keyword(Keyword::IMMUTABLE) { ensure_not_set(&body.behavior, "IMMUTABLE | STABLE | VOLATILE")?; body.behavior = Some(FunctionBehavior::Immutable); @@ -2455,6 +2458,15 @@ impl Parser { } else if self.parse_keyword(Keyword::USING) { ensure_not_set(&body.using, "USING")?; body.using = Some(self.parse_create_function_using()?); + } else if self.parse_keyword(Keyword::SYNC) { + ensure_not_set(&body.function_type, "SYNC | ASYNC")?; + body.function_type = Some(self.parse_function_type(false, false)?); + } else if self.parse_keyword(Keyword::ASYNC) { + ensure_not_set(&body.function_type, "SYNC | ASYNC")?; + body.function_type = Some(self.parse_function_type(true, false)?); + } else if self.parse_keyword(Keyword::GENERATOR) { + ensure_not_set(&body.function_type, "SYNC | ASYNC")?; + body.function_type = Some(self.parse_function_type(false, true)?); } else { return Ok(body); } @@ -2477,6 +2489,36 @@ impl Parser { } } + fn parse_function_runtime(&mut self) -> Result { + let ident = self.parse_identifier()?; + match ident.value.to_lowercase().as_str() { + "deno" => Ok(FunctionRuntime::Deno), + "quickjs" => Ok(FunctionRuntime::QuickJs), + r => Err(ParserError::ParserError(format!( + "Unsupported runtime: {r}" + ))), + } + } + + fn parse_function_type( + &mut self, + is_async: bool, + is_generator: bool, + ) -> Result { + let is_generator = if is_generator { + true + } else { + self.parse_keyword(Keyword::GENERATOR) + }; + + match (is_async, is_generator) { + (false, false) => Ok(CreateFunctionType::Sync), + (true, false) => Ok(CreateFunctionType::Async), + (false, true) => Ok(CreateFunctionType::Generator), + (true, true) => Ok(CreateFunctionType::AsyncGenerator), + } + } + // CREATE USER name [ [ WITH ] option [ ... ] ] // where option can be: // SUPERUSER | NOSUPERUSER diff --git a/src/sqlparser/tests/sqlparser_postgres.rs b/src/sqlparser/tests/sqlparser_postgres.rs index 9ef7a7f085c4..d70625c33d77 100644 --- a/src/sqlparser/tests/sqlparser_postgres.rs +++ b/src/sqlparser/tests/sqlparser_postgres.rs @@ -872,6 +872,31 @@ fn parse_create_function() { with_options: Default::default(), } ); + + let sql = "CREATE FUNCTION add(INT, INT) RETURNS INT LANGUAGE SQL IMMUTABLE AS 'select $1 + $2;' ASYNC"; + assert_eq!( + verified_stmt(sql), + Statement::CreateFunction { + or_replace: false, + temporary: false, + name: ObjectName(vec![Ident::new_unchecked("add")]), + args: Some(vec![ + OperateFunctionArg::unnamed(DataType::Int), + OperateFunctionArg::unnamed(DataType::Int), + ]), + returns: Some(CreateFunctionReturns::Value(DataType::Int)), + params: CreateFunctionBody { + language: Some("SQL".into()), + behavior: Some(FunctionBehavior::Immutable), + as_: Some(FunctionDefinition::SingleQuotedDef( + "select $1 + $2;".into() + )), + function_type: Some(CreateFunctionType::Async), + ..Default::default() + }, + with_options: Default::default(), + } + ); } #[test]