From 12dfa28d649a5172b0fd8a941554adced5dece85 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 23 Jan 2025 12:15:54 +0800 Subject: [PATCH 01/25] feat: make `privatelink.endpoint` accept json array (#20154) Signed-off-by: tabversion Co-authored-by: tabversion --- .../src/source/kafka/private_link.rs | 168 +++++++++++++++++- 1 file changed, 164 insertions(+), 4 deletions(-) diff --git a/src/connector/src/source/kafka/private_link.rs b/src/connector/src/source/kafka/private_link.rs index 06d4b8e97155e..73c15d351ac33 100644 --- a/src/connector/src/source/kafka/private_link.rs +++ b/src/connector/src/source/kafka/private_link.rs @@ -22,6 +22,7 @@ use risingwave_common::bail; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::connection::PrivateLinkService; +use serde_derive::Deserialize; use crate::connector_common::{ AwsPrivateLinkItem, PRIVATE_LINK_BROKER_REWRITE_MAP_KEY, PRIVATE_LINK_TARGETS_KEY, @@ -38,6 +39,11 @@ pub(super) enum PrivateLinkContextRole { Producer, } +#[derive(Debug, Deserialize)] +struct PrivateLinkEndpointItem { + host: String, +} + impl std::fmt::Display for PrivateLinkContextRole { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { @@ -135,10 +141,18 @@ pub fn insert_privatelink_broker_rewrite_map( } if let Some(endpoint) = privatelink_endpoint { - for (link, broker) in link_targets.iter().zip_eq_fast(broker_addrs.into_iter()) { - // rewrite the broker address to endpoint:port - broker_rewrite_map.insert(broker.to_owned(), format!("{}:{}", &endpoint, link.port)); - } + // new syntax: endpoint can either be a string or a json array of strings + // if it is a string, rewrite all broker addresses to the same endpoint + // eg. privatelink.endpoint='some_url' ==> broker1:9092 -> some_url:9092, broker2:9093 -> some_url:9093 + // if it is a json array, rewrite each broker address to the corresponding endpoint + // eg. privatelink.endpoint = '[{"host": "aaaa"}, {"host": "bbbb"}, {"host": "cccc"}]' + // ==> broker1:9092 -> aaaa:9092, broker2:9093 -> bbbb:9093, broker3:9094 -> cccc:9094 + handle_privatelink_endpoint( + &endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + )?; } else { if svc.is_none() { bail!("Privatelink endpoint not found."); @@ -166,3 +180,149 @@ pub fn insert_privatelink_broker_rewrite_map( with_options.insert(PRIVATE_LINK_BROKER_REWRITE_MAP_KEY.to_owned(), json); Ok(()) } + +fn handle_privatelink_endpoint( + endpoint: &str, + broker_rewrite_map: &mut HashMap, + link_targets: &[AwsPrivateLinkItem], + broker_addrs: &[&str], +) -> ConnectorResult<()> { + let endpoint = if let Ok(json) = serde_json::from_str::(endpoint) { + json + } else { + serde_json::Value::String(endpoint.to_owned()) + }; + if matches!(endpoint, serde_json::Value::String(_)) { + let endpoint = endpoint.as_str().unwrap(); + for (link, broker) in link_targets.iter().zip_eq_fast(broker_addrs.iter()) { + // rewrite the broker address to endpoint:port + broker_rewrite_map.insert(broker.to_string(), format!("{}:{}", endpoint, link.port)); + } + } else if matches!(endpoint, serde_json::Value::Array(_)) { + let endpoint_list: Vec = endpoint + .as_array() + .unwrap() + .iter() + .map(|v| { + serde_json::from_value(v.clone()).map_err(|_| { + anyhow!( + "expect json schema {{\"host\": \"endpoint url\"}} but got {}", + v + ) + }) + }) + .collect::, _>>()?; + for ((link, broker), endpoint) in link_targets + .iter() + .zip_eq_fast(broker_addrs.iter()) + .zip_eq_fast(endpoint_list.iter()) + { + // rewrite the broker address to endpoint:port + broker_rewrite_map.insert( + broker.to_string(), + format!("{}:{}", endpoint.host, link.port), + ); + } + } else { + bail!( + "expect a string or a json array for privatelink.endpoint, but got {:?}", + endpoint + ) + } + + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_handle_privatelink_endpoint() { + let endpoint = "some_url"; // raw string + let link_targets = vec![ + AwsPrivateLinkItem { + az_id: None, + port: 9092, + }, + AwsPrivateLinkItem { + az_id: None, + port: 9093, + }, + ]; + let broker_addrs = vec!["broker1:9092", "broker2:9093"]; + let mut broker_rewrite_map = HashMap::new(); + handle_privatelink_endpoint( + endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + ) + .unwrap(); + + assert_eq!(broker_rewrite_map.len(), 2); + assert_eq!(broker_rewrite_map["broker1:9092"], "some_url:9092"); + assert_eq!(broker_rewrite_map["broker2:9093"], "some_url:9093"); + + // example 2: json array + let endpoint = r#"[{"host": "aaaa"}, {"host": "bbbb"}, {"host": "cccc"}]"#; + let broker_addrs = vec!["broker1:9092", "broker2:9093", "broker3:9094"]; + let link_targets = vec![ + AwsPrivateLinkItem { + az_id: None, + port: 9092, + }, + AwsPrivateLinkItem { + az_id: None, + port: 9093, + }, + AwsPrivateLinkItem { + az_id: None, + port: 9094, + }, + ]; + let mut broker_rewrite_map = HashMap::new(); + handle_privatelink_endpoint( + endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + ) + .unwrap(); + + assert_eq!(broker_rewrite_map.len(), 3); + assert_eq!(broker_rewrite_map["broker1:9092"], "aaaa:9092"); + assert_eq!(broker_rewrite_map["broker2:9093"], "bbbb:9093"); + assert_eq!(broker_rewrite_map["broker3:9094"], "cccc:9094"); + + // no `host` in the json array + let endpoint = r#"[{"somekey_1": "aaaa"}, {"somekey_2": "bbbb"}, {"somekey_3": "cccc"}]"#; + let mut broker_rewrite_map = HashMap::new(); + let err = handle_privatelink_endpoint( + endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + ) + .unwrap_err(); + assert_eq!( + err.to_string(), + "expect json schema {\"host\": \"endpoint url\"} but got {\"somekey_1\":\"aaaa\"}" + ); + + // illegal json + let endpoint = r#"{}"#; + let mut broker_rewrite_map = HashMap::new(); + let err = handle_privatelink_endpoint( + endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + ) + .unwrap_err(); + assert_eq!( + err.to_string(), + "expect a string or a json array for privatelink.endpoint, but got Object {}" + ); + } +} From 07eb58207a2aca5023bc030507faeb280637e1de Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 23 Jan 2025 04:49:02 +0000 Subject: [PATCH 02/25] chore(deps): Bump fs-err from 3.0.0 to 3.1.0 (#20265) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Bugen Zhao --- Cargo.lock | 10 +++++----- src/prost/Cargo.toml | 2 +- src/risedevtool/Cargo.toml | 2 +- src/risedevtool/config/Cargo.toml | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 73178ca6ba387..387706a3fbcb6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1904,7 +1904,7 @@ dependencies = [ "bitflags 2.8.0", "cexpr", "clang-sys", - "itertools 0.10.5", + "itertools 0.12.1", "lazy_static", "lazycell", "log", @@ -1927,7 +1927,7 @@ dependencies = [ "bitflags 2.8.0", "cexpr", "clang-sys", - "itertools 0.10.5", + "itertools 0.13.0", "proc-macro2", "quote", "regex", @@ -5090,9 +5090,9 @@ dependencies = [ [[package]] name = "fs-err" -version = "3.0.0" +version = "3.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bb60e7409f34ef959985bc9d9c5ee8f5db24ee46ed9775850548021710f807f" +checksum = "1f89bda4c2a21204059a977ed3bfe746677dfd137b83c339e702b0ac91d482aa" dependencies = [ "autocfg", ] @@ -9466,7 +9466,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" dependencies = [ "anyhow", - "itertools 0.10.5", + "itertools 0.12.1", "proc-macro2", "quote", "syn 2.0.87", diff --git a/src/prost/Cargo.toml b/src/prost/Cargo.toml index c9fcb69dafd9b..f279098962542 100644 --- a/src/prost/Cargo.toml +++ b/src/prost/Cargo.toml @@ -25,7 +25,7 @@ workspace-hack = { path = "../workspace-hack" } static_assertions = "1" [build-dependencies] -fs-err = "3.0" +fs-err = "3.1" pbjson-build = "0.7" prost-build = { workspace = true } tonic-build = { workspace = true } diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index 95cbdca6cb01d..7205e6f7ac1b6 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -15,7 +15,7 @@ chrono = { version = "0.4", default-features = false, features = [ ] } clap = { workspace = true } console = "0.15" -fs-err = "3.0.0" +fs-err = "3.1.0" glob = "0.3" google-cloud-pubsub = "0.30" indicatif = "0.17" diff --git a/src/risedevtool/config/Cargo.toml b/src/risedevtool/config/Cargo.toml index 331b4d788276d..88e64c837504f 100644 --- a/src/risedevtool/config/Cargo.toml +++ b/src/risedevtool/config/Cargo.toml @@ -13,7 +13,7 @@ clap = { workspace = true } console = "0.15" dialoguer = "0.11" enum-iterator = "2" -fs-err = "3.0.0" +fs-err = "3.1.0" itertools = { workspace = true } [lints] From a2cf60d1d5876b6517617f7a21be90c206a79f85 Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 23 Jan 2025 17:01:08 +0800 Subject: [PATCH 03/25] fix(batch): fix batch anti join (#20264) --- .../executors/src/executor/join/hash_join.rs | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/batch/executors/src/executor/join/hash_join.rs b/src/batch/executors/src/executor/join/hash_join.rs index 9c31e945723f5..af89b3d1503a9 100644 --- a/src/batch/executors/src/executor/join/hash_join.rs +++ b/src/batch/executors/src/executor/join/hash_join.rs @@ -1106,7 +1106,6 @@ impl HashJoinExecutor { .enumerate() .filter_by_bitmap(probe_chunk.visibility()) { - non_equi_state.found_matched = false; if let Some(first_matched_build_row_id) = hash_map.get(probe_key) { non_equi_state .first_output_row_id @@ -2030,13 +2029,17 @@ impl DataChunkMutator { break; } } - if ANTI_JOIN && !has_more_output_rows && !*found_matched { - new_visibility.set(start_row_id, true); + if !has_more_output_rows && ANTI_JOIN { + if !*found_matched { + new_visibility.set(start_row_id, true); + } + *found_matched = false; } first_output_row_ids.clear(); - self.0.set_visibility(new_visibility.finish()); + self.0 + .set_visibility(new_visibility.finish() & self.0.visibility()); self } @@ -2058,7 +2061,8 @@ impl DataChunkMutator { build_row_ids.clear(); - self.0.set_visibility(new_visibility.finish()); + self.0 + .set_visibility(new_visibility.finish() & self.0.visibility()); self } @@ -2135,7 +2139,8 @@ impl DataChunkMutator { build_row_ids.clear(); - self.0.set_visibility(new_visibility.finish()); + self.0 + .set_visibility(new_visibility.finish() & self.0.visibility()); self } @@ -3458,7 +3463,6 @@ mod tests { &expect )); assert_eq!(state.first_output_row_id, Vec::::new()); - assert!(state.found_matched); } #[tokio::test] @@ -3558,7 +3562,6 @@ mod tests { &expect )); assert_eq!(state.first_output_row_id, Vec::::new()); - assert!(state.found_matched); } #[tokio::test] From 96e81586c39c90bdd12f5de1506e6f9293da260f Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 24 Jan 2025 14:27:38 +0800 Subject: [PATCH 04/25] fix: finish source backfill immediately for scan.startup.mode=latest (#20285) Signed-off-by: xxchan --- .../kafka/shared_source.slt.serial | 30 +++++++++++++++++++ .../src/source/kafka/source/reader.rs | 16 +++++++--- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/e2e_test/source_inline/kafka/shared_source.slt.serial b/e2e_test/source_inline/kafka/shared_source.slt.serial index 42d4cf86725ef..97129c7f6fff3 100644 --- a/e2e_test/source_inline/kafka/shared_source.slt.serial +++ b/e2e_test/source_inline/kafka/shared_source.slt.serial @@ -99,6 +99,10 @@ SET streaming_use_shared_source TO false; statement ok create materialized view mv_2 as select * from s0; +statement ok +SET streaming_use_shared_source TO true; + + sleep 2s query ?? rowsort @@ -370,3 +374,29 @@ drop source s0 cascade; statement ok drop source s_before_produce cascade; + +# test: scan.startup.mode=latest should not be blocked when there's no data to backfill +# https://github.com/risingwavelabs/risingwave/issues/20083#issuecomment-2609422824 +statement ok +create source s_latest (v1 int, v2 varchar) with ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'shared_source', + scan.startup.mode = 'latest' +) FORMAT PLAIN ENCODE JSON; + +# Note: batch kafka scan ignores scan.startup.mode +query ? rowsort +select count(*) from s_latest; +---- +55 + +statement ok +create materialized view mv_latest as select * from s_latest; + +query ? rowsort +select count(*) from mv_latest; +---- +0 + +system ok +rpk topic delete shared_source; diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index e0eddb0dee507..502f06ae31a6c 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -124,9 +124,9 @@ impl SplitReader for KafkaSplitReader { properties.common.sync_call_timeout, ) .await?; - tracing::debug!("fetch kafka watermarks: low: {low}, high: {high}, split: {split:?}"); - // note: low is inclusive, high is exclusive - if low == high { + tracing::info!("fetch kafka watermarks: low: {low}, high: {high}, split: {split:?}"); + // note: low is inclusive, high is exclusive, start_offset is exclusive + if low == high || split.start_offset.is_some_and(|offset| offset + 1 >= high) { backfill_info.insert(split.id(), BackfillInfo::NoDataToBackfill); } else { debug_assert!(high > 0); @@ -138,7 +138,15 @@ impl SplitReader for KafkaSplitReader { ); } } - tracing::debug!("backfill_info: {:?}", backfill_info); + tracing::info!( + topic = properties.common.topic, + source_name = source_ctx.source_name, + fragment_id = source_ctx.fragment_id, + source_id = source_ctx.source_id.table_id, + actor_id = source_ctx.actor_id, + "backfill_info: {:?}", + backfill_info + ); consumer.assign(&tpl)?; From 0942b6093f13a5b685357966d0af7ab17f1fb475 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 24 Jan 2025 16:10:24 +0800 Subject: [PATCH 05/25] test(frontend): add tests for schema change default value not to be refreshed (#17121) (#20291) Signed-off-by: Bugen Zhao --- .../ddl/alter_table_column_issue_17121.slt | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 e2e_test/ddl/alter_table_column_issue_17121.slt diff --git a/e2e_test/ddl/alter_table_column_issue_17121.slt b/e2e_test/ddl/alter_table_column_issue_17121.slt new file mode 100644 index 0000000000000..d96c030d0a20c --- /dev/null +++ b/e2e_test/ddl/alter_table_column_issue_17121.slt @@ -0,0 +1,41 @@ +# https://github.com/risingwavelabs/risingwave/issues/17121 + +statement ok +create table t(v int); + +statement ok +insert into t values (1); + +statement ok +alter table t add column now1 timestamptz default now(); + +# Epoch (then `now()`) will advance after `FLUSH`. +statement ok +flush; + +statement ok +insert into t values (2); + +statement ok +flush; + +query I +select v from t order by now1; +---- +1 +2 + +# Add a new column again, causing the table to be replaced again. +statement ok +alter table t add column v2 varchar; + +# We show that the "snapshot value" of `now1` does not get refreshed upon the above `ALTER TABLE`. +# Otherwise, the `now1` column of `v=1` would be greater than that of `v=2`. +query I +select v from t order by now1; +---- +1 +2 + +statement ok +drop table t; From 2351d7eac62e3712b40c78b4dfd609ea3cbcf9b6 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Fri, 24 Jan 2025 16:19:12 +0800 Subject: [PATCH 06/25] fix: list one page in fs source executor (#20292) Co-authored-by: tabversion --- .../opendal_source/opendal_enumerator.rs | 6 ++++-- .../src/source/filesystem/s3/enumerator.rs | 17 ++++++++--------- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index a770d600282a2..ff1cf47b9b364 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -18,7 +18,7 @@ use anyhow::anyhow; use async_trait::async_trait; use chrono::{DateTime, Utc}; use futures::stream::{self, BoxStream}; -use futures::StreamExt; +use futures::{StreamExt, TryStreamExt}; use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamptz; @@ -54,7 +54,9 @@ impl SplitEnumerator for OpendalEnumerator { let empty_split: OpendalFsSplit = OpendalFsSplit::empty_split(); let prefix = self.prefix.as_deref().unwrap_or("/"); - match self.op.list(prefix).await { + let mut lister = self.op.lister(prefix).await?; + // fetch one item as validation, no need to get all + match lister.try_next().await { Ok(_) => return Ok(vec![empty_split]), Err(e) => { return Err(anyhow!(e) diff --git a/src/connector/src/source/filesystem/s3/enumerator.rs b/src/connector/src/source/filesystem/s3/enumerator.rs index ff15b1c00dc37..65eb551afd183 100644 --- a/src/connector/src/source/filesystem/s3/enumerator.rs +++ b/src/connector/src/source/filesystem/s3/enumerator.rs @@ -99,15 +99,14 @@ impl SplitEnumerator for S3SplitEnumerator { } async fn list_splits(&mut self) -> crate::error::ConnectorResult> { - let mut objects = Vec::new(); - loop { - let (files, has_finished) = self.get_next_page::().await?; - objects.extend(files); - if has_finished { - break; - } - } - Ok(objects) + // fetch one page as validation, no need to get all pages + let (_, _) = self.get_next_page::().await?; + + Ok(vec![FsSplit { + name: "empty_split".to_owned(), + offset: 0, + size: 0, + }]) } } From 217dcfaa73e5450a1c61c60e62f172271ad522c9 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 24 Jan 2025 17:32:40 +0800 Subject: [PATCH 07/25] feat(risedev): parallel service startup (#20286) Signed-off-by: xxchan --- src/risedevtool/src/bin/risedev-dev.rs | 552 +++++++++++++------------ src/risedevtool/src/service_config.rs | 48 +++ src/risedevtool/src/task.rs | 3 +- src/risedevtool/src/util.rs | 10 +- 4 files changed, 356 insertions(+), 257 deletions(-) diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index fbd8dde9a7c89..2ee4c63f4b7f5 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -12,23 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![feature(trait_alias)] + +use std::collections::HashMap; use std::env; use std::fmt::Write; use std::path::{Path, PathBuf}; use std::sync::Arc; +use std::thread::JoinHandle; use std::time::{Duration, Instant}; use anyhow::{anyhow, Context, Result}; use console::style; use fs_err::OpenOptions; -use indicatif::ProgressBar; -use risedev::util::{complete_spin, fail_spin}; +use indicatif::{MultiProgress, ProgressBar}; +use risedev::util::{begin_spin, complete_spin, fail_spin}; use risedev::{ generate_risedev_env, preflight_check, CompactorService, ComputeNodeService, ConfigExpander, ConfigureTmuxTask, DummyService, EnsureStopService, ExecuteContext, FrontendService, GrafanaService, KafkaService, MetaNodeService, MinioService, MySqlService, PostgresService, PrometheusService, PubsubService, RedisService, SchemaRegistryService, ServiceConfig, - SqlServerService, SqliteConfig, Task, TempoService, RISEDEV_NAME, + SqlServerService, SqliteConfig, Task, TaskGroup, TempoService, RISEDEV_NAME, }; use tempfile::tempdir; use thiserror_ext::AsReport; @@ -36,32 +40,22 @@ use tracing::level_filters::LevelFilter; use tracing_subscriber::EnvFilter; use yaml_rust::YamlEmitter; -#[derive(Default)] pub struct ProgressManager { - pa: Option, + pa: MultiProgress, } impl ProgressManager { pub fn new() -> Self { - Self::default() + let pa = MultiProgress::default(); + pa.set_move_cursor(true); + Self { pa } } /// Create a new progress bar from task pub fn new_progress(&mut self) -> ProgressBar { - if let Some(ref pa) = self.pa { - pa.finish(); - } - let pb = risedev::util::new_spinner(); + let pb = risedev::util::new_spinner().with_finish(indicatif::ProgressFinish::AndLeave); pb.enable_steady_tick(Duration::from_millis(100)); - self.pa = Some(pb.clone()); - pb - } - - /// Finish all progress bars. - pub fn finish_all(&self) { - if let Some(ref pa) = self.pa { - pa.finish(); - } + self.pa.add(pb) } } @@ -114,267 +108,312 @@ fn task_main( // Then, start services one by one - let mut stat = vec![]; + let mut tasks = TaskScheduler::new(); for service in services { - let start_time = Instant::now(); - - match service { - ServiceConfig::Minio(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = MinioService::new(c.clone())?; - service.execute(&mut ctx)?; - - let mut task = risedev::ConfigureMinioTask::new(c.clone())?; - task.execute(&mut ctx)?; - } - ServiceConfig::Sqlite(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - - struct SqliteService(SqliteConfig); - impl Task for SqliteService { - fn execute( - &mut self, - _ctx: &mut ExecuteContext, - ) -> anyhow::Result<()> { - Ok(()) + if let ServiceConfig::Frontend(c) = service { + writeln!( + log_buffer, + "* Run {} to start Postgres interactive shell.", + style(format_args!( + "psql -h localhost -p {} -d dev -U root", + c.port + )) + .blue() + .bold() + )?; + } + let service_ = service.clone(); + let progress_bar = manager.new_progress(); + progress_bar.set_prefix(service.id().to_owned()); + progress_bar.set_message("waiting for previous service to start...".to_owned()); + let status_dir = status_dir.clone(); + let closure = move || { + let mut log = Vec::new(); + let start_time = Instant::now(); + let mut ctx = ExecuteContext::new(&mut log, progress_bar, status_dir); + let service = service_; + let id = service.id().to_owned(); + match service { + ServiceConfig::Minio(c) => { + let mut service = MinioService::new(c.clone())?; + service.execute(&mut ctx)?; + + let mut task = risedev::ConfigureMinioTask::new(c.clone())?; + task.execute(&mut ctx)?; + } + ServiceConfig::Sqlite(c) => { + struct SqliteService(SqliteConfig); + impl Task for SqliteService { + fn execute( + &mut self, + _ctx: &mut ExecuteContext, + ) -> anyhow::Result<()> { + Ok(()) + } + + fn id(&self) -> String { + self.0.id.clone() + } } - fn id(&self) -> String { - self.0.id.clone() - } - } + let prefix_data = env::var("PREFIX_DATA")?; + let file_dir = PathBuf::from(&prefix_data).join(&c.id); + std::fs::create_dir_all(&file_dir)?; + let file_path = file_dir.join(&c.file); - let prefix_data = env::var("PREFIX_DATA")?; - let file_dir = PathBuf::from(&prefix_data).join(&c.id); - std::fs::create_dir_all(&file_dir)?; - let file_path = file_dir.join(&c.file); + ctx.service(&SqliteService(c.clone())); + ctx.complete_spin(); + ctx.pb + .set_message(format!("using local sqlite: {:?}", file_path)); + } + ServiceConfig::Prometheus(c) => { + let mut service = PrometheusService::new(c.clone())?; + service.execute(&mut ctx)?; + let mut task = + risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, false)?; + task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("api http://{}:{}/", c.address, c.port)); + } + ServiceConfig::ComputeNode(c) => { + let mut service = ComputeNodeService::new(c.clone())?; + service.execute(&mut ctx)?; - ctx.service(&SqliteService(c.clone())); - ctx.complete_spin(); - ctx.pb - .set_message(format!("using local sqlite: {:?}", file_path)); - } - ServiceConfig::Prometheus(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = PrometheusService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, false)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("api http://{}:{}/", c.address, c.port)); - } - ServiceConfig::ComputeNode(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = ComputeNodeService::new(c.clone())?; - service.execute(&mut ctx)?; - - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("api grpc://{}:{}/", c.address, c.port)); - } - ServiceConfig::MetaNode(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = MetaNodeService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; - ctx.pb.set_message(format!( - "api grpc://{}:{}/, dashboard http://{}:{}/", - c.address, c.port, c.address, c.dashboard_port - )); - } - ServiceConfig::Frontend(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = FrontendService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("api postgres://{}:{}/", c.address, c.port)); - - writeln!( - log_buffer, - "* Run {} to start Postgres interactive shell.", - style(format_args!( - "psql -h localhost -p {} -d dev -U root", - c.port - )) - .blue() - .bold() - )?; - } - ServiceConfig::Compactor(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = CompactorService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("compactor {}:{}", c.address, c.port)); - } - ServiceConfig::Grafana(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = GrafanaService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, false)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("dashboard http://{}:{}/", c.address, c.port)); - } - ServiceConfig::Tempo(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = TempoService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.listen_address.clone(), c.port, false)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("api http://{}:{}/", c.listen_address, c.port)); - } - ServiceConfig::AwsS3(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - DummyService::new(&c.id).execute(&mut ctx)?; - ctx.pb - .set_message(format!("using AWS s3 bucket {}", c.bucket)); - } - ServiceConfig::Opendal(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - DummyService::new(&c.id).execute(&mut ctx)?; - ctx.pb - .set_message(format!("using Opendal, namenode = {}", c.namenode)); - } - ServiceConfig::Kafka(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = KafkaService::new(c.clone()); - service.execute(&mut ctx)?; - let mut task = risedev::KafkaReadyCheckTask::new(c.clone())?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("kafka {}:{}", c.address, c.port)); - } - ServiceConfig::SchemaRegistry(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = SchemaRegistryService::new(c.clone()); - service.execute(&mut ctx)?; - if c.user_managed { let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; task.execute(&mut ctx)?; - } else { + ctx.pb + .set_message(format!("api grpc://{}:{}/", c.address, c.port)); + } + ServiceConfig::MetaNode(c) => { + let mut service = MetaNodeService::new(c.clone())?; + service.execute(&mut ctx)?; let mut task = - risedev::LogReadyCheckTask::new("Server started, listening for requests")?; + risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; task.execute(&mut ctx)?; + ctx.pb.set_message(format!( + "api grpc://{}:{}/, dashboard http://{}:{}/", + c.address, c.port, c.address, c.dashboard_port + )); } - ctx.pb - .set_message(format!("schema registry http://{}:{}", c.address, c.port)); - } - - ServiceConfig::Pubsub(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = PubsubService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = risedev::PubsubReadyTaskCheck::new(c.clone())?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("pubsub {}:{}", c.address, c.port)); - } - ServiceConfig::RedPanda(_) => { - return Err(anyhow!("redpanda is only supported in RiseDev compose.")); - } - ServiceConfig::Redis(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = RedisService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = risedev::RedisReadyCheckTask::new(c.clone())?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("redis {}:{}", c.address, c.port)); - } - ServiceConfig::MySql(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - MySqlService::new(c.clone()).execute(&mut ctx)?; - if c.user_managed { + ServiceConfig::Frontend(c) => { + let mut service = FrontendService::new(c.clone())?; + service.execute(&mut ctx)?; let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; task.execute(&mut ctx)?; - } else { - // When starting a MySQL container, the MySQL process is set as the main process. - // Since the first process in a container always gets PID 1, the MySQL log always shows - // "starting as process 1". - let mut task = risedev::LogReadyCheckTask::new("starting as process 1\n")?; - task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("api postgres://{}:{}/", c.address, c.port)); } - ctx.pb - .set_message(format!("mysql {}:{}", c.address, c.port)); - } - ServiceConfig::Postgres(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - PostgresService::new(c.clone()).execute(&mut ctx)?; - if c.user_managed { + ServiceConfig::Compactor(c) => { + let mut service = CompactorService::new(c.clone())?; + service.execute(&mut ctx)?; let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; task.execute(&mut ctx)?; - } else { - let mut task = risedev::LogReadyCheckTask::new_all([ - "ready to accept connections", // also appears in init process - "listening on IPv4 address", // only appears when ready - ])?; + ctx.pb + .set_message(format!("compactor {}:{}", c.address, c.port)); + } + ServiceConfig::Grafana(c) => { + let mut service = GrafanaService::new(c.clone())?; + service.execute(&mut ctx)?; + let mut task = + risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, false)?; task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("dashboard http://{}:{}/", c.address, c.port)); } - ctx.pb - .set_message(format!("postgres {}:{}", c.address, c.port)); - } - ServiceConfig::SqlServer(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - // only `c.password` will be used in `SqlServerService` as the password for user `sa`. - SqlServerService::new(c.clone()).execute(&mut ctx)?; - if c.user_managed { + ServiceConfig::Tempo(c) => { + let mut service = TempoService::new(c.clone())?; + service.execute(&mut ctx)?; let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; + risedev::TcpReadyCheckTask::new(c.listen_address.clone(), c.port, false)?; + task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("api http://{}:{}/", c.listen_address, c.port)); + } + ServiceConfig::AwsS3(c) => { + DummyService::new(&c.id).execute(&mut ctx)?; + ctx.pb + .set_message(format!("using AWS s3 bucket {}", c.bucket)); + } + ServiceConfig::Opendal(c) => { + DummyService::new(&c.id).execute(&mut ctx)?; + ctx.pb + .set_message(format!("using Opendal, namenode = {}", c.namenode)); + } + ServiceConfig::Kafka(c) => { + let mut service = KafkaService::new(c.clone()); + service.execute(&mut ctx)?; + let mut task = risedev::KafkaReadyCheckTask::new(c.clone())?; + task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("kafka {}:{}", c.address, c.port)); + } + ServiceConfig::SchemaRegistry(c) => { + let mut service = SchemaRegistryService::new(c.clone()); + service.execute(&mut ctx)?; + if c.user_managed { + let mut task = risedev::TcpReadyCheckTask::new( + c.address.clone(), + c.port, + c.user_managed, + )?; + task.execute(&mut ctx)?; + } else { + let mut task = risedev::LogReadyCheckTask::new( + "Server started, listening for requests", + )?; + task.execute(&mut ctx)?; + } + ctx.pb + .set_message(format!("schema registry http://{}:{}", c.address, c.port)); + } + + ServiceConfig::Pubsub(c) => { + let mut service = PubsubService::new(c.clone())?; + service.execute(&mut ctx)?; + let mut task = risedev::PubsubReadyTaskCheck::new(c.clone())?; task.execute(&mut ctx)?; - } else { - let mut task = risedev::LogReadyCheckTask::new( - "SQL Server is now ready for client connections.", - )?; + ctx.pb + .set_message(format!("pubsub {}:{}", c.address, c.port)); + } + ServiceConfig::RedPanda(_) => { + return Err(anyhow!("redpanda is only supported in RiseDev compose.")); + } + ServiceConfig::Redis(c) => { + let mut service = RedisService::new(c.clone())?; + service.execute(&mut ctx)?; + let mut task = risedev::RedisReadyCheckTask::new(c.clone())?; task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("redis {}:{}", c.address, c.port)); + } + ServiceConfig::MySql(c) => { + MySqlService::new(c.clone()).execute(&mut ctx)?; + if c.user_managed { + let mut task = risedev::TcpReadyCheckTask::new( + c.address.clone(), + c.port, + c.user_managed, + )?; + task.execute(&mut ctx)?; + } else { + // When starting a MySQL container, the MySQL process is set as the main process. + // Since the first process in a container always gets PID 1, the MySQL log always shows + // "starting as process 1". + let mut task = risedev::LogReadyCheckTask::new("starting as process 1\n")?; + task.execute(&mut ctx)?; + } + ctx.pb + .set_message(format!("mysql {}:{}", c.address, c.port)); + } + ServiceConfig::Postgres(c) => { + PostgresService::new(c.clone()).execute(&mut ctx)?; + if c.user_managed { + let mut task = risedev::TcpReadyCheckTask::new( + c.address.clone(), + c.port, + c.user_managed, + )?; + task.execute(&mut ctx)?; + } else { + let mut task = risedev::LogReadyCheckTask::new_all([ + "ready to accept connections", // also appears in init process + "listening on IPv4 address", // only appears when ready + ])?; + task.execute(&mut ctx)?; + } + ctx.pb + .set_message(format!("postgres {}:{}", c.address, c.port)); + } + ServiceConfig::SqlServer(c) => { + // only `c.password` will be used in `SqlServerService` as the password for user `sa`. + SqlServerService::new(c.clone()).execute(&mut ctx)?; + if c.user_managed { + let mut task = risedev::TcpReadyCheckTask::new( + c.address.clone(), + c.port, + c.user_managed, + )?; + task.execute(&mut ctx)?; + } else { + let mut task = risedev::LogReadyCheckTask::new( + "SQL Server is now ready for client connections.", + )?; + task.execute(&mut ctx)?; + } + ctx.pb + .set_message(format!("sqlserver {}:{}", c.address, c.port)); } - ctx.pb - .set_message(format!("sqlserver {}:{}", c.address, c.port)); } - } - let service_id = service.id().to_owned(); - let duration = Instant::now() - start_time; - stat.push((service_id, duration)); + let duration = Instant::now() - start_time; + Ok(TaskResult { + id, + time: duration, + log: String::from_utf8(log)?, + }) + }; + tasks.add(service, closure); } + let stat = tasks.run(&mut logger)?; + Ok((stat, log_buffer)) } +struct TaskResult { + id: String, + time: Duration, + log: String, +} +trait TaskFn = FnOnce() -> anyhow::Result + Send + 'static; +struct TaskScheduler { + /// In each group, the tasks are executed in sequence. + task_groups: HashMap>>, +} + +impl TaskScheduler { + fn new() -> Self { + Self { + task_groups: HashMap::new(), + } + } + + fn add(&mut self, config: &ServiceConfig, task: impl TaskFn) { + self.task_groups + .entry(config.task_group()) + .or_default() + .push(Box::new(task)); + } + + fn run(self, logger: &mut impl std::io::Write) -> anyhow::Result> { + let mut handles: Vec>>> = vec![]; + let mut stats = vec![]; + + let task_groups = self.task_groups; + for (_, tasks) in task_groups { + handles.push(std::thread::spawn(move || { + let mut res = vec![]; + for task in tasks { + let res_ = task()?; + res.push(res_); + } + Ok(res) + })); + } + for handle in handles { + for TaskResult { id, time, log } in handle.join().unwrap()? { + stats.push((id, time)); + write!(logger, "{}", log)?; + } + } + Ok(stats) + } +} + fn main() -> Result<()> { // Intentionally disable backtrace to provide more compact error message for `risedev dev`. // Backtraces for RisingWave components are enabled in `Task::execute`. @@ -386,17 +425,19 @@ fn main() -> Result<()> { EnvFilter::builder() .with_default_directive(LevelFilter::INFO.into()) .with_env_var("RISEDEV_RUST_LOG") - .from_env_lossy(), + .from_env_lossy() + // This log may pollute the progress bar. + .add_directive("librdkafka=off".parse().unwrap()), ) .init(); preflight_check()?; - let task_name = std::env::args() + let profile = std::env::args() .nth(1) .unwrap_or_else(|| "default".to_owned()); - let (config_path, env, risedev_config) = ConfigExpander::expand(".", &task_name)?; + let (config_path, env, risedev_config) = ConfigExpander::expand(".", &profile)?; if let Some(config_path) = &config_path { let target = Path::new(&env::var("PREFIX_CONFIG")?).join("risingwave.toml"); @@ -418,11 +459,12 @@ fn main() -> Result<()> { // Always create a progress before calling `task_main`. Otherwise the progress bar won't be // shown. let p = manager.new_progress(); + begin_spin(&p); p.set_prefix("dev cluster"); p.set_message(format!( "starting {} services for {}...", services.len(), - task_name + profile )); let task_result = task_main(&mut manager, &services, env); @@ -430,19 +472,19 @@ fn main() -> Result<()> { Ok(_) => { p.set_message(format!( "done bootstrapping with config {}", - style(task_name).bold() + style(profile).bold() )); complete_spin(&p); } Err(_) => { p.set_message(format!( "failed to bootstrap with config {}", - style(task_name).bold() + style(profile).bold() )); fail_spin(&p); } } - manager.finish_all(); + p.finish(); use risedev::util::stylized_risedev_subcmd as r; diff --git a/src/risedevtool/src/service_config.rs b/src/risedevtool/src/service_config.rs index c1070d43261c6..f30e69701f438 100644 --- a/src/risedevtool/src/service_config.rs +++ b/src/risedevtool/src/service_config.rs @@ -432,6 +432,18 @@ pub enum ServiceConfig { SqlServer(SqlServerConfig), } +#[derive(PartialEq, Eq, Hash)] +pub enum TaskGroup { + RisingWave, + Observability, + Kafka, + Pubsub, + MySql, + Postgres, + SqlServer, + Redis, +} + impl ServiceConfig { pub fn id(&self) -> &str { match self { @@ -505,6 +517,42 @@ impl ServiceConfig { Self::SchemaRegistry(c) => c.user_managed, } } + + pub fn task_group(&self) -> TaskGroup { + use TaskGroup::*; + match self { + ServiceConfig::ComputeNode(_) + | ServiceConfig::MetaNode(_) + | ServiceConfig::Frontend(_) + | ServiceConfig::Compactor(_) + | ServiceConfig::Minio(_) + | ServiceConfig::Sqlite(_) => RisingWave, + ServiceConfig::Prometheus(_) | ServiceConfig::Grafana(_) | ServiceConfig::Tempo(_) => { + Observability + } + ServiceConfig::Opendal(_) | ServiceConfig::AwsS3(_) => RisingWave, + ServiceConfig::Kafka(_) + | ServiceConfig::SchemaRegistry(_) + | ServiceConfig::RedPanda(_) => Kafka, + ServiceConfig::Pubsub(_) => Pubsub, + ServiceConfig::Redis(_) => Redis, + ServiceConfig::MySql(my_sql_config) => { + if matches!(my_sql_config.application, Application::Metastore) { + RisingWave + } else { + MySql + } + } + ServiceConfig::Postgres(postgres_config) => { + if matches!(postgres_config.application, Application::Metastore) { + RisingWave + } else { + Postgres + } + } + ServiceConfig::SqlServer(_) => SqlServer, + } + } } mod string { diff --git a/src/risedevtool/src/task.rs b/src/risedevtool/src/task.rs index 65c34c649ed68..c4dcab8426aba 100644 --- a/src/risedevtool/src/task.rs +++ b/src/risedevtool/src/task.rs @@ -76,7 +76,7 @@ pub use self::task_pubsub_emu_ready_check::*; pub use self::task_redis_ready_check::*; pub use self::task_tcp_ready_check::*; pub use self::tempo_service::*; -use crate::util::{complete_spin, get_program_args, get_program_name}; +use crate::util::{begin_spin, complete_spin, get_program_args, get_program_name}; use crate::wait::{wait, wait_tcp_available}; pub trait Task: 'static + Send { @@ -134,6 +134,7 @@ where pub fn service(&mut self, task: &impl Task) { let id = task.id(); if !id.is_empty() { + begin_spin(&self.pb); self.pb.set_prefix(id.clone()); self.id = Some(id.clone()); self.status_file = Some(self.status_dir.path().join(format!("{}.status", id))); diff --git a/src/risedevtool/src/util.rs b/src/risedevtool/src/util.rs index d25ad3d92788f..79bab7977d209 100644 --- a/src/risedevtool/src/util.rs +++ b/src/risedevtool/src/util.rs @@ -47,12 +47,20 @@ pub fn new_spinner() -> ProgressBar { let pb = ProgressBar::new(0); pb.set_style( ProgressStyle::default_spinner() - .template("{spinner} {prefix}: {msg}") + .template("🟡 {prefix}: {msg}") .unwrap(), ); pb } +pub fn begin_spin(pb: &ProgressBar) { + pb.set_style( + ProgressStyle::default_spinner() + .template("{spinner} {prefix}: {msg}") + .unwrap(), + ); +} + pub fn complete_spin(pb: &ProgressBar) { pb.set_style( ProgressStyle::default_spinner() From a1e5188af4362f24de90130889abe4f4ff416c22 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 24 Jan 2025 17:37:31 +0800 Subject: [PATCH 08/25] fix(test): didn't drop source but deleted topic (#20293) Signed-off-by: xxchan --- e2e_test/source_inline/kafka/shared_source.slt.serial | 3 +++ 1 file changed, 3 insertions(+) diff --git a/e2e_test/source_inline/kafka/shared_source.slt.serial b/e2e_test/source_inline/kafka/shared_source.slt.serial index 97129c7f6fff3..1559962f8b878 100644 --- a/e2e_test/source_inline/kafka/shared_source.slt.serial +++ b/e2e_test/source_inline/kafka/shared_source.slt.serial @@ -398,5 +398,8 @@ select count(*) from mv_latest; ---- 0 +statement ok +drop source s_latest cascade; + system ok rpk topic delete shared_source; From fa4c46373f5ebe3b10b4573e19b7c6abc243fa23 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Sat, 25 Jan 2025 23:02:53 +0800 Subject: [PATCH 09/25] fix(playground): listen to `127.0.0.1` instead of `0.0.0.0` for gRPC services (#20297) Signed-off-by: Bugen Zhao --- src/cmd_all/src/bin/risingwave.rs | 6 +++++- src/cmd_all/src/single_node.rs | 9 +++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index a8cd77bf6e21d..ebdeab941a82e 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -122,7 +122,7 @@ impl Component { Self::Frontend => frontend(parse_opts(matches)), Self::Compactor => compactor(parse_opts(matches)), Self::Ctl => ctl(parse_opts(matches)), - Self::Playground => single_node(SingleNodeOpts::new_for_playground()), + Self::Playground => playground(), Self::Standalone => standalone(parse_opts(matches)), Self::SingleNode => single_node(parse_opts(matches)), } @@ -248,6 +248,10 @@ fn single_node(opts: SingleNodeOpts) -> ! { risingwave_rt::main_okk(|shutdown| risingwave_cmd_all::standalone(opts, shutdown)); } +fn playground() -> ! { + single_node(SingleNodeOpts::new_for_playground()); +} + #[cfg(test)] mod tests { use std::assert_matches::assert_matches; diff --git a/src/cmd_all/src/single_node.rs b/src/cmd_all/src/single_node.rs index 0f6f574ad5a8c..5ecc60bff743c 100644 --- a/src/cmd_all/src/single_node.rs +++ b/src/cmd_all/src/single_node.rs @@ -194,14 +194,11 @@ pub fn map_single_node_opts_to_standalone_opts(opts: SingleNodeOpts) -> ParsedSt } // Set listen addresses (force to override) - meta_opts.listen_addr = "0.0.0.0:5690".to_owned(); + meta_opts.listen_addr = "127.0.0.1:5690".to_owned(); meta_opts.advertise_addr = "127.0.0.1:5690".to_owned(); meta_opts.dashboard_host = Some("0.0.0.0:5691".to_owned()); - compute_opts.listen_addr = "0.0.0.0:5688".to_owned(); - compactor_opts.listen_addr = "0.0.0.0:6660".to_owned(); - if let Some(frontend_addr) = &opts.node_opts.listen_addr { - frontend_opts.listen_addr.clone_from(frontend_addr); - } + compute_opts.listen_addr = "127.0.0.1:5688".to_owned(); + compactor_opts.listen_addr = "127.0.0.1:6660".to_owned(); // Set Meta addresses for all nodes (force to override) let meta_addr = "http://127.0.0.1:5690".to_owned(); From 1384d4592ba94ec873c5f513946eb7b7fbfa36e5 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Sun, 26 Jan 2025 18:40:16 +0800 Subject: [PATCH 10/25] fix(parquet): handle nested data types correctly (#20156) --- e2e_test/s3/file_sink.py | 36 ++++-- src/connector/src/parser/parquet_parser.rs | 5 + .../source/iceberg/parquet_file_handler.rs | 108 +++++++++--------- 3 files changed, 85 insertions(+), 64 deletions(-) diff --git a/e2e_test/s3/file_sink.py b/e2e_test/s3/file_sink.py index 6eca0e2b9194f..1979b7f6606ab 100644 --- a/e2e_test/s3/file_sink.py +++ b/e2e_test/s3/file_sink.py @@ -17,6 +17,12 @@ def gen_data(file_num, item_num_per_file): assert item_num_per_file % 2 == 0, \ f'item_num_per_file should be even to ensure sum(mark) == 0: {item_num_per_file}' + + struct_type = pa.struct([ + ('field1', pa.int32()), + ('field2', pa.string()) + ]) + return [ [{ 'id': file_id * item_num_per_file + item_id, @@ -44,6 +50,7 @@ def gen_data(file_num, item_num_per_file): 'test_timestamptz_ms': pa.scalar(datetime.now().timestamp() * 1000, type=pa.timestamp('ms', tz='+00:00')), 'test_timestamptz_us': pa.scalar(datetime.now().timestamp() * 1000000, type=pa.timestamp('us', tz='+00:00')), 'test_timestamptz_ns': pa.scalar(datetime.now().timestamp() * 1000000000, type=pa.timestamp('ns', tz='+00:00')), + 'nested_struct': pa.scalar((item_id, f'struct_value_{item_id}'), type=struct_type), } for item_id in range(item_num_per_file)] for file_id in range(file_num) ] @@ -65,7 +72,7 @@ def _table(): print("test table function file scan") cur.execute(f''' SELECT - id, + id, name, sex, mark, @@ -89,7 +96,8 @@ def _table(): test_timestamptz_s, test_timestamptz_ms, test_timestamptz_us, - test_timestamptz_ns + test_timestamptz_ns, + nested_struct FROM file_scan( 'parquet', 's3', @@ -104,7 +112,6 @@ def _table(): except ValueError as e: print(f"cur.fetchone() got ValueError: {e}") - print("file scan test pass") # Execute a SELECT statement cur.execute(f'''CREATE TABLE {_table()}( @@ -132,8 +139,8 @@ def _table(): test_timestamptz_s timestamptz, test_timestamptz_ms timestamptz, test_timestamptz_us timestamptz, - test_timestamptz_ns timestamptz - + test_timestamptz_ns timestamptz, + nested_struct STRUCT<"field1" int, "field2" varchar> ) WITH ( connector = 's3', match_pattern = '*.parquet', @@ -213,7 +220,8 @@ def _table(): test_timestamptz_s, test_timestamptz_ms, test_timestamptz_us, - test_timestamptz_ns + test_timestamptz_ns, + nested_struct from {_table()} WITH ( connector = 's3', match_pattern = '*.parquet', @@ -230,7 +238,7 @@ def _table(): print('Sink into s3 in parquet encode...') # Execute a SELECT statement cur.execute(f'''CREATE TABLE test_parquet_sink_table( - id bigint primary key,\ + id bigint primary key, name TEXT, sex bigint, mark bigint, @@ -254,7 +262,8 @@ def _table(): test_timestamptz_s timestamptz, test_timestamptz_ms timestamptz, test_timestamptz_us timestamptz, - test_timestamptz_ns timestamptz + test_timestamptz_ns timestamptz, + nested_struct STRUCT<"field1" int, "field2" varchar>, ) WITH ( connector = 's3', match_pattern = 'test_parquet_sink/*.parquet', @@ -263,8 +272,8 @@ def _table(): s3.credentials.access = 'hummockadmin', s3.credentials.secret = 'hummockadmin', s3.endpoint_url = 'http://hummock001.127.0.0.1:9301', + refresh.interval.sec = 1, ) FORMAT PLAIN ENCODE PARQUET;''') - total_rows = file_num * item_num_per_file MAX_RETRIES = 40 for retry_no in range(MAX_RETRIES): @@ -305,7 +314,8 @@ def _table(): test_timestamptz_s, test_timestamptz_ms, test_timestamptz_us, - test_timestamptz_ns + test_timestamptz_ns, + nested_struct from {_table()} WITH ( connector = 'snowflake', match_pattern = '*.parquet', @@ -316,7 +326,8 @@ def _table(): s3.endpoint_url = 'http://hummock001.127.0.0.1:9301', s3.path = 'test_json_sink/', type = 'append-only', - force_append_only='true' + force_append_only='true', + refresh.interval.sec = 1, ) FORMAT PLAIN ENCODE JSON(force_append_only='true');''') print('Sink into s3 in json encode...') @@ -346,7 +357,8 @@ def _table(): test_timestamptz_s timestamptz, test_timestamptz_ms timestamptz, test_timestamptz_us timestamptz, - test_timestamptz_ns timestamptz + test_timestamptz_ns timestamptz, + nested_struct STRUCT<"field1" int, "field2" varchar> ) WITH ( connector = 's3', match_pattern = 'test_json_sink/*.json', diff --git a/src/connector/src/parser/parquet_parser.rs b/src/connector/src/parser/parquet_parser.rs index 5bef3b6310981..29148c9305ddc 100644 --- a/src/connector/src/parser/parquet_parser.rs +++ b/src/connector/src/parser/parquet_parser.rs @@ -26,6 +26,7 @@ use risingwave_common::util::tokio_util::compat::FuturesAsyncReadCompatExt; use crate::parser::ConnectorResult; use crate::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use crate::source::filesystem::opendal_source::{OpendalGcs, OpendalPosixFs, OpendalS3}; +use crate::source::iceberg::is_parquet_schema_match_source_schema; use crate::source::reader::desc::SourceDesc; use crate::source::{ConnectorProperties, SourceColumnDesc}; /// `ParquetParser` is responsible for converting the incoming `record_batch_stream` @@ -109,6 +110,10 @@ impl ParquetParser { if let Some(parquet_column) = record_batch.column_by_name(rw_column_name) + && is_parquet_schema_match_source_schema( + parquet_column.data_type(), + rw_data_type, + ) { let arrow_field = IcebergArrowConvert .to_arrow_field(rw_column_name, rw_data_type)?; diff --git a/src/connector/src/source/iceberg/parquet_file_handler.rs b/src/connector/src/source/iceberg/parquet_file_handler.rs index 187140855c29b..6baf30a5795e8 100644 --- a/src/connector/src/source/iceberg/parquet_file_handler.rs +++ b/src/connector/src/source/iceberg/parquet_file_handler.rs @@ -36,7 +36,7 @@ use parquet::file::metadata::{FileMetaData, ParquetMetaData, ParquetMetaDataRead use risingwave_common::array::arrow::arrow_schema_udf::{DataType as ArrowDateType, IntervalUnit}; use risingwave_common::array::arrow::IcebergArrowConvert; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::ColumnId; +use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::types::DataType as RwDataType; use risingwave_common::util::tokio_util::compat::FuturesAsyncReadCompatExt; use url::Url; @@ -217,55 +217,64 @@ pub async fn list_data_directory( } } -/// Extracts valid column indices from a Parquet file schema based on the user's requested schema. +/// Extracts a suitable `ProjectionMask` from a Parquet file schema based on the user's requested schema. /// -/// This function is used for column pruning of Parquet files. It calculates the intersection -/// between the columns in the currently read Parquet file and the schema provided by the user. -/// This is useful for reading a `RecordBatch` with the appropriate `ProjectionMask`, ensuring that -/// only the necessary columns are read. +/// This function is utilized for column pruning of Parquet files. It checks the user's requested schema +/// against the schema of the currently read Parquet file. If the provided `columns` are `None` +/// or if the Parquet file contains nested data types, it returns `ProjectionMask::all()`. Otherwise, +/// it returns only the columns where both the data type and column name match the requested schema, +/// facilitating efficient reading of the `RecordBatch`. /// /// # Parameters -/// - `columns`: A vector of `Column` representing the user's requested schema. +/// - `columns`: An optional vector of `Column` representing the user's requested schema. /// - `metadata`: A reference to `FileMetaData` containing the schema and metadata of the Parquet file. /// /// # Returns -/// - A `ConnectorResult>`, which contains the indices of the valid columns in the -/// Parquet file schema that match the requested schema. If an error occurs during processing, -/// it returns an appropriate error. -pub fn extract_valid_column_indices( - rw_columns: Vec, +/// - A `ConnectorResult`, which represents the valid columns in the Parquet file schema +/// that correspond to the requested schema. If an error occurs during processing, it returns an +/// appropriate error. +pub fn get_project_mask( + columns: Option>, metadata: &FileMetaData, -) -> ConnectorResult> { - let parquet_column_names = metadata - .schema_descr() - .columns() - .iter() - .map(|c| c.name()) - .collect_vec(); +) -> ConnectorResult { + match columns { + Some(rw_columns) => { + let root_column_names = metadata + .schema_descr() + .root_schema() + .get_fields() + .iter() + .map(|field| field.name()) + .collect_vec(); - let converted_arrow_schema = - parquet_to_arrow_schema(metadata.schema_descr(), metadata.key_value_metadata()) - .map_err(anyhow::Error::from)?; + let converted_arrow_schema = + parquet_to_arrow_schema(metadata.schema_descr(), metadata.key_value_metadata()) + .map_err(anyhow::Error::from)?; + let valid_column_indices: Vec = rw_columns + .iter() + .filter_map(|column| { + root_column_names + .iter() + .position(|&name| name == column.name) + .and_then(|pos| { + let arrow_data_type: &risingwave_common::array::arrow::arrow_schema_udf::DataType = converted_arrow_schema.field_with_name(&column.name).ok()?.data_type(); + let rw_data_type: &risingwave_common::types::DataType = &column.data_type; + if is_parquet_schema_match_source_schema(arrow_data_type, rw_data_type) { + Some(pos) + } else { + None + } + }) + }) + .collect(); - let valid_column_indices: Vec = rw_columns - .iter() - .filter_map(|column| { - parquet_column_names - .iter() - .position(|&name| name == column.name) - .and_then(|pos| { - let arrow_data_type: &risingwave_common::array::arrow::arrow_schema_udf::DataType = converted_arrow_schema.field(pos).data_type(); - let rw_data_type: &risingwave_common::types::DataType = &column.data_type; - - if is_parquet_schema_match_source_schema(arrow_data_type, rw_data_type) { - Some(pos) - } else { - None - } - }) - }) - .collect(); - Ok(valid_column_indices) + Ok(ProjectionMask::roots( + metadata.schema_descr(), + valid_column_indices, + )) + } + None => Ok(ProjectionMask::all()), + } } /// Reads a specified Parquet file and converts its content into a stream of chunks. @@ -289,13 +298,7 @@ pub async fn read_parquet_file( let parquet_metadata = reader.get_metadata().await.map_err(anyhow::Error::from)?; let file_metadata = parquet_metadata.file_metadata(); - let projection_mask = match rw_columns { - Some(columns) => { - let column_indices = extract_valid_column_indices(columns, file_metadata)?; - ProjectionMask::leaves(file_metadata.schema_descr(), column_indices) - } - None => ProjectionMask::all(), - }; + let projection_mask = get_project_mask(rw_columns, file_metadata)?; // For the Parquet format, we directly convert from a record batch to a stream chunk. // Therefore, the offset of the Parquet file represents the current position in terms of the number of rows read from the file. @@ -318,11 +321,12 @@ pub async fn read_parquet_file( .enumerate() .map(|(index, field_ref)| { let data_type = IcebergArrowConvert.type_from_field(field_ref).unwrap(); - SourceColumnDesc::simple( + let column_desc = ColumnDesc::named( field_ref.name().clone(), - data_type, ColumnId::new(index as i32), - ) + data_type, + ); + SourceColumnDesc::from(&column_desc) }) .collect(), }; @@ -367,7 +371,7 @@ pub async fn get_parquet_fields( /// - Arrow's `UInt32` matches with RisingWave's `Int64`. /// - Arrow's `UInt64` matches with RisingWave's `Decimal`. /// - Arrow's `Float16` matches with RisingWave's `Float32`. -fn is_parquet_schema_match_source_schema( +pub fn is_parquet_schema_match_source_schema( arrow_data_type: &ArrowDateType, rw_data_type: &RwDataType, ) -> bool { From f6bebeae436c7065f9aa39a6fdbaf3b95d3d4c81 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Sun, 26 Jan 2025 23:36:27 +0800 Subject: [PATCH 11/25] refactor: refactor series of visit_stream_node method (#20313) --- src/common/src/util/stream_graph_visitor.rs | 41 ++++++++++---------- src/meta/src/controller/fragment.rs | 16 ++++---- src/meta/src/controller/streaming_job.rs | 26 ++++++------- src/meta/src/model/stream.rs | 6 +-- src/meta/src/rpc/ddl_controller.rs | 6 +-- src/meta/src/stream/stream_graph/fragment.rs | 2 +- src/meta/src/stream/stream_graph/schedule.rs | 2 +- 7 files changed, 49 insertions(+), 50 deletions(-) diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index 5e579565b7e9f..f14ceadc09c42 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -19,21 +19,11 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{agg_call_state, StreamNode}; /// A utility for visiting and mutating the [`NodeBody`] of the [`StreamNode`]s recursively. -pub fn visit_stream_node(stream_node: &mut StreamNode, mut f: F) -where - F: FnMut(&mut NodeBody), -{ - fn visit_inner(stream_node: &mut StreamNode, f: &mut F) - where - F: FnMut(&mut NodeBody), - { +pub fn visit_stream_node_mut(stream_node: &mut StreamNode, mut f: impl FnMut(&mut NodeBody)) { + visit_stream_node_cont_mut(stream_node, |stream_node| { f(stream_node.node_body.as_mut().unwrap()); - for input in &mut stream_node.input { - visit_inner(input, f); - } - } - - visit_inner(stream_node, &mut f) + true + }) } /// A utility for to accessing the [`StreamNode`] mutably. The returned bool is used to determine whether the access needs to continue. @@ -56,6 +46,14 @@ where visit_inner(stream_node, &mut f) } +/// A utility for visiting the [`NodeBody`] of the [`StreamNode`]s recursively. +pub fn visit_stream_node(stream_node: &StreamNode, mut f: impl FnMut(&NodeBody)) { + visit_stream_node_cont(stream_node, |stream_node| { + f(stream_node.node_body.as_ref().unwrap()); + true + }) +} + /// A utility for to accessing the [`StreamNode`] immutably. The returned bool is used to determine whether the access needs to continue. pub fn visit_stream_node_cont(stream_node: &StreamNode, mut f: F) where @@ -78,11 +76,14 @@ where /// A utility for visiting and mutating the [`NodeBody`] of the [`StreamNode`]s in a /// [`StreamFragment`] recursively. -pub fn visit_fragment(fragment: &mut StreamFragment, f: F) -where - F: FnMut(&mut NodeBody), -{ - visit_stream_node(fragment.node.as_mut().unwrap(), f) +pub fn visit_fragment_mut(fragment: &mut StreamFragment, f: impl FnMut(&mut NodeBody)) { + visit_stream_node_mut(fragment.node.as_mut().unwrap(), f) +} + +/// A utility for visiting the [`NodeBody`] of the [`StreamNode`]s in a +/// [`StreamFragment`] recursively. +pub fn visit_fragment(fragment: &StreamFragment, f: impl FnMut(&NodeBody)) { + visit_stream_node(fragment.node.as_ref().unwrap(), f) } /// Visit the tables of a [`StreamNode`]. @@ -279,7 +280,7 @@ pub fn visit_stream_node_tables_inner( } }; if visit_child_recursively { - visit_stream_node(stream_node, visit_body) + visit_stream_node_mut(stream_node, visit_body) } else { visit_body(stream_node.node_body.as_mut().unwrap()) } diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index f2c95b0f59e3b..4a13155c547aa 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -20,7 +20,7 @@ use anyhow::Context; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::hash::{VnodeCount, VnodeCountCompat, WorkerSlotId}; -use risingwave_common::util::stream_graph_visitor::visit_stream_node; +use risingwave_common::util::stream_graph_visitor::visit_stream_node_mut; use risingwave_meta_model::actor::ActorStatus; use risingwave_meta_model::fragment::DistributionType; use risingwave_meta_model::object::ObjectType; @@ -227,7 +227,7 @@ impl CatalogController { let stream_node = { let actor_template = pb_actors.first().cloned().unwrap(); let mut stream_node = actor_template.nodes.unwrap(); - visit_stream_node(&mut stream_node, |body| { + visit_stream_node_mut(&mut stream_node, |body| { if let NodeBody::Merge(m) = body { m.upstream_actor_id = vec![]; } @@ -244,7 +244,7 @@ impl CatalogController { let node = actor.nodes.as_mut().context("nodes are empty")?; - visit_stream_node(node, |body| { + visit_stream_node_mut(node, |body| { if let NodeBody::Merge(m) = body { let mut upstream_actor_ids = vec![]; swap(&mut m.upstream_actor_id, &mut upstream_actor_ids); @@ -435,7 +435,7 @@ impl CatalogController { let pb_nodes = { let mut nodes = stream_node_template.clone(); - visit_stream_node(&mut nodes, |body| { + visit_stream_node_mut(&mut nodes, |body| { if let NodeBody::Merge(m) = body && let Some(upstream_actor_ids) = upstream_fragment_actors.get(&(m.upstream_fragment_id as _)) @@ -1625,7 +1625,7 @@ mod tests { use itertools::Itertools; use risingwave_common::hash::{ActorMapping, VirtualNode, VnodeCount}; use risingwave_common::util::iter_util::ZipEqDebug; - use risingwave_common::util::stream_graph_visitor::visit_stream_node; + use risingwave_common::util::stream_graph_visitor::{visit_stream_node, visit_stream_node_mut}; use risingwave_meta_model::actor::ActorStatus; use risingwave_meta_model::fragment::DistributionType; use risingwave_meta_model::{ @@ -1800,7 +1800,7 @@ mod tests { let nodes = nodes.unwrap(); let actor_upstream_actor_ids = upstream_actor_ids.get(&(actor_id as _)).cloned().unwrap(); - visit_stream_node(&mut template_node, |body| { + visit_stream_node_mut(&mut template_node, |body| { if let NodeBody::Merge(m) = body { m.upstream_actor_id = actor_upstream_actor_ids .get(&(m.upstream_fragment_id as _)) @@ -1978,9 +1978,7 @@ mod tests { assert_eq!(mview_definition, ""); - let mut pb_nodes = pb_nodes.unwrap(); - - visit_stream_node(&mut pb_nodes, |body| { + visit_stream_node(pb_nodes.as_ref().unwrap(), |body| { if let PbNodeBody::Merge(m) = body { let upstream_actor_ids = upstream_actor_ids .get(&(m.upstream_fragment_id as _)) diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 29d900663443f..669c6494667b7 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use risingwave_common::config::DefaultParallelism; use risingwave_common::hash::VnodeCountCompat; use risingwave_common::util::column_index_mapping::ColIndexMapping; -use risingwave_common::util::stream_graph_visitor::visit_stream_node; +use risingwave_common::util::stream_graph_visitor::{visit_stream_node, visit_stream_node_mut}; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::WithPropertiesExt; use risingwave_meta_model::actor::ActorStatus; @@ -1183,7 +1183,7 @@ impl CatalogController { .await? .map(|(id, node, upstream)| (id, node.to_protobuf(), upstream)) .ok_or_else(|| MetaError::catalog_id_not_found("fragment", fragment_id))?; - visit_stream_node(&mut stream_node, |body| { + visit_stream_node_mut(&mut stream_node, |body| { if let PbNodeBody::Merge(m) = body && let Some((new_fragment_id, new_actor_ids)) = fragment_replace_map.get(&m.upstream_fragment_id) @@ -1356,7 +1356,7 @@ impl CatalogController { fragments.retain_mut(|(_, fragment_type_mask, stream_node)| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::Source as i32 != 0 { - visit_stream_node(stream_node, |node| { + visit_stream_node_mut(stream_node, |node| { if let PbNodeBody::Source(node) = node { if let Some(node_inner) = &mut node.source_inner && node_inner.source_id == source_id as u32 @@ -1370,7 +1370,7 @@ impl CatalogController { if is_fs_source { // in older versions, there's no fragment type flag for `FsFetch` node, // so we just scan all fragments for StreamFsFetch node if using fs connector - visit_stream_node(stream_node, |node| { + visit_stream_node_mut(stream_node, |node| { if let PbNodeBody::StreamFsFetch(node) = node { *fragment_type_mask |= PbFragmentTypeFlag::FsFetch as i32; if let Some(node_inner) = &mut node.node_inner @@ -1486,7 +1486,7 @@ impl CatalogController { |fragment_type_mask: &mut i32, stream_node: &mut PbStreamNode| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::backfill_rate_limit_fragments() != 0 { - visit_stream_node(stream_node, |node| match node { + visit_stream_node_mut(stream_node, |node| match node { PbNodeBody::StreamCdcScan(node) => { node.rate_limit = rate_limit; found = true; @@ -1528,7 +1528,7 @@ impl CatalogController { |fragment_type_mask: &mut i32, stream_node: &mut PbStreamNode| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::sink_rate_limit_fragments() != 0 { - visit_stream_node(stream_node, |node| { + visit_stream_node_mut(stream_node, |node| { if let PbNodeBody::Sink(node) = node { node.rate_limit = rate_limit; found = true; @@ -1551,7 +1551,7 @@ impl CatalogController { |fragment_type_mask: &mut i32, stream_node: &mut PbStreamNode| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::dml_rate_limit_fragments() != 0 { - visit_stream_node(stream_node, |node| { + visit_stream_node_mut(stream_node, |node| { if let PbNodeBody::Dml(node) = node { node.rate_limit = rate_limit; found = true; @@ -1635,7 +1635,7 @@ impl CatalogController { PbStreamActor { actor_id, fragment_id, - mut nodes, + nodes, dispatcher, upstream_actor_id, vnode_bitmap, @@ -1648,7 +1648,7 @@ impl CatalogController { let mut actor_upstreams = BTreeMap::>::new(); let mut new_actor_dispatchers = vec![]; - if let Some(nodes) = &mut nodes { + if let Some(nodes) = &nodes { visit_stream_node(nodes, |node| { if let PbNodeBody::Merge(node) = node { actor_upstreams @@ -1922,15 +1922,15 @@ impl CatalogController { let mut rate_limits = Vec::new(); for (fragment_id, job_id, fragment_type_mask, stream_node) in fragments { - let mut stream_node = stream_node.to_protobuf(); + let stream_node = stream_node.to_protobuf(); let mut rate_limit = None; let mut node_name = None; - visit_stream_node(&mut stream_node, |node| { + visit_stream_node(&stream_node, |node| { match node { // source rate limit PbNodeBody::Source(node) => { - if let Some(node_inner) = &mut node.source_inner { + if let Some(node_inner) = &node.source_inner { debug_assert!( rate_limit.is_none(), "one fragment should only have 1 rate limit node" @@ -1940,7 +1940,7 @@ impl CatalogController { } } PbNodeBody::StreamFsFetch(node) => { - if let Some(node_inner) = &mut node.node_inner { + if let Some(node_inner) = &node.node_inner { debug_assert!( rate_limit.is_none(), "one fragment should only have 1 rate limit node" diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 4c2eb57fe6cb7..e11dfb5d27117 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -457,9 +457,9 @@ impl StreamJobFragments { /// Panics if not found. pub fn union_fragment_for_table(&mut self) -> &mut Fragment { let mut union_fragment_id = None; - for (fragment_id, fragment) in &mut self.fragments { - for actor in &mut fragment.actors { - if let Some(node) = &mut actor.nodes { + for (fragment_id, fragment) in &self.fragments { + for actor in &fragment.actors { + if let Some(node) = &actor.nodes { visit_stream_node(node, |body| { if let NodeBody::Union(_) = body { if let Some(union_fragment_id) = union_fragment_id.as_mut() { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 5212a73ccd0c8..362e888a90fac 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -752,9 +752,9 @@ impl DdlController { } // check if the union fragment is fully assigned. - for fragment in stream_job_fragments.fragments.values_mut() { - for actor in &mut fragment.actors { - if let Some(node) = &mut actor.nodes { + for fragment in stream_job_fragments.fragments.values() { + for actor in &fragment.actors { + if let Some(node) = &actor.nodes { visit_stream_node(node, |node| { if let NodeBody::Merge(merge_node) = node { assert!(!merge_node.upstream_actor_id.is_empty(), "All the mergers for the union should have been fully assigned beforehand."); diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 16264becc05d6..f472265c41db7 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -132,7 +132,7 @@ impl BuildingFragment { let fragment_id = fragment.fragment_id; let mut has_job = false; - stream_graph_visitor::visit_fragment(fragment, |node_body| match node_body { + stream_graph_visitor::visit_fragment_mut(fragment, |node_body| match node_body { NodeBody::Materialize(materialize_node) => { materialize_node.table_id = job_id; diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index f80ff57599b9c..84965eb2dda78 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -293,7 +293,7 @@ impl Scheduler { // Vnode count requirements: if a fragment is going to look up an existing table, // it must have the same vnode count as that table. for (&id, fragment) in graph.building_fragments() { - visit_fragment(&mut (*fragment).clone(), |node| { + visit_fragment(fragment, |node| { use risingwave_pb::stream_plan::stream_node::NodeBody; let vnode_count = match node { NodeBody::StreamScan(node) => { From 6bf7184d55906785c1f74c84078b8971135cc8b8 Mon Sep 17 00:00:00 2001 From: Li0k Date: Mon, 27 Jan 2025 11:01:55 +0800 Subject: [PATCH 12/25] fix(compactor): fix full key can concat (#20314) --- src/storage/hummock_sdk/src/lib.rs | 58 +++++++++++++++++++++++++----- 1 file changed, 50 insertions(+), 8 deletions(-) diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index f4da340b0a90f..fdf739fbe8375 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -318,17 +318,19 @@ pub fn full_key_can_concat(ssts: &[SstableInfo]) -> bool { let sst_2 = &ssts[i]; if sst_1.key_range.right_exclusive { - if sst_1 - .key_range - .compare_right_with(&sst_2.key_range.left) - .is_gt() + if KeyComparator::compare_encoded_full_key( + &sst_1.key_range.right, + &sst_2.key_range.left, + ) + .is_gt() { return false; } - } else if sst_1 - .key_range - .compare_right_with(&sst_2.key_range.left) - .is_ge() + } else if KeyComparator::compare_encoded_full_key( + &sst_1.key_range.right, + &sst_2.key_range.left, + ) + .is_ge() { return false; } @@ -440,6 +442,9 @@ pub fn get_object_id_from_path(path: &str) -> HummockSstableObjectId { #[cfg(test)] mod tests { + use bytes::Bytes; + use sstable_info::SstableInfoInner; + use super::*; #[test] @@ -447,4 +452,41 @@ mod tests { let len = HummockSstableObjectId::MAX.to_string().len(); assert_eq!(len, HUMMOCK_SSTABLE_OBJECT_ID_MAX_DECIMAL_LENGTH) } + + #[test] + fn test_full_key_concat() { + let key1 = b"\0\0\0\x08\0\0\0\x0112-3\0\0\0\0\x04\0\x1c\x16l'\xe2\0\0"; + let key2 = b"\0\0\0\x08\0\0\0\x0112-3\0\0\0\0\x04\0\x1c\x16l \x12\0\0"; + + let sst_1 = SstableInfoInner { + key_range: key_range::KeyRange { + left: Bytes::from(key1.to_vec()), + right: Bytes::from(key1.to_vec()), + right_exclusive: false, + }, + ..Default::default() + }; + + let sst_2 = SstableInfoInner { + key_range: key_range::KeyRange { + left: Bytes::from(key2.to_vec()), + right: Bytes::from(key2.to_vec()), + right_exclusive: false, + }, + ..Default::default() + }; + + let sst_3 = SstableInfoInner { + key_range: key_range::KeyRange { + left: Bytes::from(key1.to_vec()), + right: Bytes::from(key2.to_vec()), + right_exclusive: false, + }, + ..Default::default() + }; + + assert!(full_key_can_concat(&[sst_1.clone().into(), sst_2.into()])); + + assert!(!full_key_can_concat(&[sst_1.into(), sst_3.into()])); + } } From ef07a495a07fd38dfbb234a82aa9a2b98506dab4 Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Mon, 27 Jan 2025 13:39:26 +0800 Subject: [PATCH 13/25] fix: Fix the issue of actor migration panic caused by the in-place scale-down (#20316) Signed-off-by: Shanicky Chen --- src/meta/src/barrier/context/recovery.rs | 8 +++++--- src/meta/src/controller/fragment.rs | 16 +++++++++++----- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/meta/src/barrier/context/recovery.rs b/src/meta/src/barrier/context/recovery.rs index c13eaabf90822..2dd75e1bb57c9 100644 --- a/src/meta/src/barrier/context/recovery.rs +++ b/src/meta/src/barrier/context/recovery.rs @@ -427,13 +427,13 @@ impl GlobalBarrierWorkerContextImpl { .collect(); if expired_worker_slots.is_empty() { - debug!("no expired worker slots, skipping."); + info!("no expired worker slots, skipping."); return self.resolve_graph_info(None).await; } - debug!("start migrate actors."); + info!("start migrate actors."); let mut to_migrate_worker_slots = expired_worker_slots.into_iter().rev().collect_vec(); - debug!("got to migrate worker slots {:#?}", to_migrate_worker_slots); + info!("got to migrate worker slots {:#?}", to_migrate_worker_slots); let mut inuse_worker_slots: HashSet<_> = all_inuse_worker_slots .intersection(&active_worker_slots) @@ -535,6 +535,8 @@ impl GlobalBarrierWorkerContextImpl { warn!(?changed, "get worker changed or timed out. Retry migrate"); } + info!("migration plan {:?}", plan); + mgr.catalog_controller.migrate_actors(plan).await?; info!("migrate actors succeed."); diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 4a13155c547aa..b41e0cd39ba01 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -1125,12 +1125,17 @@ impl CatalogController { .insert(*actor_id); } - let expired_workers: HashSet<_> = plan.keys().map(|k| k.worker_id() as WorkerId).collect(); + let expired_or_changed_workers: HashSet<_> = + plan.keys().map(|k| k.worker_id() as WorkerId).collect(); let mut actor_migration_plan = HashMap::new(); for (worker, fragment) in actor_locations { - if expired_workers.contains(&worker) { - for (_, actors) in fragment { + if expired_or_changed_workers.contains(&worker) { + for (fragment_id, actors) in fragment { + debug!( + "worker {} expired or changed, migrating fragment {}", + worker, fragment_id + ); let worker_slot_to_actor: HashMap<_, _> = actors .iter() .enumerate() @@ -1140,8 +1145,9 @@ impl CatalogController { .collect(); for (worker_slot, actor) in worker_slot_to_actor { - actor_migration_plan - .insert(actor, plan[&worker_slot].worker_id() as WorkerId); + if let Some(target) = plan.get(&worker_slot) { + actor_migration_plan.insert(actor, target.worker_id() as WorkerId); + } } } } From cdfdb97871452402b7a74523dbf9c015993aef89 Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 27 Jan 2025 13:40:15 +0800 Subject: [PATCH 14/25] refactor: rename fs executor mods (#20322) Signed-off-by: xxchan --- src/connector/src/source/filesystem/mod.rs | 2 +- src/connector/src/source/filesystem/s3/mod.rs | 4 ++-- src/connector/src/source/mod.rs | 2 +- src/connector/src/with_options.rs | 10 ++++++++-- src/frontend/src/handler/create_source.rs | 4 ++-- src/frontend/src/handler/create_source/validate.rs | 6 +++--- .../{fetch_executor.rs => fs_fetch_executor.rs} | 0 .../source/{list_executor.rs => fs_list_executor.rs} | 5 +++-- ...urce_executor.rs => legacy_fs_source_executor.rs} | 12 ++++++------ src/stream/src/executor/source/mod.rs | 12 ++++++------ .../src/executor/source/state_table_handler.rs | 4 ++-- src/stream/src/from_proto/source/trad_source.rs | 6 ++---- 12 files changed, 36 insertions(+), 31 deletions(-) rename src/stream/src/executor/source/{fetch_executor.rs => fs_fetch_executor.rs} (100%) rename src/stream/src/executor/source/{list_executor.rs => fs_list_executor.rs} (98%) rename src/stream/src/executor/source/{fs_source_executor.rs => legacy_fs_source_executor.rs} (98%) diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index dd6f5c82ef715..22f2629be2d4b 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -13,7 +13,7 @@ // limitations under the License. pub use opendal_source::GcsProperties; -pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; +pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, LEGACY_S3_CONNECTOR}; pub mod file_common; pub mod nd_streaming; pub use file_common::{FsPage, FsPageItem, FsSplit, OpendalFsSplit}; diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 76b7e9f1f0ca1..21250c8c81b09 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -25,7 +25,7 @@ use crate::connector_common::AwsAuthProps; use crate::source::filesystem::FsSplit; use crate::source::{SourceProperties, UnknownFields}; -pub const S3_CONNECTOR: &str = "s3"; +pub const LEGACY_S3_CONNECTOR: &str = "s3"; /// These are supported by both `s3` and `s3_v2` (opendal) sources. #[derive(Clone, Debug, Deserialize, PartialEq, with_options::WithOptions)] @@ -69,7 +69,7 @@ impl SourceProperties for S3Properties { type SplitEnumerator = S3SplitEnumerator; type SplitReader = S3FileReader; - const SOURCE_NAME: &'static str = S3_CONNECTOR; + const SOURCE_NAME: &'static str = LEGACY_S3_CONNECTOR; } impl UnknownFields for S3Properties { diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 12b843dcf749f..5585dc5378ae5 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -83,7 +83,7 @@ pub use util::fill_adaptive_split; pub use crate::source::filesystem::opendal_source::{ AZBLOB_CONNECTOR, GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, }; -pub use crate::source::filesystem::S3_CONNECTOR; +pub use crate::source::filesystem::LEGACY_S3_CONNECTOR; pub use crate::source::nexmark::NEXMARK_CONNECTOR; pub use crate::source::pulsar::PULSAR_CONNECTOR; diff --git a/src/connector/src/with_options.rs b/src/connector/src/with_options.rs index 1e84c3012184c..ad72d370254b0 100644 --- a/src/connector/src/with_options.rs +++ b/src/connector/src/with_options.rs @@ -21,8 +21,8 @@ use crate::source::cdc::external::CdcTableType; use crate::source::cdc::MYSQL_CDC_CONNECTOR; use crate::source::iceberg::ICEBERG_CONNECTOR; use crate::source::{ - AZBLOB_CONNECTOR, GCS_CONNECTOR, KAFKA_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, - UPSTREAM_SOURCE_KEY, + AZBLOB_CONNECTOR, GCS_CONNECTOR, KAFKA_CONNECTOR, LEGACY_S3_CONNECTOR, OPENDAL_S3_CONNECTOR, + POSIX_FS_CONNECTOR, UPSTREAM_SOURCE_KEY, }; /// Marker trait for `WITH` options. Only for `#[derive(WithOptions)]`, should not be used manually. @@ -153,6 +153,12 @@ pub trait WithPropertiesExt: Get + Sized { !self.is_iceberg_connector() } + fn is_legacy_fs_connector(&self) -> bool { + self.get(UPSTREAM_SOURCE_KEY) + .map(|s| s.eq_ignore_ascii_case(LEGACY_S3_CONNECTOR)) + .unwrap_or(false) + } + fn is_new_fs_connector(&self) -> bool { self.get(UPSTREAM_SOURCE_KEY) .map(|s| { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 731db3f9f3363..f8cec19c9313a 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -60,8 +60,8 @@ use risingwave_connector::source::nexmark::source::{get_event_data_types_with_na use risingwave_connector::source::test_source::TEST_CONNECTOR; use risingwave_connector::source::{ ConnectorProperties, AZBLOB_CONNECTOR, GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, - KINESIS_CONNECTOR, MQTT_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR, - POSIX_FS_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, + KINESIS_CONNECTOR, LEGACY_S3_CONNECTOR, MQTT_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, + OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, PULSAR_CONNECTOR, }; pub use risingwave_connector::source::{UPSTREAM_SOURCE_KEY, WEBHOOK_CONNECTOR}; use risingwave_connector::WithPropertiesExt; diff --git a/src/frontend/src/handler/create_source/validate.rs b/src/frontend/src/handler/create_source/validate.rs index 605c57f1f3736..d20f6656ebead 100644 --- a/src/frontend/src/handler/create_source/validate.rs +++ b/src/frontend/src/handler/create_source/validate.rs @@ -71,7 +71,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Native], Format::Plain => vec![Encode::Bytes, Encode::Json], ), - S3_CONNECTOR => hashmap!( + LEGACY_S3_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Csv, Encode::Json], ), OPENDAL_S3_CONNECTOR => hashmap!( @@ -143,10 +143,10 @@ pub fn validate_compatibility( // reject s3_v2 creation return Err(RwError::from(Deprecated( OPENDAL_S3_CONNECTOR.to_owned(), - S3_CONNECTOR.to_owned(), + LEGACY_S3_CONNECTOR.to_owned(), ))); } - if connector == S3_CONNECTOR { + if connector == LEGACY_S3_CONNECTOR { // S3 connector is deprecated, use OPENDAL_S3_CONNECTOR instead // do s3 -> s3_v2 migration let entry = props.get_mut(UPSTREAM_SOURCE_KEY).unwrap(); diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fs_fetch_executor.rs similarity index 100% rename from src/stream/src/executor/source/fetch_executor.rs rename to src/stream/src/executor/source/fs_fetch_executor.rs diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/fs_list_executor.rs similarity index 98% rename from src/stream/src/executor/source/list_executor.rs rename to src/stream/src/executor/source/fs_list_executor.rs index 97f3ba12aa9fb..b418300588bcc 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/fs_list_executor.rs @@ -26,7 +26,6 @@ use super::{barrier_to_message_stream, StreamSourceCore}; use crate::executor::prelude::*; use crate::executor::stream_reader::StreamReaderWithPause; -#[allow(dead_code)] pub struct FsListExecutor { actor_ctx: ActorContextRef, @@ -34,15 +33,18 @@ pub struct FsListExecutor { stream_source_core: Option>, /// Metrics for monitor. + #[expect(dead_code)] metrics: Arc, /// Receiver of barrier channel. barrier_receiver: Option>, /// System parameter reader to read barrier interval + #[expect(dead_code)] system_params: SystemParamsReaderRef, /// Rate limit in rows/s. + #[expect(dead_code)] rate_limit_rps: Option, } @@ -65,7 +67,6 @@ impl FsListExecutor { } } - #[allow(clippy::disallowed_types)] fn build_chunked_paginate_stream( &self, source_desc: &SourceDesc, diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/legacy_fs_source_executor.rs similarity index 98% rename from src/stream/src/executor/source/fs_source_executor.rs rename to src/stream/src/executor/source/legacy_fs_source_executor.rs index 2e62865ecff99..5ac97261677a6 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/legacy_fs_source_executor.rs @@ -47,9 +47,9 @@ use crate::executor::UpdateMutation; /// some latencies in network and cost in meta. const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; -/// [`FsSourceExecutor`] is a streaming source, fir external file systems +/// [`LegacyFsSourceExecutor`] is a streaming source, fir external file systems /// such as s3. -pub struct FsSourceExecutor { +pub struct LegacyFsSourceExecutor { actor_ctx: ActorContextRef, /// Streaming source for external @@ -68,7 +68,7 @@ pub struct FsSourceExecutor { rate_limit_rps: Option, } -impl FsSourceExecutor { +impl LegacyFsSourceExecutor { pub fn new( actor_ctx: ActorContextRef, stream_source_core: StreamSourceCore, @@ -506,15 +506,15 @@ impl FsSourceExecutor { } } -impl Execute for FsSourceExecutor { +impl Execute for LegacyFsSourceExecutor { fn execute(self: Box) -> BoxedMessageStream { self.into_stream().boxed() } } -impl Debug for FsSourceExecutor { +impl Debug for LegacyFsSourceExecutor { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("FsSourceExecutor") + f.debug_struct("LegacyFsSourceExecutor") .field("source_id", &self.stream_source_core.source_id) .field("column_ids", &self.stream_source_core.column_ids) .finish() diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 6aabca713c7c2..94a8ab623ab1e 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -30,17 +30,17 @@ pub use state_table_handler::*; mod executor_core; pub use executor_core::StreamSourceCore; -mod fs_source_executor; +mod legacy_fs_source_executor; #[expect(deprecated)] -pub use fs_source_executor::*; +pub use legacy_fs_source_executor::*; mod source_executor; pub use source_executor::*; mod source_backfill_executor; pub use source_backfill_executor::*; -mod list_executor; -pub use list_executor::*; -mod fetch_executor; -pub use fetch_executor::*; +mod fs_list_executor; +pub use fs_list_executor::*; +mod fs_fetch_executor; +pub use fs_fetch_executor::*; mod source_backfill_state_table; pub use source_backfill_state_table::BackfillStateTableHandler; diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index 02a07255d7588..d03a978f6b8b3 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -86,7 +86,7 @@ impl SourceStateTableHandler { .map_err(StreamExecutorError::from) } - /// this method should only be used by [`FsSourceExecutor`](super::FsSourceExecutor) + /// this method should only be used by [`LegacyFsSourceExecutor`](super::LegacyFsSourceExecutor) pub(crate) async fn get_all_completed(&self) -> StreamExecutorResult> { let start = Bound::Excluded(row::once(Some(Self::string_to_scalar( COMPLETE_SPLIT_PREFIX, @@ -137,7 +137,7 @@ impl SourceStateTableHandler { } /// set all complete - /// can only used by [`FsSourceExecutor`](super::FsSourceExecutor) + /// can only used by [`LegacyFsSourceExecutor`](super::LegacyFsSourceExecutor) pub(crate) async fn set_all_complete( &mut self, states: Vec, diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 42cf0e1c1da23..89ec19b84f4b6 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -34,7 +34,6 @@ use crate::executor::source::{ }; use crate::executor::TroublemakerExecutor; -const FS_CONNECTORS: &[&str] = &["s3"]; pub struct SourceExecutorBuilder; pub fn create_source_desc_builder( @@ -197,13 +196,12 @@ impl ExecutorBuilder for SourceExecutorBuilder { state_table_handler, ); - let connector = get_connector_name(&source.with_properties); - let is_fs_connector = FS_CONNECTORS.contains(&connector.as_str()); + let is_fs_connector = source.with_properties.is_legacy_fs_connector(); let is_fs_v2_connector = source.with_properties.is_new_fs_connector(); if is_fs_connector { #[expect(deprecated)] - crate::executor::source::FsSourceExecutor::new( + crate::executor::source::LegacyFsSourceExecutor::new( params.actor_context.clone(), stream_source_core, params.executor_stats, From 18092eebb7adb672ae7f6aee64bdd8754f18509c Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 27 Jan 2025 16:39:58 +0800 Subject: [PATCH 15/25] refactor: more rename for legacy fs source (#20323) --- src/connector/src/macros.rs | 2 +- src/connector/src/source/base.rs | 13 ---- .../src/source/filesystem/file_common.rs | 10 +-- src/connector/src/source/filesystem/mod.rs | 7 +- .../src/source/filesystem/s3/enumerator.rs | 73 ++++++++++++++++--- src/connector/src/source/filesystem/s3/mod.rs | 24 +++--- .../src/source/filesystem/s3/source/mod.rs | 2 +- .../src/source/filesystem/s3/source/reader.rs | 41 ++++++----- .../src/source/filesystem/s3_v2/lister.rs | 67 ----------------- .../src/source/filesystem/s3_v2/mod.rs | 15 ---- src/connector/src/source/mod.rs | 2 +- src/connector/src/source/reader/desc.rs | 17 +++-- src/connector/src/source/reader/fs_reader.rs | 5 +- src/connector/with_options_source.yaml | 54 +++++++------- .../source/legacy_fs_source_executor.rs | 10 +-- 15 files changed, 151 insertions(+), 191 deletions(-) delete mode 100644 src/connector/src/source/filesystem/s3_v2/lister.rs delete mode 100644 src/connector/src/source/filesystem/s3_v2/mod.rs diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index ac498839ed67c..186ecd314c5fc 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -35,7 +35,7 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Mqtt, $crate::source::mqtt::MqttProperties, $crate::source::mqtt::split::MqttSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, - { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, + { S3, $crate::source::filesystem::LegacyS3Properties, $crate::source::filesystem::LegacyFsSplit }, { Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalGcs> }, { OpendalS3, $crate::source::filesystem::opendal_source::OpendalS3Properties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalS3> }, { PosixFs, $crate::source::filesystem::opendal_source::PosixFsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalPosixFs> }, diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 406d678a65ee6..3b7c01c185c18 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use anyhow::anyhow; use async_trait::async_trait; -use aws_sdk_s3::types::Object; use bytes::Bytes; use enum_as_inner::EnumAsInner; use futures::future::try_join_all; @@ -49,7 +48,6 @@ use super::{AZBLOB_CONNECTOR, GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONN use crate::error::ConnectorResult as Result; use crate::parser::schema_change::SchemaChangeEnvelope; use crate::parser::ParserConfig; -use crate::source::filesystem::FsPageItem; use crate::source::monitor::EnumeratorMetrics; use crate::source::SplitImpl::{CitusCdc, MongodbCdc, MysqlCdc, PostgresCdc, SqlServerCdc}; use crate::with_options::WithOptions; @@ -824,17 +822,6 @@ pub trait SplitMetaData: Sized { /// [`None`] and the created source stream will be a pending stream. pub type ConnectorState = Option>; -#[derive(Debug, Clone, Default)] -pub struct FsFilterCtrlCtx; -pub type FsFilterCtrlCtxRef = Arc; - -#[async_trait] -pub trait FsListInner: Sized { - // fixme: better to implement as an Iterator, but the last page still have some contents - async fn get_next_page From<&'a Object>>(&mut self) -> Result<(Vec, bool)>; - fn filter_policy(&self, ctx: &FsFilterCtrlCtx, page_num: usize, item: &FsPageItem) -> bool; -} - #[cfg(test)] mod tests { use maplit::*; diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index f89758a0ef51c..50f56ea1f53fe 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -24,16 +24,16 @@ use super::opendal_source::OpendalSource; use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; -/// [`FsSplit`] Describes a file or a split of a file. A file is a generic concept, +/// [`LegacyFsSplit`] Describes a file or a split of a file. A file is a generic concept, /// and can be a local file, a distributed file system, or am object in S3 bucket. #[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct FsSplit { +pub struct LegacyFsSplit { pub name: String, pub offset: usize, pub size: usize, } -impl From<&Object> for FsSplit { +impl From<&Object> for LegacyFsSplit { fn from(value: &Object) -> Self { Self { name: value.key().unwrap().to_owned(), @@ -43,7 +43,7 @@ impl From<&Object> for FsSplit { } } -impl SplitMetaData for FsSplit { +impl SplitMetaData for LegacyFsSplit { fn id(&self) -> SplitId { self.name.as_str().into() } @@ -63,7 +63,7 @@ impl SplitMetaData for FsSplit { } } -impl FsSplit { +impl LegacyFsSplit { pub fn new(name: String, start: usize, size: usize) -> Self { Self { name, diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index 22f2629be2d4b..b2f4e091cef2b 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -13,10 +13,11 @@ // limitations under the License. pub use opendal_source::GcsProperties; -pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, LEGACY_S3_CONNECTOR}; +pub use s3::{ + LegacyS3FileReader, LegacyS3Properties, LegacyS3SplitEnumerator, LEGACY_S3_CONNECTOR, +}; pub mod file_common; pub mod nd_streaming; -pub use file_common::{FsPage, FsPageItem, FsSplit, OpendalFsSplit}; +pub use file_common::{FsPage, FsPageItem, LegacyFsSplit, OpendalFsSplit}; pub mod opendal_source; mod s3; -pub mod s3_v2; diff --git a/src/connector/src/source/filesystem/s3/enumerator.rs b/src/connector/src/source/filesystem/s3/enumerator.rs index 65eb551afd183..79c55a6a0282d 100644 --- a/src/connector/src/source/filesystem/s3/enumerator.rs +++ b/src/connector/src/source/filesystem/s3/enumerator.rs @@ -15,12 +15,14 @@ use anyhow::Context; use async_trait::async_trait; use aws_sdk_s3::client::Client; +use itertools::Itertools; use crate::aws_utils::{default_conn_config, s3_client}; use crate::connector_common::AwsAuthProps; -use crate::source::filesystem::file_common::FsSplit; -use crate::source::filesystem::s3::S3Properties; -use crate::source::{FsListInner, SourceEnumeratorContextRef, SplitEnumerator}; +use crate::error::ConnectorResult; +use crate::source::filesystem::file_common::LegacyFsSplit; +use crate::source::filesystem::s3::LegacyS3Properties; +use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; /// Get the prefix from a glob pub fn get_prefix(glob: &str) -> String { @@ -56,7 +58,7 @@ pub fn get_prefix(glob: &str) -> String { } #[derive(Debug, Clone)] -pub struct S3SplitEnumerator { +pub struct LegacyS3SplitEnumerator { pub(crate) bucket_name: String, // prefix is used to reduce the number of objects to be listed pub(crate) prefix: Option, @@ -68,9 +70,9 @@ pub struct S3SplitEnumerator { } #[async_trait] -impl SplitEnumerator for S3SplitEnumerator { - type Properties = S3Properties; - type Split = FsSplit; +impl SplitEnumerator for LegacyS3SplitEnumerator { + type Properties = LegacyS3Properties; + type Split = LegacyFsSplit; async fn new( properties: Self::Properties, @@ -89,7 +91,7 @@ impl SplitEnumerator for S3SplitEnumerator { (None, None) }; - Ok(S3SplitEnumerator { + Ok(LegacyS3SplitEnumerator { bucket_name: properties.bucket_name, matcher, prefix, @@ -100,9 +102,9 @@ impl SplitEnumerator for S3SplitEnumerator { async fn list_splits(&mut self) -> crate::error::ConnectorResult> { // fetch one page as validation, no need to get all pages - let (_, _) = self.get_next_page::().await?; + let (_, _) = self.get_next_page::().await?; - Ok(vec![FsSplit { + Ok(vec![LegacyFsSplit { name: "empty_split".to_owned(), offset: 0, size: 0, @@ -110,6 +112,55 @@ impl SplitEnumerator for S3SplitEnumerator { } } +#[async_trait] +pub trait FsListInner: Sized { + // fixme: better to implement as an Iterator, but the last page still have some contents + async fn get_next_page From<&'a aws_sdk_s3::types::Object>>( + &mut self, + ) -> ConnectorResult<(Vec, bool)>; +} + +#[async_trait] +impl FsListInner for LegacyS3SplitEnumerator { + async fn get_next_page From<&'a aws_sdk_s3::types::Object>>( + &mut self, + ) -> ConnectorResult<(Vec, bool)> { + let mut has_finished = false; + let mut req = self + .client + .list_objects_v2() + .bucket(&self.bucket_name) + .set_prefix(self.prefix.clone()); + if let Some(continuation_token) = self.next_continuation_token.take() { + req = req.continuation_token(continuation_token); + } + let mut res = req + .send() + .await + .with_context(|| format!("failed to list objects in bucket `{}`", self.bucket_name))?; + if res.is_truncated().unwrap_or_default() { + self.next_continuation_token + .clone_from(&res.next_continuation_token); + } else { + has_finished = true; + self.next_continuation_token = None; + } + let objects = res.contents.take().unwrap_or_default(); + let matched_objs: Vec = objects + .iter() + .filter(|obj| obj.key().is_some()) + .filter(|obj| { + self.matcher + .as_ref() + .map(|m| m.matches(obj.key().unwrap())) + .unwrap_or(true) + }) + .map(T::from) + .collect_vec(); + Ok((matched_objs, has_finished)) + } +} + #[cfg(test)] mod tests { use itertools::Itertools; @@ -141,7 +192,7 @@ mod tests { compression_format: CompressionFormat::None, }; let mut enumerator = - S3SplitEnumerator::new(props.into(), SourceEnumeratorContext::dummy().into()) + LegacyS3SplitEnumerator::new(props.into(), SourceEnumeratorContext::dummy().into()) .await .unwrap(); let splits = enumerator.list_splits().await.unwrap(); diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 21250c8c81b09..0e7960789b2c3 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -14,15 +14,15 @@ pub mod enumerator; use std::collections::HashMap; -pub use enumerator::S3SplitEnumerator; +pub use enumerator::LegacyS3SplitEnumerator; use crate::source::filesystem::file_common::CompressionFormat; mod source; use serde::Deserialize; -pub use source::S3FileReader; +pub use source::LegacyS3FileReader; use crate::connector_common::AwsAuthProps; -use crate::source::filesystem::FsSplit; +use crate::source::filesystem::LegacyFsSplit; use crate::source::{SourceProperties, UnknownFields}; pub const LEGACY_S3_CONNECTOR: &str = "s3"; @@ -47,7 +47,7 @@ pub struct S3PropertiesCommon { } #[derive(Clone, Debug, Deserialize, PartialEq, with_options::WithOptions)] -pub struct S3Properties { +pub struct LegacyS3Properties { #[serde(flatten)] pub common: S3PropertiesCommon, @@ -55,7 +55,7 @@ pub struct S3Properties { pub unknown_fields: HashMap, } -impl From for S3Properties { +impl From for LegacyS3Properties { fn from(common: S3PropertiesCommon) -> Self { Self { common, @@ -64,22 +64,22 @@ impl From for S3Properties { } } -impl SourceProperties for S3Properties { - type Split = FsSplit; - type SplitEnumerator = S3SplitEnumerator; - type SplitReader = S3FileReader; +impl SourceProperties for LegacyS3Properties { + type Split = LegacyFsSplit; + type SplitEnumerator = LegacyS3SplitEnumerator; + type SplitReader = LegacyS3FileReader; const SOURCE_NAME: &'static str = LEGACY_S3_CONNECTOR; } -impl UnknownFields for S3Properties { +impl UnknownFields for LegacyS3Properties { fn unknown_fields(&self) -> HashMap { self.unknown_fields.clone() } } -impl From<&S3Properties> for AwsAuthProps { - fn from(props: &S3Properties) -> Self { +impl From<&LegacyS3Properties> for AwsAuthProps { + fn from(props: &LegacyS3Properties) -> Self { let props = &props.common; Self { region: Some(props.region_name.clone()), diff --git a/src/connector/src/source/filesystem/s3/source/mod.rs b/src/connector/src/source/filesystem/s3/source/mod.rs index e7e6c5db0daed..87e9067537ad7 100644 --- a/src/connector/src/source/filesystem/s3/source/mod.rs +++ b/src/connector/src/source/filesystem/s3/source/mod.rs @@ -14,4 +14,4 @@ mod reader; mod split_stream; -pub use reader::S3FileReader; +pub use reader::LegacyS3FileReader; diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 5f2631b913ab4..eeb613f52c48d 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -36,9 +36,9 @@ use crate::connector_common::AwsAuthProps; use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::base::{SplitMetaData, SplitReader}; -use crate::source::filesystem::file_common::FsSplit; +use crate::source::filesystem::file_common::LegacyFsSplit; use crate::source::filesystem::nd_streaming::need_nd_streaming; -use crate::source::filesystem::s3::S3Properties; +use crate::source::filesystem::s3::LegacyS3Properties; use crate::source::{ into_chunk_stream, BoxSourceChunkStream, Column, SourceContextRef, SourceMessage, SourceMeta, }; @@ -46,22 +46,22 @@ use crate::source::{ const STREAM_READER_CAPACITY: usize = 4096; #[derive(Debug)] -pub struct S3FileReader { +pub struct LegacyS3FileReader { #[expect(dead_code)] split_offset: HashMap, bucket_name: String, s3_client: s3_client::Client, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, } -impl S3FileReader { +impl LegacyS3FileReader { #[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] pub async fn stream_read_object( client_for_s3: s3_client::Client, bucket_name: String, - split: FsSplit, + split: LegacyFsSplit, source_ctx: SourceContextRef, ) { let actor_id = source_ctx.actor_id.to_string(); @@ -73,7 +73,7 @@ impl S3FileReader { let object_name = split.name.clone(); - let byte_stream = match S3FileReader::get_object( + let byte_stream = match LegacyS3FileReader::get_object( &client_for_s3, &bucket_name, &object_name, @@ -171,13 +171,13 @@ impl S3FileReader { } #[async_trait] -impl SplitReader for S3FileReader { - type Properties = S3Properties; - type Split = FsSplit; +impl SplitReader for LegacyS3FileReader { + type Properties = LegacyS3Properties; + type Split = LegacyFsSplit; async fn new( - props: S3Properties, - splits: Vec, + props: LegacyS3Properties, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -189,7 +189,7 @@ impl SplitReader for S3FileReader { let bucket_name = props.common.bucket_name; let s3_client = s3_client(&sdk_config, Some(default_conn_config())); - let s3_file_reader = S3FileReader { + let s3_file_reader = LegacyS3FileReader { split_offset: HashMap::new(), bucket_name, s3_client, @@ -206,7 +206,7 @@ impl SplitReader for S3FileReader { } } -impl S3FileReader { +impl LegacyS3FileReader { #[try_stream(boxed, ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_stream_inner(self) { for split in self.splits { @@ -248,7 +248,7 @@ mod tests { }; use crate::source::filesystem::file_common::CompressionFormat; use crate::source::filesystem::s3::S3PropertiesCommon; - use crate::source::filesystem::S3SplitEnumerator; + use crate::source::filesystem::LegacyS3SplitEnumerator; use crate::source::{ SourceColumnDesc, SourceContext, SourceEnumeratorContext, SplitEnumerator, }; @@ -256,7 +256,7 @@ mod tests { #[tokio::test] #[ignore] async fn test_s3_split_reader() { - let props: S3Properties = S3PropertiesCommon { + let props: LegacyS3Properties = S3PropertiesCommon { region_name: "ap-southeast-1".to_owned(), bucket_name: "mingchao-s3-source".to_owned(), match_pattern: None, @@ -267,7 +267,7 @@ mod tests { } .into(); let mut enumerator = - S3SplitEnumerator::new(props.clone(), SourceEnumeratorContext::dummy().into()) + LegacyS3SplitEnumerator::new(props.clone(), SourceEnumeratorContext::dummy().into()) .await .unwrap(); let splits = enumerator.list_splits().await.unwrap(); @@ -292,9 +292,10 @@ mod tests { }, }; - let reader = S3FileReader::new(props, splits, config, SourceContext::dummy().into(), None) - .await - .unwrap(); + let reader = + LegacyS3FileReader::new(props, splits, config, SourceContext::dummy().into(), None) + .await + .unwrap(); let msg_stream = reader.into_stream_inner(); #[for_await] diff --git a/src/connector/src/source/filesystem/s3_v2/lister.rs b/src/connector/src/source/filesystem/s3_v2/lister.rs deleted file mode 100644 index ee0132742a40d..0000000000000 --- a/src/connector/src/source/filesystem/s3_v2/lister.rs +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright 2025 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 async_trait::async_trait; -use aws_sdk_s3::types::Object; -use itertools::Itertools; - -use crate::error::ConnectorResult; -use crate::source::filesystem::{FsPageItem, S3SplitEnumerator}; -use crate::source::{FsFilterCtrlCtx, FsListInner}; - -#[async_trait] -impl FsListInner for S3SplitEnumerator { - async fn get_next_page From<&'a Object>>( - &mut self, - ) -> ConnectorResult<(Vec, bool)> { - let mut has_finished = false; - let mut req = self - .client - .list_objects_v2() - .bucket(&self.bucket_name) - .set_prefix(self.prefix.clone()); - if let Some(continuation_token) = self.next_continuation_token.take() { - req = req.continuation_token(continuation_token); - } - let mut res = req - .send() - .await - .with_context(|| format!("failed to list objects in bucket `{}`", self.bucket_name))?; - if res.is_truncated().unwrap_or_default() { - self.next_continuation_token - .clone_from(&res.next_continuation_token); - } else { - has_finished = true; - self.next_continuation_token = None; - } - let objects = res.contents.take().unwrap_or_default(); - let matched_objs: Vec = objects - .iter() - .filter(|obj| obj.key().is_some()) - .filter(|obj| { - self.matcher - .as_ref() - .map(|m| m.matches(obj.key().unwrap())) - .unwrap_or(true) - }) - .map(T::from) - .collect_vec(); - Ok((matched_objs, has_finished)) - } - - fn filter_policy(&self, _ctx: &FsFilterCtrlCtx, _page_num: usize, _item: &FsPageItem) -> bool { - true - } -} diff --git a/src/connector/src/source/filesystem/s3_v2/mod.rs b/src/connector/src/source/filesystem/s3_v2/mod.rs deleted file mode 100644 index ae95f6fedb186..0000000000000 --- a/src/connector/src/source/filesystem/s3_v2/mod.rs +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright 2025 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. - -pub mod lister; diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 5585dc5378ae5..3ea03f7b4df19 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -16,7 +16,7 @@ pub mod prelude { // import all split enumerators pub use crate::source::datagen::DatagenSplitEnumerator; pub use crate::source::filesystem::opendal_source::OpendalEnumerator; - pub use crate::source::filesystem::S3SplitEnumerator; + pub use crate::source::filesystem::LegacyS3SplitEnumerator; pub use crate::source::google_pubsub::PubsubSplitEnumerator as GooglePubsubSplitEnumerator; pub use crate::source::iceberg::IcebergSplitEnumerator; pub use crate::source::kafka::KafkaSplitEnumerator; diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index 95fdf24b111aa..ed4692a2db2a5 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -21,7 +21,7 @@ use risingwave_pb::catalog::PbStreamSourceInfo; use risingwave_pb::plan_common::PbColumnCatalog; #[expect(deprecated)] -use super::fs_reader::FsSourceReader; +use super::fs_reader::LegacyFsSourceReader; use super::reader::SourceReader; use crate::error::ConnectorResult; use crate::parser::additional_columns::source_add_partition_offset_cols; @@ -44,8 +44,8 @@ pub struct SourceDesc { #[deprecated = "will be replaced by new fs source (list + fetch)"] #[expect(deprecated)] #[derive(Debug)] -pub struct FsSourceDesc { - pub source: FsSourceReader, +pub struct LegacyFsSourceDesc { + pub source: LegacyFsSourceReader, pub columns: Vec, pub metrics: Arc, } @@ -139,7 +139,7 @@ impl SourceDescBuilder { #[deprecated = "will be replaced by new fs source (list + fetch)"] #[expect(deprecated)] - pub fn build_fs_source_desc(&self) -> ConnectorResult { + pub fn build_fs_source_desc(&self) -> ConnectorResult { let parser_config = SpecificParserConfig::new(&self.source_info, &self.with_properties)?; match ( @@ -161,10 +161,13 @@ impl SourceDescBuilder { let columns = self.column_catalogs_to_source_column_descs(); - let source = - FsSourceReader::new(self.with_properties.clone(), columns.clone(), parser_config)?; + let source = LegacyFsSourceReader::new( + self.with_properties.clone(), + columns.clone(), + parser_config, + )?; - Ok(FsSourceDesc { + Ok(LegacyFsSourceDesc { source, columns, metrics: self.metrics.clone(), diff --git a/src/connector/src/source/reader/fs_reader.rs b/src/connector/src/source/reader/fs_reader.rs index 41817c3a85a97..6a2ea3223e0a5 100644 --- a/src/connector/src/source/reader/fs_reader.rs +++ b/src/connector/src/source/reader/fs_reader.rs @@ -29,15 +29,14 @@ use crate::source::{ use crate::WithOptionsSecResolved; #[derive(Clone, Debug)] -pub struct FsSourceReader { +pub struct LegacyFsSourceReader { pub config: ConnectorProperties, pub columns: Vec, pub properties: WithOptionsSecResolved, pub parser_config: SpecificParserConfig, } -impl FsSourceReader { - #[allow(clippy::too_many_arguments)] +impl LegacyFsSourceReader { pub fn new( properties: WithOptionsSecResolved, columns: Vec, diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 4cb45b983f0fa..5b6545e7086f8 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -473,6 +473,33 @@ KinesisProperties: required: false alias: - kinesis.assumerole.external_id +LegacyS3Properties: + fields: + - name: s3.region_name + field_type: String + required: true + - name: s3.bucket_name + field_type: String + required: true + - name: match_pattern + field_type: String + required: false + default: Default::default + - name: s3.credentials.access + field_type: String + required: false + default: Default::default + - name: s3.credentials.secret + field_type: String + required: false + default: Default::default + - name: s3.endpoint_url + field_type: String + required: false + - name: compression_format + field_type: CompressionFormat + required: false + default: Default::default MongodbCommon: fields: - name: mongodb.url @@ -1065,33 +1092,6 @@ PulsarProperties: contains a generated suffix in the subscription name. The subscription name will be `{subscription_name_prefix}-{fragment_id}-{actor_id}`. required: false -S3Properties: - fields: - - name: s3.region_name - field_type: String - required: true - - name: s3.bucket_name - field_type: String - required: true - - name: match_pattern - field_type: String - required: false - default: Default::default - - name: s3.credentials.access - field_type: String - required: false - default: Default::default - - name: s3.credentials.secret - field_type: String - required: false - default: Default::default - - name: s3.endpoint_url - field_type: String - required: false - - name: compression_format - field_type: CompressionFormat - required: false - default: Default::default TestSourceProperties: fields: - name: properties diff --git a/src/stream/src/executor/source/legacy_fs_source_executor.rs b/src/stream/src/executor/source/legacy_fs_source_executor.rs index 5ac97261677a6..8774d5c468b7f 100644 --- a/src/stream/src/executor/source/legacy_fs_source_executor.rs +++ b/src/stream/src/executor/source/legacy_fs_source_executor.rs @@ -25,7 +25,7 @@ use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::epoch::EpochPair; use risingwave_connector::error::ConnectorError; -use risingwave_connector::source::reader::desc::{FsSourceDesc, SourceDescBuilder}; +use risingwave_connector::source::reader::desc::{LegacyFsSourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ BoxSourceChunkStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, SplitMetaData, @@ -89,7 +89,7 @@ impl LegacyFsSourceExecutor { async fn build_stream_source_reader( &mut self, - source_desc: &FsSourceDesc, + source_desc: &LegacyFsSourceDesc, state: ConnectorState, ) -> StreamExecutorResult { let column_ids = source_desc @@ -121,7 +121,7 @@ impl LegacyFsSourceExecutor { async fn rebuild_stream_reader( &mut self, - source_desc: &FsSourceDesc, + source_desc: &LegacyFsSourceDesc, stream: &mut StreamReaderWithPause, ) -> StreamExecutorResult<()> { let target_state: Vec = self @@ -141,7 +141,7 @@ impl LegacyFsSourceExecutor { async fn apply_split_change( &mut self, - source_desc: &FsSourceDesc, + source_desc: &LegacyFsSourceDesc, stream: &mut StreamReaderWithPause, mapping: &HashMap>, ) -> StreamExecutorResult<()> { @@ -207,7 +207,7 @@ impl LegacyFsSourceExecutor { async fn replace_stream_reader_with_target_state( &mut self, - source_desc: &FsSourceDesc, + source_desc: &LegacyFsSourceDesc, stream: &mut StreamReaderWithPause, target_state: Vec, ) -> StreamExecutorResult<()> { From 0f5bae08111dcb4fd3912c04476889d5fe564d8e Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 27 Jan 2025 17:53:05 +0800 Subject: [PATCH 16/25] feat(license): invalidate license when cpu exceeds limit instead of rejecting new compute nodes from joining (#20276) Signed-off-by: Bugen Zhao --- .typos.toml | 2 +- proto/meta.proto | 2 + .../common_service/src/observer_manager.rs | 1 + src/compute/src/observer/observer_manager.rs | 5 + src/frontend/src/observer/observer_manager.rs | 6 ++ src/license/src/cpu.rs | 66 +++--------- src/license/src/feature.rs | 34 +++--- src/license/src/lib.rs | 2 + src/license/src/manager.rs | 47 ++++++-- src/meta/service/src/notification_service.rs | 17 +++ src/meta/src/controller/cluster.rs | 101 +++++++++--------- src/meta/src/manager/notification.rs | 11 ++ .../compactor_observer/observer_manager.rs | 5 + src/storage/src/hummock/observer_manager.rs | 6 ++ .../integration_tests/license_cpu_limit.rs | 89 +++++++++++++++ .../tests/integration_tests/main.rs | 6 +- 16 files changed, 269 insertions(+), 131 deletions(-) create mode 100644 src/tests/simulation/tests/integration_tests/license_cpu_limit.rs diff --git a/.typos.toml b/.typos.toml index 4b6af71b5ddf6..001a8f9d276ec 100644 --- a/.typos.toml +++ b/.typos.toml @@ -33,7 +33,7 @@ extend-exclude = [ "src/sqlparser/tests/testdata/", "src/frontend/planner_test/tests/testdata", "src/tests/sqlsmith/tests/freeze", - "src/license/src/manager.rs", + "src/license/**/*.rs", # JWT license key "Cargo.lock", "**/Cargo.toml", "**/go.mod", diff --git a/proto/meta.proto b/proto/meta.proto index 975798de90da9..d8351060d72a5 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -471,6 +471,7 @@ message MetaSnapshot { reserved "parallel_unit_mappings"; GetSessionParamsResponse session_params = 20; repeated catalog.Secret secrets = 23; + uint64 compute_node_total_cpu_count = 24; repeated common.WorkerNode nodes = 10; hummock.HummockVersion hummock_version = 12; backup_service.MetaBackupManifestId meta_backup_manifest_id = 14; @@ -540,6 +541,7 @@ message SubscribeResponse { FragmentWorkerSlotMapping streaming_worker_slot_mapping = 27; FragmentWorkerSlotMappings serving_worker_slot_mappings = 28; catalog.Secret secret = 29; + uint64 compute_node_total_cpu_count = 30; } reserved 12; reserved "parallel_unit_mapping"; diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index c1c62abd33443..b56215fe163b8 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -122,6 +122,7 @@ where Info::Snapshot(_) | Info::HummockWriteLimits(_) => unreachable!(), Info::HummockStats(_) => true, Info::Recovery(_) => true, + Info::ComputeNodeTotalCpuCount(_) => true, Info::StreamingWorkerSlotMapping(_) => { notification.version > info diff --git a/src/compute/src/observer/observer_manager.rs b/src/compute/src/observer/observer_manager.rs index b21e01bf3c3f2..19c273f9bd4f2 100644 --- a/src/compute/src/observer/observer_manager.rs +++ b/src/compute/src/observer/observer_manager.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::license::LicenseManager; use risingwave_common::secret::LocalSecretManager; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::ObserverState; @@ -45,6 +46,9 @@ impl ObserverState for ComputeObserverNode { panic!("error type notification"); } }, + Info::ComputeNodeTotalCpuCount(count) => { + LicenseManager::get().update_cpu_core_count(count as _); + } _ => { panic!("error type notification"); } @@ -57,6 +61,7 @@ impl ObserverState for ComputeObserverNode { unreachable!(); }; LocalSecretManager::global().init_secrets(snapshot.secrets); + LicenseManager::get().update_cpu_core_count(snapshot.compute_node_total_cpu_count as _); } } diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index faff51aec1a82..299120bbf17ee 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::WorkerSlotMapping; +use risingwave_common::license::LicenseManager; use risingwave_common::secret::LocalSecretManager; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; @@ -114,6 +115,9 @@ impl ObserverState for FrontendObserverNode { Info::Recovery(_) => { self.compute_client_pool.invalidate_all(); } + Info::ComputeNodeTotalCpuCount(count) => { + LicenseManager::get().update_cpu_core_count(count as _); + } } } @@ -147,6 +151,7 @@ impl ObserverState for FrontendObserverNode { session_params, version, secrets, + compute_node_total_cpu_count, } = snapshot; for db in databases { @@ -208,6 +213,7 @@ impl ObserverState for FrontendObserverNode { *self.session_params.write() = serde_json::from_str(&session_params.unwrap().params).unwrap(); LocalSecretManager::global().init_secrets(secrets); + LicenseManager::get().update_cpu_core_count(compute_node_total_cpu_count as _); } } diff --git a/src/license/src/cpu.rs b/src/license/src/cpu.rs index 5dd279f58ec6c..f65687ba7fd5f 100644 --- a/src/license/src/cpu.rs +++ b/src/license/src/cpu.rs @@ -12,41 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::num::NonZeroU64; - -use thiserror::Error; - -use crate::{LicenseKeyError, LicenseManager}; - -/// The error type for CPU core limit exceeded as per the license key. -#[derive(Debug, Clone, Error)] -#[error("invalid license key")] -pub enum CpuCoreLimitExceeded { - #[error("cannot check CPU core limit due to license key error")] - LicenseKeyError(#[from] LicenseKeyError), - - #[error( - "CPU core limit exceeded as per the license key, \ - requesting {actual} while the maximum allowed is {limit}" - )] - Exceeded { limit: NonZeroU64, actual: u64 }, -} - -impl LicenseManager { - /// Check if the given CPU core count exceeds the limit as per the license key. - pub fn check_cpu_core_limit(&self, cpu_core_count: u64) -> Result<(), CpuCoreLimitExceeded> { - let license = self.license()?; - - match license.cpu_core_limit { - Some(limit) if cpu_core_count > limit.get() => Err(CpuCoreLimitExceeded::Exceeded { - limit, - actual: cpu_core_count, - }), - _ => Ok(()), - } - } -} - // Tests below only work in debug mode. #[cfg(debug_assertions)] #[cfg(test)] @@ -54,14 +19,14 @@ mod tests { use expect_test::expect; use thiserror_ext::AsReport as _; - use super::*; - use crate::{LicenseKey, TEST_PAID_LICENSE_KEY_CONTENT}; + use crate::{Feature, LicenseKey, LicenseManager, TEST_PAID_LICENSE_KEY_CONTENT}; - fn do_test(key: &str, cpu_core_count: u64, expect: expect_test::Expect) { + fn do_test(key: &str, cpu_core_count: usize, expect: expect_test::Expect) { let manager = LicenseManager::new(); manager.refresh(LicenseKey(key)); + manager.update_cpu_core_count(cpu_core_count); - match manager.check_cpu_core_limit(cpu_core_count) { + match Feature::TestPaid.check_available_with(&manager) { Ok(_) => expect.assert_eq("ok"), Err(error) => expect.assert_eq(&error.to_report_string()), } @@ -72,28 +37,31 @@ mod tests { do_test(TEST_PAID_LICENSE_KEY_CONTENT, 114514, expect!["ok"]); } - #[test] - fn test_no_license_key_no_limit() { - do_test("", 114514, expect!["ok"]); - } - #[test] fn test_invalid_license_key() { const KEY: &str = "invalid"; - do_test(KEY, 0, expect!["cannot check CPU core limit due to license key error: invalid license key: InvalidToken"]); - do_test(KEY, 114514, expect!["cannot check CPU core limit due to license key error: invalid license key: InvalidToken"]); + do_test( + KEY, + 0, + expect!["feature TestPaid is not available due to license error: invalid license key: InvalidToken"], + ); + do_test( + KEY, + 114514, + expect!["feature TestPaid is not available due to license error: invalid license key: InvalidToken"], + ); } #[test] fn test_limit() { const KEY: &str = "eyJhbGciOiJSUzUxMiIsInR5cCI6IkpXVCJ9.\ - eyJzdWIiOiJmcmVlLXRlc3QtMzIiLCJpc3MiOiJwcm9kLnJpc2luZ3dhdmUuY29tIiwidGllciI6ImZyZWUiLCJleHAiOjE4NTI1NTk5OTksImlhdCI6MTcyMzcwMTk5NCwiY3B1X2NvcmVfbGltaXQiOjMyfQ.\ - rsATtzlduLUkGQeXkOROtyGUpafdDhi18iKdYAzAldWQuO9KevNcnD8a6geCShZSGte65bI7oYtv7GHx8i66ge3B1SVsgGgYr10ebphPUNUQenYoN0mpD4Wn0prPStOgANzYZOI2ntMGAaeWStji1x67_iho6r0W9r6RX3kMvzFSbiObSIfvTdrMULeg-xeHc3bT_ErRhaXq7MAa2Oiq3lcK2sNgEvc9KYSP9YbhSik9CBkc8lcyeVoc48SSWEaBU-c8-Ge0fzjgWHI9KIsUV5Ihe66KEfs0PqdRoSWbgskYGzA3o8wHIbtJbJiPzra373kkFH9MGY0HOsw9QeJLGQ"; + eyJzdWIiOiJwYWlkLXRlc3QtMzIiLCJpc3MiOiJ0ZXN0LnJpc2luZ3dhdmUuY29tIiwidGllciI6InBhaWQiLCJleHAiOjIxNTA0OTU5OTksImlhdCI6MTczNzYxMjQ5NSwiY3B1X2NvcmVfbGltaXQiOjMyfQ.\ + SQpX2Dmon5Mb04VUbHyxsU7owJhcdLZHqUefxAXBwG5AqgKdpfS0XUePW5E4D-EfxtH_cWJiD4QDFsfdRUz88g_n_KvfNUObMW7NV5TUoRs_ImtS4ySugExNX3JzJi71QqgI8kugStQ7uOR9kZ_C-cCc_IG2CwwEmhhW1Ij0vX7qjhG5JNMit_bhxPY7Rh27ppgPTqWxJFTTsw-9B7O5WR_yIlaDjxVzk0ALm_j6DPB249gG3dkeK0rP0AK_ip2cK6iQdy8Cge7ATD6yUh4c_aR6GILDF6-vyB7QdWU6DdQS4KhdkPNWoe_Z9psotcXQJ7NhQ39hk8tdLzmTfGDDBA"; do_test(KEY, 31, expect!["ok"]); do_test(KEY, 32, expect!["ok"]); - do_test(KEY, 33, expect!["CPU core limit exceeded as per the license key, requesting 33 while the maximum allowed is 32"]); + do_test(KEY, 33, expect!["feature TestPaid is not available due to license error: the license key is currently not effective because the CPU core in the cluster (33) exceeds the maximum allowed by the license key (32); consider removing some nodes or acquiring a new license key with a higher limit"]); } } diff --git a/src/license/src/feature.rs b/src/license/src/feature.rs index 5243001634e22..0d10d76869b0f 100644 --- a/src/license/src/feature.rs +++ b/src/license/src/feature.rs @@ -14,7 +14,7 @@ use thiserror::Error; -use super::{report_telemetry, License, LicenseKeyError, LicenseManager, Tier}; +use super::{report_telemetry, LicenseError, LicenseManager, Tier}; /// Define all features that are available based on the tier of the license. /// @@ -113,14 +113,17 @@ pub enum FeatureNotAvailable { #[error("feature {feature:?} is not available due to license error")] LicenseError { feature: Feature, - source: LicenseKeyError, + source: LicenseError, }, } impl Feature { - /// Check whether the feature is available based on the current license. - pub fn check_available(self) -> Result<(), FeatureNotAvailable> { - let check_res = match LicenseManager::get().license() { + /// Check whether the feature is available based on the given license manager. + pub(crate) fn check_available_with( + self, + manager: &LicenseManager, + ) -> Result<(), FeatureNotAvailable> { + let check_res = match manager.license() { Ok(license) => { if license.tier >= self.min_tier() { Ok(()) @@ -131,22 +134,19 @@ impl Feature { }) } } - Err(error) => { - // If there's a license key error, we still try against the default license first - // to see if the feature is available for free. - if License::default().tier >= self.min_tier() { - Ok(()) - } else { - Err(FeatureNotAvailable::LicenseError { - feature: self, - source: error, - }) - } - } + Err(error) => Err(FeatureNotAvailable::LicenseError { + feature: self, + source: error, + }), }; report_telemetry(&self, self.get_feature_name(), check_res.is_ok()); check_res } + + /// Check whether the feature is available based on the current license. + pub fn check_available(self) -> Result<(), FeatureNotAvailable> { + self.check_available_with(LicenseManager::get()) + } } diff --git a/src/license/src/lib.rs b/src/license/src/lib.rs index 56a98f97ba5b2..1f30cc0875727 100644 --- a/src/license/src/lib.rs +++ b/src/license/src/lib.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![feature(let_chains)] + mod cpu; mod feature; mod key; diff --git a/src/license/src/manager.rs b/src/license/src/manager.rs index f8d9d8d42a627..ab176bf1db37e 100644 --- a/src/license/src/manager.rs +++ b/src/license/src/manager.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::num::NonZeroU64; +use std::num::NonZeroUsize; use std::sync::{LazyLock, RwLock}; use jsonwebtoken::{Algorithm, DecodingKey, Validation}; @@ -81,7 +81,7 @@ pub struct License { pub tier: Tier, /// Maximum number of compute-node CPU cores allowed to use. Typically used for the paid tier. - pub cpu_core_limit: Option, + pub cpu_core_limit: Option, /// Expiration time in seconds since UNIX epoch. /// @@ -106,11 +106,21 @@ impl Default for License { /// The error type for invalid license key when verifying as JWT. #[derive(Debug, Clone, Error)] -#[error("invalid license key")] -pub struct LicenseKeyError(#[source] jsonwebtoken::errors::Error); +pub enum LicenseError { + #[error("invalid license key")] + InvalidKey(#[source] jsonwebtoken::errors::Error), + + #[error( + "the license key is currently not effective because the CPU core in the cluster \ + ({actual}) exceeds the maximum allowed by the license key ({limit}); \ + consider removing some nodes or acquiring a new license key with a higher limit" + )] + CpuCoreLimitExceeded { limit: NonZeroUsize, actual: usize }, +} struct Inner { - license: Result, + license: Result, + cached_cpu_core_count: usize, } /// The singleton license manager. @@ -129,6 +139,7 @@ impl LicenseManager { Self { inner: RwLock::new(Inner { license: Ok(License::default()), + cached_cpu_core_count: 0, }), } } @@ -162,7 +173,7 @@ impl LicenseManager { inner.license = match jsonwebtoken::decode(license_key, &PUBLIC_KEY, &validation) { Ok(data) => Ok(data.claims), - Err(error) => Err(LicenseKeyError(error)), + Err(error) => Err(LicenseError::InvalidKey(error)), }; match &inner.license { @@ -171,22 +182,40 @@ impl LicenseManager { } } + /// Update the cached CPU core count. + pub fn update_cpu_core_count(&self, cpu_core_count: usize) { + let mut inner = self.inner.write().unwrap(); + inner.cached_cpu_core_count = cpu_core_count; + } + /// Get the current license if it is valid. /// /// Since the license can expire, the returned license should not be cached by the caller. /// /// Prefer calling [`crate::Feature::check_available`] to check the availability of a feature, /// other than directly calling this method and checking the content of the license. - pub fn license(&self) -> Result { - let license = self.inner.read().unwrap().license.clone()?; + pub fn license(&self) -> Result { + let inner = self.inner.read().unwrap(); + let license = inner.license.clone()?; // Check the expiration time additionally. if license.exp < jsonwebtoken::get_current_timestamp() { - return Err(LicenseKeyError( + return Err(LicenseError::InvalidKey( jsonwebtoken::errors::ErrorKind::ExpiredSignature.into(), )); } + // Check the CPU core limit. + let actual_cpu_core = inner.cached_cpu_core_count; + if let Some(limit) = license.cpu_core_limit + && actual_cpu_core > limit.get() + { + return Err(LicenseError::CpuCoreLimitExceeded { + limit, + actual: actual_cpu_core, + }); + } + Ok(license) } } diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index b766689cc9bf3..91a8c5aae6292 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -207,8 +207,16 @@ impl NotificationServiceImpl { Ok((tables, notification_version)) } + async fn get_compute_node_total_cpu_count(&self) -> usize { + self.metadata_manager + .cluster_controller + .compute_node_total_cpu_count() + .await + } + async fn compactor_subscribe(&self) -> MetaResult { let (tables, catalog_version) = self.get_tables_and_creating_tables_snapshot().await?; + let compute_node_total_cpu_count = self.get_compute_node_total_cpu_count().await; Ok(MetaSnapshot { tables, @@ -216,6 +224,7 @@ impl NotificationServiceImpl { catalog_version, ..Default::default() }), + compute_node_total_cpu_count: compute_node_total_cpu_count as _, ..Default::default() }) } @@ -266,6 +275,8 @@ impl NotificationServiceImpl { .context("failed to encode session params")?, }); + let compute_node_total_cpu_count = self.get_compute_node_total_cpu_count().await; + Ok(MetaSnapshot { databases, schemas, @@ -289,6 +300,7 @@ impl NotificationServiceImpl { serving_worker_slot_mappings, streaming_worker_slot_mappings, session_params, + compute_node_total_cpu_count: compute_node_total_cpu_count as _, ..Default::default() }) } @@ -301,6 +313,7 @@ impl NotificationServiceImpl { .await; let hummock_write_limits = self.hummock_manager.write_limits().await; let meta_backup_manifest_id = self.backup_manager.manifest().manifest_id; + let compute_node_total_cpu_count = self.get_compute_node_total_cpu_count().await; Ok(MetaSnapshot { tables, @@ -315,18 +328,22 @@ impl NotificationServiceImpl { hummock_write_limits: Some(WriteLimits { write_limits: hummock_write_limits, }), + compute_node_total_cpu_count: compute_node_total_cpu_count as _, ..Default::default() }) } async fn compute_subscribe(&self) -> MetaResult { let (secrets, catalog_version) = self.get_decrypted_secret_snapshot().await?; + let compute_node_total_cpu_count = self.get_compute_node_total_cpu_count().await; + Ok(MetaSnapshot { secrets, version: Some(SnapshotVersion { catalog_version, ..Default::default() }), + compute_node_total_cpu_count: compute_node_total_cpu_count as _, ..Default::default() }) } diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index bde42304d58bd..d5b144ce54765 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -39,8 +39,8 @@ use risingwave_pb::meta::update_worker_node_schedulability_request::Schedulabili use sea_orm::prelude::Expr; use sea_orm::ActiveValue::Set; use sea_orm::{ - ActiveModelTrait, ColumnTrait, DatabaseConnection, DatabaseTransaction, EntityTrait, - QueryFilter, QuerySelect, TransactionTrait, + ActiveModelTrait, ColumnTrait, DatabaseConnection, EntityTrait, QueryFilter, QuerySelect, + TransactionTrait, }; use thiserror_ext::AsReport; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver}; @@ -88,7 +88,7 @@ impl From for PbWorkerNode { parallelism: info.1.as_ref().map(|p| p.parallelism).unwrap_or_default() as u32, }), transactional_id: info.0.transaction_id.map(|id| id as _), - resource: info.2.resource, + resource: Some(info.2.resource), started_at: info.2.started_at, } } @@ -119,6 +119,24 @@ impl ClusterController { self.inner.read().await.count_worker_by_type().await } + pub async fn compute_node_total_cpu_count(&self) -> usize { + self.inner.read().await.compute_node_total_cpu_count() + } + + async fn update_compute_node_total_cpu_count(&self) -> MetaResult<()> { + let total_cpu_cores = self.compute_node_total_cpu_count().await; + + // Update local license manager. + LicenseManager::get().update_cpu_core_count(total_cpu_cores); + // Notify all other nodes. + self.env.notification_manager().notify_all_without_version( + Operation::Update, // unused + Info::ComputeNodeTotalCpuCount(total_cpu_cores as _), + ); + + Ok(()) + } + /// A worker node will immediately register itself to meta when it bootstraps. /// The meta will assign it with a unique ID and set its state as `Starting`. /// When the worker node is fully ready to serve, it will request meta again @@ -130,7 +148,8 @@ impl ClusterController { property: AddNodeProperty, resource: PbResource, ) -> MetaResult { - self.inner + let worker_id = self + .inner .write() .await .add_worker( @@ -140,7 +159,13 @@ impl ClusterController { resource, self.max_heartbeat_interval, ) - .await + .await?; + + if r#type == PbWorkerType::ComputeNode { + self.update_compute_node_total_cpu_count().await?; + } + + Ok(worker_id) } pub async fn activate_worker(&self, worker_id: WorkerId) -> MetaResult<()> { @@ -164,13 +189,15 @@ impl ClusterController { } pub async fn delete_worker(&self, host_address: HostAddress) -> MetaResult { - let mut inner = self.inner.write().await; - let worker = inner.delete_worker(host_address).await?; + let worker = self.inner.write().await.delete_worker(host_address).await?; + if worker.r#type() == PbWorkerType::ComputeNode { self.env .notification_manager() .notify_frontend(Operation::Delete, Info::Node(worker.clone())) .await; + + self.update_compute_node_total_cpu_count().await?; } // Notify local subscribers. @@ -432,7 +459,8 @@ pub struct WorkerExtraInfo { // Unix timestamp that the worker will expire at. expire_at: Option, started_at: Option, - resource: Option, + resource: PbResource, + r#type: PbWorkerType, } impl WorkerExtraInfo { @@ -561,7 +589,7 @@ impl ClusterControllerInner { resource: PbResource, ) -> MetaResult<()> { if let Some(info) = self.worker_extra_info.get_mut(&worker_id) { - info.resource = Some(resource); + info.resource = resource; info.update_started_at(); Ok(()) } else { @@ -585,41 +613,12 @@ impl ClusterControllerInner { } } - /// Check if the total CPU cores in the cluster exceed the license limit, after counting the - /// newly joined compute node. - pub async fn check_cpu_core_limit_on_newly_joined_compute_node( - &self, - txn: &DatabaseTransaction, - host_address: &HostAddress, - resource: &PbResource, - ) -> MetaResult<()> { - let this = resource.total_cpu_cores; - - let other_worker_ids: Vec = Worker::find() - .filter( - (worker::Column::Host - .eq(host_address.host.clone()) - .and(worker::Column::Port.eq(host_address.port))) - .not() - .and(worker::Column::WorkerType.eq(WorkerType::ComputeNode)), - ) - .select_only() - .column(worker::Column::WorkerId) - .into_tuple() - .all(txn) - .await?; - - let others = other_worker_ids - .into_iter() - .flat_map(|id| self.worker_extra_info.get(&id)) - .flat_map(|info| info.resource.as_ref().map(|r| r.total_cpu_cores)) - .sum::(); - - LicenseManager::get() - .check_cpu_core_limit(this + others) - .map_err(anyhow::Error::from)?; - - Ok(()) + fn compute_node_total_cpu_count(&self) -> usize { + self.worker_extra_info + .values() + .filter(|info| info.r#type == PbWorkerType::ComputeNode) + .map(|info| info.resource.total_cpu_cores as usize) + .sum() } pub async fn add_worker( @@ -632,11 +631,6 @@ impl ClusterControllerInner { ) -> MetaResult { let txn = self.db.begin().await?; - if let PbWorkerType::ComputeNode = r#type { - self.check_cpu_core_limit_on_newly_joined_compute_node(&txn, &host_address, &resource) - .await?; - } - let worker = Worker::find() .filter( worker::Column::Host @@ -768,8 +762,9 @@ impl ClusterControllerInner { } let extra_info = WorkerExtraInfo { started_at: Some(timestamp_now_sec()), - resource: Some(resource), - ..Default::default() + expire_at: None, + resource, + r#type, }; self.worker_extra_info.insert(worker_id, extra_info); @@ -834,7 +829,9 @@ impl ClusterControllerInner { if let Some(txn_id) = &worker.transaction_id { self.available_transactional_ids.push_back(*txn_id); } - Ok(WorkerInfo(worker, property, extra_info).into()) + let worker: PbWorkerNode = WorkerInfo(worker, property, extra_info).into(); + + Ok(worker) } pub fn heartbeat(&mut self, worker_id: WorkerId, ttl: Duration) -> MetaResult<()> { diff --git a/src/meta/src/manager/notification.rs b/src/meta/src/manager/notification.rs index dce9fabc7ff11..b31f22993ae5a 100644 --- a/src/meta/src/manager/notification.rs +++ b/src/meta/src/manager/notification.rs @@ -170,6 +170,17 @@ impl NotificationManager { ) } + pub fn notify_all_without_version(&self, operation: Operation, info: Info) { + for subscribe_type in [ + SubscribeType::Frontend, + SubscribeType::Hummock, + SubscribeType::Compactor, + SubscribeType::Compute, + ] { + self.notify_without_version(subscribe_type.into(), operation, info.clone()); + } + } + pub async fn notify_frontend(&self, operation: Operation, info: Info) -> NotificationVersion { self.notify_with_version(SubscribeType::Frontend.into(), operation, info) .await diff --git a/src/storage/compactor/src/compactor_observer/observer_manager.rs b/src/storage/compactor/src/compactor_observer/observer_manager.rs index 03eb165fd5dfc..12ca3371bd087 100644 --- a/src/storage/compactor/src/compactor_observer/observer_manager.rs +++ b/src/storage/compactor/src/compactor_observer/observer_manager.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::license::LicenseManager; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::ObserverState; use risingwave_pb::catalog::Table; @@ -60,6 +61,9 @@ impl ObserverState for CompactorObserverNode { Info::SystemParams(p) => { self.system_params_manager.try_set_params(p); } + Info::ComputeNodeTotalCpuCount(count) => { + LicenseManager::get().update_cpu_core_count(count as _); + } _ => { panic!("error type notification"); } @@ -73,6 +77,7 @@ impl ObserverState for CompactorObserverNode { self.handle_catalog_snapshot(snapshot.tables); let snapshot_version = snapshot.version.unwrap(); self.version = snapshot_version.catalog_version; + LicenseManager::get().update_cpu_core_count(snapshot.compute_node_total_cpu_count as _); } } diff --git a/src/storage/src/hummock/observer_manager.rs b/src/storage/src/hummock/observer_manager.rs index 1c208e23f474c..6604ad10e3573 100644 --- a/src/storage/src/hummock/observer_manager.rs +++ b/src/storage/src/hummock/observer_manager.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::license::LicenseManager; use risingwave_common_service::ObserverState; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_trace::TraceSpan; @@ -91,6 +92,10 @@ impl ObserverState for HummockObserverNode { .update_write_limits(write_limits.write_limits); } + Info::ComputeNodeTotalCpuCount(count) => { + LicenseManager::get().update_cpu_core_count(count as _); + } + _ => { panic!("error type notification"); } @@ -132,6 +137,7 @@ impl ObserverState for HummockObserverNode { }); let snapshot_version = snapshot.version.unwrap(); self.version = snapshot_version.catalog_version; + LicenseManager::get().update_cpu_core_count(snapshot.compute_node_total_cpu_count as _); } } diff --git a/src/tests/simulation/tests/integration_tests/license_cpu_limit.rs b/src/tests/simulation/tests/integration_tests/license_cpu_limit.rs new file mode 100644 index 0000000000000..fbce9dfa3fc71 --- /dev/null +++ b/src/tests/simulation/tests/integration_tests/license_cpu_limit.rs @@ -0,0 +1,89 @@ +// Copyright 2025 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::Result; +use risingwave_common::error::AsReport; +use risingwave_simulation::cluster::{Cluster, Configuration}; +use risingwave_simulation::utils::AssertResult; + +/// Paid-tier key with CPU limit 20. +const KEY_20: &str = "eyJhbGciOiJSUzUxMiIsInR5cCI6IkpXVCJ9.eyJzdWIiOiJwYWlkLXRlc3QtMzIiLCJpc3MiOiJ0ZXN0LnJpc2luZ3dhdmUuY29tIiwidGllciI6InBhaWQiLCJleHAiOjIxNTA0OTU5OTksImlhdCI6MTczNzYxNTI2NywiY3B1X2NvcmVfbGltaXQiOjIwfQ.V8546BDZydv1aNk8IlVaSVlCriDtMC_75nq8CIaRPKlrltcwRJYKfK-Ru3WbKj-MDFebmW_3CqA4jR77BBtmmmtj-lPHa4qrdgrMItxm9RC_qoSU1YbI8Kb_ClYkrnFug5MAbK3wGlO8CrrjqOOt-Q5ggKChtl0uFj4zgI-S80d8Hse5LYSKHv8cU-ECKvEFe451kXE9x7nN_f8MqTSOqBwfY5o17gTD8oU3XH2k1mpesdci18kDmJPK5DeLPDYht_nRt7WGbVQvx7iiol1nzj5OBjdH_eVbX7pfk9M-JNwqZKaqfOmBbwV2F-Sf7-tK33O-XqSfXjnLAzflfjkoLQ"; +/// Paid-tier key with CPU limit 100. +const KEY_100: &str = "eyJhbGciOiJSUzUxMiIsInR5cCI6IkpXVCJ9.eyJzdWIiOiJwYWlkLXRlc3QtMzIiLCJpc3MiOiJ0ZXN0LnJpc2luZ3dhdmUuY29tIiwidGllciI6InBhaWQiLCJleHAiOjIxNTA0OTU5OTksImlhdCI6MTczNzYyMzc2MywiY3B1X2NvcmVfbGltaXQiOjEwMH0.ZGQjZa6t3va5MHMHKvgMgOXVLymEvvy1Yvd6teRUgCIF7en5BYaKXWuXwwtWLpLxr7LXyIQ3LQeDXag4k_fQOUTwV4oYTLTFVF8GcJ8JvGdTjBfjnM_2helLEhjZFgXSnhEy-xTOj5yM0BbqKCwanYlSODXQtp5owalt7a0JDwpId9_O8Pl24CjImZzPZLevJ_wSu4wv2IhVjK5QhkfBKBeaOxCeJaKfMVT5AzDQ-WwtJwahr1Dk0H2BxD6Hmp4KKBFRlVwGxq9-8uKBpbrmlClSuxPreBJ_xhP3SHtBFbVfcr38uaT_Bdh-gPRPgi-59tKOWPCY2FytO-Ls1U2l7w"; + +#[tokio::test] +async fn test_license_cpu_limit() -> Result<()> { + // Now 8 * 3 = 24 cores in total. + let mut cluster = Cluster::start(Configuration { + compute_nodes: 3, + compute_node_cores: 8, + ..Default::default() + }) + .await?; + + let mut session = cluster.start_session(); + + macro_rules! set_license_key { + ($key:expr) => { + session + .run(format!("ALTER SYSTEM SET license_key TO '{}';", $key)) + .await?; + tokio::time::sleep(std::time::Duration::from_secs(10)).await; + }; + } + + macro_rules! test_paid_tier { + () => { + session.run("SELECT rw_test_paid_tier();").await + }; + } + + set_license_key!(""); + let error = test_paid_tier!().unwrap_err().to_report_string(); + assert!(error.contains("feature TestPaid is only available for tier Paid and above, while the current tier is Free"), "{error}"); + + // Set a license key with CPU limit 100, it should work. + set_license_key!(KEY_100); + test_paid_tier!().unwrap().assert_result_eq("t"); + + // Set a license key with CPU limit 20, which is lower than the current CPU cores. + // Paid-tier features should not be available. + set_license_key!(KEY_20); + let error = test_paid_tier!().unwrap_err().to_report_string(); + assert!( + error.contains("the license key is currently not effective"), + "{error}" + ); + + // Kill a compute node, the total cores will be reduced to 16, which is under the limit. + // The paid-tier features should be available again. + cluster.simple_kill_nodes(["compute-2"]).await; + tokio::time::sleep(std::time::Duration::from_secs(100)).await; + test_paid_tier!().unwrap().assert_result_eq("t"); + + // Add it back, will be unavailable again. + cluster.simple_restart_nodes(["compute-2"]).await; + tokio::time::sleep(std::time::Duration::from_secs(100)).await; + let error = test_paid_tier!().unwrap_err().to_report_string(); + assert!( + error.contains("the license key is currently not effective"), + "{error}" + ); + + // Set a license key with CPU limit 100, it should work again. + set_license_key!(KEY_100); + test_paid_tier!().unwrap().assert_result_eq("t"); + + Ok(()) +} diff --git a/src/tests/simulation/tests/integration_tests/main.rs b/src/tests/simulation/tests/integration_tests/main.rs index ad8e854a30e5f..1bb58688192c8 100644 --- a/src/tests/simulation/tests/integration_tests/main.rs +++ b/src/tests/simulation/tests/integration_tests/main.rs @@ -22,12 +22,12 @@ mod backfill_tests; mod batch; +mod compaction; +mod default_parallelism; +mod license_cpu_limit; mod recovery; mod scale; mod sink; mod storage; mod throttle; - -mod compaction; -mod default_parallelism; mod utils; From c89eeed35ad4496fed9380200a61fb9c2e93ea41 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Mon, 27 Jan 2025 22:55:01 -0600 Subject: [PATCH 17/25] feat(batch): introduce batch AsOf join (#19790) --- e2e_test/batch/join/asof_join.slt | 43 ++++ proto/batch_plan.proto | 1 + src/batch/executors/benches/hash_join.rs | 1 + .../executors/src/executor/join/hash_join.rs | 198 +++++++++++++++--- .../src/executor/join/lookup_join_base.rs | 9 +- src/batch/executors/src/executor/join/mod.rs | 64 +++--- .../src/executor/join/nested_loop_join.rs | 3 + .../tests/testdata/input/asof_join.yaml | 6 +- .../tests/testdata/output/asof_join.yaml | 22 +- .../optimizer/plan_node/batch_hash_join.rs | 38 +++- .../optimizer/plan_node/eq_join_predicate.rs | 10 + .../src/optimizer/plan_node/logical_join.rs | 80 ++++++- .../optimizer/plan_node/stream_asof_join.rs | 52 +---- .../src/scheduler/distributed/query.rs | 2 +- 14 files changed, 402 insertions(+), 127 deletions(-) create mode 100644 e2e_test/batch/join/asof_join.slt diff --git a/e2e_test/batch/join/asof_join.slt b/e2e_test/batch/join/asof_join.slt new file mode 100644 index 0000000000000..bf905b661e107 --- /dev/null +++ b/e2e_test/batch/join/asof_join.slt @@ -0,0 +1,43 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table t1 (v1 int, v2 int, v3 int primary key); + +statement ok +create table t2 (v1 int, v2 int, v3 int primary key); + +statement ok +insert into t1 values (1, 2, 3), (2, 3, 4), (1, 2, 9); + +statement ok +insert into t2 values (1, NULL, 8), (1, 3, 4), (1, 2, 5), (1, 2, 6); + +# asof inner join +query IIIIII +SELECT t1.v1 t1_v1, t1.v2 t1_v2, t1.v3 t1_v3, t2.v1 t2_v1, t2.v2 t2_v2, t2.v3 t2_v3 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 and t1.v2 < t2.v2 order by t1.v1, t1.v3; +---- +1 2 3 1 3 4 +1 2 9 1 3 4 + +# asof left join +query IIIIII +SELECT t1.v1 t1_v1, t1.v2 t1_v2, t1.v3 t1_v3, t2.v1 t2_v1, t2.v2 t2_v2, t2.v3 t2_v3 FROM t1 ASOF LEFT JOIN t2 ON t1.v1 = t2.v1 and t1.v2 < t2.v2 order by t1.v1, t1.v3; +---- +1 2 3 1 3 4 +1 2 9 1 3 4 +2 3 4 NULL NULL NULL + +# asof left join +query IIIIII +SELECT t1.v1 t1_v1, t1.v2 t1_v2, t1.v3 t1_v3, t2.v1 t2_v1, t2.v2 t2_v2, t2.v3 t2_v3 FROM t1 ASOF LEFT JOIN t2 ON t1.v1 = t2.v1 and t1.v2 > t2.v2 order by t1.v1, t1.v3; +---- +1 2 3 NULL NULL NULL +1 2 9 NULL NULL NULL +2 3 4 NULL NULL NULL + +statement ok +drop table t1; + +statement ok +drop table t2; \ No newline at end of file diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 7ffdf94e3c30a..6e07ceae4d5d4 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -289,6 +289,7 @@ message HashJoinNode { // Null safe means it treats `null = null` as true. // Each key pair can be null safe independently. (left_key, right_key, null_safe) repeated bool null_safe = 6; + optional plan_common.AsOfJoinDesc asof_desc = 7; } message SortMergeJoinNode { diff --git a/src/batch/executors/benches/hash_join.rs b/src/batch/executors/benches/hash_join.rs index 330fc299594d0..6d64461dd1c2d 100644 --- a/src/batch/executors/benches/hash_join.rs +++ b/src/batch/executors/benches/hash_join.rs @@ -76,6 +76,7 @@ fn create_hash_join_executor( "HashJoinExecutor".into(), CHUNK_SIZE, None, + None, BatchSpillMetrics::for_test(), ShutdownToken::empty(), MemoryContext::none(), diff --git a/src/batch/executors/src/executor/join/hash_join.rs b/src/batch/executors/src/executor/join/hash_join.rs index af89b3d1503a9..44518e5155496 100644 --- a/src/batch/executors/src/executor/join/hash_join.rs +++ b/src/batch/executors/src/executor/join/hash_join.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp::Ordering; use std::iter; use std::iter::empty; use std::marker::PhantomData; @@ -25,8 +26,8 @@ use risingwave_common::bitmap::{Bitmap, BitmapBuilder, FilterByBitmap}; use risingwave_common::catalog::Schema; use risingwave_common::hash::{HashKey, HashKeyDispatcher, PrecomputedBuildHasher}; use risingwave_common::memory::{MemoryContext, MonitoredGlobalAlloc}; -use risingwave_common::row::{repeat_n, RowExt}; -use risingwave_common::types::{DataType, Datum}; +use risingwave_common::row::{repeat_n, Row, RowExt}; +use risingwave_common::types::{DataType, Datum, DefaultOrd}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common_estimate_size::EstimateSize; @@ -35,7 +36,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::data::DataChunk as PbDataChunk; use risingwave_pb::Message; -use super::{ChunkedData, JoinType, RowId}; +use super::{AsOfDesc, AsOfInequalityType, ChunkedData, JoinType, RowId}; use crate::error::{BatchError, Result}; use crate::executor::{ BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder, @@ -83,6 +84,8 @@ pub struct HashJoinExecutor { null_matched: Vec, identity: String, chunk_size: usize, + /// Whether the join is an as-of join + asof_desc: Option, spill_backend: Option, spill_metrics: Arc, @@ -179,6 +182,7 @@ pub struct EquiJoinParams { next_build_row_with_same_key: ChunkedData>, chunk_size: usize, shutdown_rx: ShutdownToken, + asof_desc: Option, } impl EquiJoinParams { @@ -194,6 +198,7 @@ impl EquiJoinParams { next_build_row_with_same_key: ChunkedData>, chunk_size: usize, shutdown_rx: ShutdownToken, + asof_desc: Option, ) -> Self { Self { probe_side, @@ -206,6 +211,7 @@ impl EquiJoinParams { next_build_row_with_same_key, chunk_size, shutdown_rx, + asof_desc, } } } @@ -648,6 +654,7 @@ impl HashJoinExecutor { self.cond.clone(), format!("{}-sub{}", self.identity.clone(), i), self.chunk_size, + self.asof_desc.clone(), self.spill_backend.clone(), self.spill_metrics.clone(), Some(partition_size), @@ -683,9 +690,12 @@ impl HashJoinExecutor { next_build_row_with_same_key, self.chunk_size, self.shutdown_rx.clone(), + self.asof_desc, ); - if let Some(cond) = self.cond.as_ref() { + if let Some(cond) = self.cond.as_ref() + && params.asof_desc.is_none() + { let stream = match self.join_type { JoinType::Inner => Self::do_inner_join_with_non_equi_condition(params, cond), JoinType::LeftOuter => { @@ -709,6 +719,9 @@ impl HashJoinExecutor { JoinType::FullOuter => { Self::do_full_outer_join_with_non_equi_condition(params, cond) } + JoinType::AsOfInner | JoinType::AsOfLeftOuter => { + unreachable!("AsOf join should not reach here") + } }; // For non-equi join, we need an output chunk builder to align the output chunks. let mut output_chunk_builder = @@ -726,8 +739,10 @@ impl HashJoinExecutor { } } else { let stream = match self.join_type { - JoinType::Inner => Self::do_inner_join(params), - JoinType::LeftOuter => Self::do_left_outer_join(params), + JoinType::Inner | JoinType::AsOfInner => Self::do_inner_join(params), + JoinType::LeftOuter | JoinType::AsOfLeftOuter => { + Self::do_left_outer_join(params) + } JoinType::LeftSemi => Self::do_left_semi_anti_join::(params), JoinType::LeftAnti => Self::do_left_semi_anti_join::(params), JoinType::RightOuter => Self::do_right_outer_join(params), @@ -754,6 +769,7 @@ impl HashJoinExecutor { next_build_row_with_same_key, chunk_size, shutdown_rx, + asof_desc, .. }: EquiJoinParams, ) { @@ -767,19 +783,39 @@ impl HashJoinExecutor { .enumerate() .filter_by_bitmap(probe_chunk.visibility()) { - for build_row_id in - next_build_row_with_same_key.row_id_iter(hash_map.get(probe_key).copied()) - { - shutdown_rx.check()?; - let build_chunk = &build_side[build_row_id.chunk_id()]; - if let Some(spilled) = Self::append_one_row( - &mut chunk_builder, - &probe_chunk, - probe_row_id, - build_chunk, - build_row_id.row_id(), + let build_side_row_iter = + next_build_row_with_same_key.row_id_iter(hash_map.get(probe_key).copied()); + if let Some(asof_desc) = &asof_desc { + if let Some(build_row_id) = Self::find_asof_matched_rows( + probe_chunk.row_at_unchecked_vis(probe_row_id), + &build_side, + build_side_row_iter, + asof_desc, ) { - yield spilled + shutdown_rx.check()?; + if let Some(spilled) = Self::append_one_row( + &mut chunk_builder, + &probe_chunk, + probe_row_id, + &build_side[build_row_id.chunk_id()], + build_row_id.row_id(), + ) { + yield spilled + } + } + } else { + for build_row_id in build_side_row_iter { + shutdown_rx.check()?; + let build_chunk = &build_side[build_row_id.chunk_id()]; + if let Some(spilled) = Self::append_one_row( + &mut chunk_builder, + &probe_chunk, + probe_row_id, + build_chunk, + build_row_id.row_id(), + ) { + yield spilled + } } } } @@ -814,6 +850,7 @@ impl HashJoinExecutor { next_build_row_with_same_key, chunk_size, shutdown_rx, + asof_desc, .. }: EquiJoinParams, ) { @@ -828,19 +865,49 @@ impl HashJoinExecutor { .filter_by_bitmap(probe_chunk.visibility()) { if let Some(first_matched_build_row_id) = hash_map.get(probe_key) { - for build_row_id in - next_build_row_with_same_key.row_id_iter(Some(*first_matched_build_row_id)) - { - shutdown_rx.check()?; - let build_chunk = &build_side[build_row_id.chunk_id()]; - if let Some(spilled) = Self::append_one_row( - &mut chunk_builder, - &probe_chunk, - probe_row_id, - build_chunk, - build_row_id.row_id(), + let build_side_row_iter = + next_build_row_with_same_key.row_id_iter(Some(*first_matched_build_row_id)); + if let Some(asof_desc) = &asof_desc { + if let Some(build_row_id) = Self::find_asof_matched_rows( + probe_chunk.row_at_unchecked_vis(probe_row_id), + &build_side, + build_side_row_iter, + asof_desc, ) { - yield spilled + shutdown_rx.check()?; + if let Some(spilled) = Self::append_one_row( + &mut chunk_builder, + &probe_chunk, + probe_row_id, + &build_side[build_row_id.chunk_id()], + build_row_id.row_id(), + ) { + yield spilled + } + } else { + shutdown_rx.check()?; + let probe_row = probe_chunk.row_at_unchecked_vis(probe_row_id); + if let Some(spilled) = Self::append_one_row_with_null_build_side( + &mut chunk_builder, + probe_row, + build_data_types.len(), + ) { + yield spilled + } + } + } else { + for build_row_id in build_side_row_iter { + shutdown_rx.check()?; + let build_chunk = &build_side[build_row_id.chunk_id()]; + if let Some(spilled) = Self::append_one_row( + &mut chunk_builder, + &probe_chunk, + probe_row_id, + build_chunk, + build_row_id.row_id(), + ) { + yield spilled + } } } } else { @@ -1916,6 +1983,64 @@ impl HashJoinExecutor { ) -> Option { chunk_builder.append_one_row(repeat_n(Datum::None, probe_column_count).chain(build_row_ref)) } + + fn find_asof_matched_rows( + probe_row_ref: RowRef<'_>, + build_side: &[DataChunk], + build_side_row_iter: RowIdIter<'_>, + asof_join_condition: &AsOfDesc, + ) -> Option { + let probe_inequality_value = probe_row_ref.datum_at(asof_join_condition.left_idx); + if let Some(probe_inequality_scalar) = probe_inequality_value { + let mut result_row_id: Option = None; + let mut build_row_ref; + + for build_row_id in build_side_row_iter { + build_row_ref = + build_side[build_row_id.chunk_id()].row_at_unchecked_vis(build_row_id.row_id()); + let build_inequality_value = build_row_ref.datum_at(asof_join_condition.right_idx); + if let Some(build_inequality_scalar) = build_inequality_value { + let mut pick_result = |compare: fn(Ordering) -> bool| { + if let Some(result_row_id_inner) = result_row_id { + let result_row_ref = build_side[result_row_id_inner.chunk_id()] + .row_at_unchecked_vis(result_row_id_inner.row_id()); + let result_inequality_scalar = result_row_ref + .datum_at(asof_join_condition.right_idx) + .unwrap(); + if compare( + probe_inequality_scalar.default_cmp(&build_inequality_scalar), + ) && compare( + probe_inequality_scalar.default_cmp(&result_inequality_scalar), + ) { + result_row_id = Some(build_row_id); + } + } else if compare( + probe_inequality_scalar.default_cmp(&build_inequality_scalar), + ) { + result_row_id = Some(build_row_id); + } + }; + match asof_join_condition.inequality_type { + AsOfInequalityType::Lt => { + pick_result(Ordering::is_lt); + } + AsOfInequalityType::Le => { + pick_result(Ordering::is_le); + } + AsOfInequalityType::Gt => { + pick_result(Ordering::is_gt); + } + AsOfInequalityType::Ge => { + pick_result(Ordering::is_ge); + } + } + } + } + result_row_id + } else { + None + } + } } /// `DataChunkMutator` transforms the given data chunk for non-equi join. @@ -2195,6 +2320,11 @@ impl BoxedExecutorBuilder for HashJoinExecutor<()> { let identity = context.plan_node().get_identity().clone(); + let asof_desc = hash_join_node + .asof_desc + .map(|desc| AsOfDesc::from_protobuf(&desc)) + .transpose()?; + Ok(HashJoinExecutorArgs { join_type, output_indices, @@ -2207,6 +2337,7 @@ impl BoxedExecutorBuilder for HashJoinExecutor<()> { identity: identity.clone(), right_key_types, chunk_size: context.context().get_config().developer.chunk_size, + asof_desc, spill_backend: if context.context().get_config().enable_spill { Some(Disk) } else { @@ -2232,6 +2363,7 @@ struct HashJoinExecutorArgs { identity: String, right_key_types: Vec, chunk_size: usize, + asof_desc: Option, spill_backend: Option, spill_metrics: Arc, shutdown_rx: ShutdownToken, @@ -2253,6 +2385,7 @@ impl HashKeyDispatcher for HashJoinExecutorArgs { self.cond.map(Arc::new), self.identity, self.chunk_size, + self.asof_desc, self.spill_backend, self.spill_metrics, self.shutdown_rx, @@ -2278,6 +2411,7 @@ impl HashJoinExecutor { cond: Option>, identity: String, chunk_size: usize, + asof_desc: Option, spill_backend: Option, spill_metrics: Arc, shutdown_rx: ShutdownToken, @@ -2294,6 +2428,7 @@ impl HashJoinExecutor { cond, identity, chunk_size, + asof_desc, spill_backend, spill_metrics, None, @@ -2314,6 +2449,7 @@ impl HashJoinExecutor { cond: Option>, identity: String, chunk_size: usize, + asof_desc: Option, spill_backend: Option, spill_metrics: Arc, memory_upper_bound: Option, @@ -2352,6 +2488,7 @@ impl HashJoinExecutor { cond, identity, chunk_size, + asof_desc, shutdown_rx, spill_backend, spill_metrics, @@ -2632,6 +2769,7 @@ mod tests { cond, "HashJoinExecutor".to_owned(), chunk_size, + None, if test_spill { Some(SpillBackend::Memory) } else { diff --git a/src/batch/executors/src/executor/join/lookup_join_base.rs b/src/batch/executors/src/executor/join/lookup_join_base.rs index d3f806e01c547..3a8466cbad26e 100644 --- a/src/batch/executors/src/executor/join/lookup_join_base.rs +++ b/src/batch/executors/src/executor/join/lookup_join_base.rs @@ -178,6 +178,7 @@ impl LookupJoinBase { next_build_row_with_same_key, self.chunk_size, self.shutdown_rx.clone(), + None, ); if let Some(cond) = self.condition.as_ref() { @@ -197,7 +198,9 @@ impl LookupJoinBase { JoinType::RightOuter | JoinType::RightSemi | JoinType::RightAnti - | JoinType::FullOuter => unimplemented!(), + | JoinType::FullOuter + | JoinType::AsOfInner + | JoinType::AsOfLeftOuter => unimplemented!(), }; // For non-equi join, we need an output chunk builder to align the output chunks. let mut output_chunk_builder = @@ -222,7 +225,9 @@ impl LookupJoinBase { JoinType::RightOuter | JoinType::RightSemi | JoinType::RightAnti - | JoinType::FullOuter => unimplemented!(), + | JoinType::FullOuter + | JoinType::AsOfInner + | JoinType::AsOfLeftOuter => unimplemented!(), }; #[for_await] for chunk in stream { diff --git a/src/batch/executors/src/executor/join/mod.rs b/src/batch/executors/src/executor/join/mod.rs index 6c404a7cf5b6b..571babc4ca1fb 100644 --- a/src/batch/executors/src/executor/join/mod.rs +++ b/src/batch/executors/src/executor/join/mod.rs @@ -30,7 +30,7 @@ use risingwave_common::array::{DataChunk, RowRef}; use risingwave_common::row::Row; use risingwave_common::types::{DataType, DatumRef}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_pb::plan_common::JoinType as PbJoinType; +use risingwave_pb::plan_common::{AsOfJoinDesc, AsOfJoinInequalityType, JoinType as PbJoinType}; use crate::error::Result; @@ -49,6 +49,8 @@ pub enum JoinType { /// Anti join when build side should output when matched RightAnti, FullOuter, + AsOfInner, + AsOfLeftOuter, } impl JoinType { @@ -62,7 +64,9 @@ impl JoinType { PbJoinType::RightSemi => JoinType::RightSemi, PbJoinType::RightAnti => JoinType::RightAnti, PbJoinType::FullOuter => JoinType::FullOuter, - PbJoinType::AsofInner | PbJoinType::AsofLeftOuter | PbJoinType::Unspecified => { + PbJoinType::AsofInner => JoinType::AsOfInner, + PbJoinType::AsofLeftOuter => JoinType::AsOfLeftOuter, + PbJoinType::Unspecified => { unreachable!() } } @@ -73,28 +77,6 @@ impl JoinType { impl JoinType { #![allow(dead_code)] - #[inline(always)] - pub(super) fn need_join_remaining(self) -> bool { - matches!( - self, - JoinType::RightOuter | JoinType::RightAnti | JoinType::FullOuter - ) - } - - fn need_build(self) -> bool { - match self { - JoinType::RightSemi => true, - other => other.need_join_remaining(), - } - } - - fn need_probe(self) -> bool { - matches!( - self, - JoinType::FullOuter | JoinType::LeftOuter | JoinType::LeftAnti | JoinType::LeftSemi - ) - } - fn keep_all(self) -> bool { matches!( self, @@ -111,6 +93,40 @@ impl JoinType { } } +#[derive(Clone, Debug)] +pub enum AsOfInequalityType { + Le, + Lt, + Ge, + Gt, +} + +#[derive(Clone, Debug)] +pub struct AsOfDesc { + pub left_idx: usize, + pub right_idx: usize, + pub inequality_type: AsOfInequalityType, +} + +impl AsOfDesc { + pub fn from_protobuf(desc_proto: &AsOfJoinDesc) -> crate::error::Result { + let typ = match desc_proto.inequality_type() { + AsOfJoinInequalityType::AsOfInequalityTypeLt => AsOfInequalityType::Lt, + AsOfJoinInequalityType::AsOfInequalityTypeLe => AsOfInequalityType::Le, + AsOfJoinInequalityType::AsOfInequalityTypeGt => AsOfInequalityType::Gt, + AsOfJoinInequalityType::AsOfInequalityTypeGe => AsOfInequalityType::Ge, + AsOfJoinInequalityType::AsOfInequalityTypeUnspecified => { + bail!("unspecified AsOf join inequality type") + } + }; + Ok(Self { + left_idx: desc_proto.left_idx as usize, + right_idx: desc_proto.right_idx as usize, + inequality_type: typ, + }) + } +} + /// The layout be like: /// /// [ `left` chunk | `right` chunk ] diff --git a/src/batch/executors/src/executor/join/nested_loop_join.rs b/src/batch/executors/src/executor/join/nested_loop_join.rs index 14917646e84c1..19bc46baccf3f 100644 --- a/src/batch/executors/src/executor/join/nested_loop_join.rs +++ b/src/batch/executors/src/executor/join/nested_loop_join.rs @@ -116,6 +116,9 @@ impl NestedLoopJoinExecutor { JoinType::RightSemi => Self::do_right_semi_anti_join::, JoinType::RightAnti => Self::do_right_semi_anti_join::, JoinType::FullOuter => Self::do_full_outer_join, + JoinType::AsOfInner | JoinType::AsOfLeftOuter => { + unimplemented!("AsOf join is not supported in NestedLoopJoinExecutor") + } }; #[for_await] diff --git a/src/frontend/planner_test/tests/testdata/input/asof_join.yaml b/src/frontend/planner_test/tests/testdata/input/asof_join.yaml index f6ca65716c2ea..5697a8f2c111d 100644 --- a/src/frontend/planner_test/tests/testdata/input/asof_join.yaml +++ b/src/frontend/planner_test/tests/testdata/input/asof_join.yaml @@ -4,14 +4,15 @@ SELECT * FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1; expected_outputs: - stream_error + - batch_error - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 || 'a' and t1.v2 > t2.v2; expected_outputs: - - batch_error - stream_plan + - batch_plan - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); @@ -19,6 +20,7 @@ SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF LEFT JOIN t2 ON t1.v1 = t2.v1 and t1.v2 *2 < t2.v2; expected_outputs: - stream_plan + - batch_plan - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); @@ -26,6 +28,7 @@ SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 and t1.v2 < t2.v2 and t1.v3 < t2.v3; expected_outputs: - stream_error + - batch_error - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); @@ -33,3 +36,4 @@ SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v2 < t2.v2; expected_outputs: - stream_error + - batch_error diff --git a/src/frontend/planner_test/tests/testdata/output/asof_join.yaml b/src/frontend/planner_test/tests/testdata/output/asof_join.yaml index 508c9de04f18d..74f8464a74bd7 100644 --- a/src/frontend/planner_test/tests/testdata/output/asof_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/asof_join.yaml @@ -1,7 +1,16 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT * FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1; + batch_error: 'Invalid input syntax: AsOf join requires exactly 1 ineuquality condition' stream_error: 'Invalid input syntax: AsOf join requires exactly 1 ineuquality condition' - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 || 'a' and t1.v2 > t2.v2; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: AsofInner, predicate: t1.v1 = $expr1 AND (t1.v2 > t2.v2), output: [t1.v1, t1.v2, t2.v1, t2.v2] } + ├─BatchExchange { order: [], dist: HashShard(t1.v1) } + │ └─BatchScan { table: t1, columns: [t1.v1, t1.v2], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard($expr1) } + └─BatchProject { exprs: [t2.v1, t2.v2, ConcatOp(t2.v1, 'a':Varchar) as $expr1] } + └─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck } └─StreamAsOfJoin { type: AsofInner, predicate: t1.v1 = $expr1 AND (t1.v2 > t2.v2), output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } @@ -10,10 +19,15 @@ └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [t2.v1, t2.v2, ConcatOp(t2.v1, 'a':Varchar) as $expr1, t2._row_id] } └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } - batch_error: |- - Not supported: AsOf join in batch query - HINT: AsOf join is only supported in streaming query - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF LEFT JOIN t2 ON t1.v1 = t2.v1 and t1.v2 *2 < t2.v2; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: AsofLeftOuter, predicate: t1.v1 = t2.v1 AND ($expr1 < t2.v2), output: [t1.v1, t1.v2, t2.v1, t2.v2] } + ├─BatchExchange { order: [], dist: HashShard(t1.v1) } + │ └─BatchProject { exprs: [t1.v1, t1.v2, (t1.v2 * 2:Int32) as $expr1] } + │ └─BatchScan { table: t1, columns: [t1.v1, t1.v2], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(t2.v1) } + └─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck } └─StreamAsOfJoin { type: AsofLeftOuter, predicate: t1.v1 = t2.v1 AND ($expr1 < t2.v2), output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } @@ -23,6 +37,8 @@ └─StreamExchange { dist: HashShard(t2.v1) } └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 and t1.v2 < t2.v2 and t1.v3 < t2.v3; + batch_error: 'Invalid input syntax: AsOf join requires exactly 1 ineuquality condition' stream_error: 'Invalid input syntax: AsOf join requires exactly 1 ineuquality condition' - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v2 < t2.v2; + batch_error: 'Invalid input syntax: AsOf join requires at least 1 equal condition' stream_error: 'Invalid input syntax: AsOf join requires at least 1 equal condition' diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index 9256131f569ad..a902160066844 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -15,13 +15,13 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashJoinNode; -use risingwave_pb::plan_common::JoinType; +use risingwave_pb::plan_common::{AsOfJoinDesc, JoinType}; use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ - generic, EqJoinPredicate, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, - ToDistributedBatch, + generic, EqJoinPredicate, ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, + ToBatchPb, ToDistributedBatch, }; use crate::error::Result; use crate::expr::{Expr, ExprRewriter, ExprVisitor}; @@ -38,14 +38,19 @@ use crate::utils::ColIndexMappingRewriteExt; pub struct BatchHashJoin { pub base: PlanBase, core: generic::Join, - /// The join condition must be equivalent to `logical.on`, but separated into equal and /// non-equal parts to facilitate execution later eq_join_predicate: EqJoinPredicate, + /// `AsOf` desc + asof_desc: Option, } impl BatchHashJoin { - pub fn new(core: generic::Join, eq_join_predicate: EqJoinPredicate) -> Self { + pub fn new( + core: generic::Join, + eq_join_predicate: EqJoinPredicate, + asof_desc: Option, + ) -> Self { let dist = Self::derive_dist(core.left.distribution(), core.right.distribution(), &core); let base = PlanBase::new_batch_with_core(&core, dist, Order::any()); @@ -53,6 +58,7 @@ impl BatchHashJoin { base, core, eq_join_predicate, + asof_desc, } } @@ -66,11 +72,16 @@ impl BatchHashJoin { // we can not derive the hash distribution from the side where outer join can generate a // NULL row (Distribution::HashShard(_), Distribution::HashShard(_)) => match join.join_type { - JoinType::AsofInner | JoinType::AsofLeftOuter | JoinType::Unspecified => { + JoinType::Unspecified => { unreachable!() } JoinType::FullOuter => Distribution::SomeShard, - JoinType::Inner | JoinType::LeftOuter | JoinType::LeftSemi | JoinType::LeftAnti => { + JoinType::Inner + | JoinType::LeftOuter + | JoinType::LeftSemi + | JoinType::LeftAnti + | JoinType::AsofInner + | JoinType::AsofLeftOuter => { let l2o = join.l2i_col_mapping().composite(&join.i2o_col_mapping()); l2o.rewrite_provided_distribution(left) } @@ -127,7 +138,7 @@ impl PlanTreeNodeBinary for BatchHashJoin { let mut core = self.core.clone(); core.left = left; core.right = right; - Self::new(core, self.eq_join_predicate.clone()) + Self::new(core, self.eq_join_predicate.clone(), self.asof_desc) } } @@ -215,6 +226,7 @@ impl ToBatchPb for BatchHashJoin { .as_expr_unless_true() .map(|x| x.to_expr_proto()), output_indices: self.core.output_indices.iter().map(|&x| x as u32).collect(), + asof_desc: self.asof_desc, }) } } @@ -238,7 +250,15 @@ impl ExprRewritable for BatchHashJoin { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); - Self::new(core, self.eq_join_predicate.rewrite_exprs(r)).into() + let eq_join_predicate = self.eq_join_predicate.rewrite_exprs(r); + let desc = self.asof_desc.map(|_| { + LogicalJoin::get_inequality_desc_from_predicate( + eq_join_predicate.other_cond().clone(), + core.left.schema().len(), + ) + .unwrap() + }); + Self::new(core, eq_join_predicate, desc).into() } } diff --git a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs index 449e26434d34b..071e63dfc6281 100644 --- a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs +++ b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs @@ -162,6 +162,16 @@ impl EqJoinPredicate { &mut self.other_cond } + /// Get the equal predicate + pub fn eq_predicate(&self) -> Self { + Self { + other_cond: Condition::true_cond(), + eq_keys: self.eq_keys.clone(), + left_cols_num: self.left_cols_num, + right_cols_num: self.right_cols_num, + } + } + /// Get a reference to the join predicate's eq keys. /// /// Note: `right_col_index` starts from `left_cols_num` diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 208e947ef40d5..180989a2b0467 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -17,7 +17,9 @@ use std::collections::HashMap; use fixedbitset::FixedBitSet; use itertools::{EitherOrBoth, Itertools}; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_pb::plan_common::JoinType; +use risingwave_expr::bail; +use risingwave_pb::expr::expr_node::PbType; +use risingwave_pb::plan_common::{AsOfJoinDesc, JoinType, PbAsOfJoinInequalityType}; use risingwave_pb::stream_plan::StreamScanType; use risingwave_sqlparser::ast::AsOf; @@ -1379,7 +1381,7 @@ impl LogicalJoin { let logical_join = self.clone_with_left_right(left, right); let inequality_desc = - StreamAsOfJoin::get_inequality_desc_from_predicate(predicate.clone(), left_len)?; + Self::get_inequality_desc_from_predicate(predicate.other_cond().clone(), left_len)?; Ok(StreamAsOfJoin::new( logical_join.core.clone(), @@ -1387,17 +1389,71 @@ impl LogicalJoin { inequality_desc, )) } -} -impl ToBatch for LogicalJoin { - fn to_batch(&self) -> Result { - if JoinType::AsofInner == self.join_type() || JoinType::AsofLeftOuter == self.join_type() { - return Err(ErrorCode::NotSupported( - "AsOf join in batch query".to_owned(), - "AsOf join is only supported in streaming query".to_owned(), + /// Convert the logical `AsOf` join to a Hash join + a Group top 1. + fn to_batch_asof_join( + &self, + logical_join: generic::Join, + predicate: EqJoinPredicate, + ) -> Result { + use super::batch::prelude::*; + + if predicate.eq_keys().is_empty() { + return Err(ErrorCode::InvalidInputSyntax( + "AsOf join requires at least 1 equal condition".to_owned(), ) .into()); } + + let left_schema_len = logical_join.left.schema().len(); + let asof_desc = + Self::get_inequality_desc_from_predicate(predicate.non_eq_cond(), left_schema_len)?; + + let batch_join = BatchHashJoin::new(logical_join, predicate, Some(asof_desc)); + Ok(batch_join.into()) + } + + pub fn get_inequality_desc_from_predicate( + predicate: Condition, + left_input_len: usize, + ) -> Result { + let expr: ExprImpl = predicate.into(); + if let Some((left_input_ref, expr_type, right_input_ref)) = expr.as_comparison_cond() { + if left_input_ref.index() < left_input_len && right_input_ref.index() >= left_input_len + { + Ok(AsOfJoinDesc { + left_idx: left_input_ref.index() as u32, + right_idx: (right_input_ref.index() - left_input_len) as u32, + inequality_type: Self::expr_type_to_comparison_type(expr_type)?.into(), + }) + } else { + bail!("inequal condition from the same side should be push down in optimizer"); + } + } else { + Err(ErrorCode::InvalidInputSyntax( + "AsOf join requires exactly 1 ineuquality condition".to_owned(), + ) + .into()) + } + } + + fn expr_type_to_comparison_type(expr_type: PbType) -> Result { + match expr_type { + PbType::LessThan => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeLt), + PbType::LessThanOrEqual => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeLe), + PbType::GreaterThan => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeGt), + PbType::GreaterThanOrEqual => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeGe), + _ => Err(ErrorCode::InvalidInputSyntax(format!( + "Invalid comparison type: {}", + expr_type.as_str_name() + )) + .into()), + } + } +} + +impl ToBatch for LogicalJoin { + fn to_batch(&self) -> Result { let predicate = EqJoinPredicate::create( self.left().schema().len(), self.right().schema().len(), @@ -1411,7 +1467,9 @@ impl ToBatch for LogicalJoin { let ctx = self.base.ctx(); let config = ctx.session_ctx().config(); - if predicate.has_eq() { + if self.join_type() == JoinType::AsofInner || self.join_type() == JoinType::AsofLeftOuter { + self.to_batch_asof_join(logical_join, predicate) + } else if predicate.has_eq() { if !predicate.eq_keys_are_type_aligned() { return Err(ErrorCode::InternalError(format!( "Join eq keys are not aligned for predicate: {predicate:?}" @@ -1427,7 +1485,7 @@ impl ToBatch for LogicalJoin { } } - Ok(BatchHashJoin::new(logical_join, predicate).into()) + Ok(BatchHashJoin::new(logical_join, predicate, None).into()) } else { // Convert to Nested-loop Join for non-equal joins Ok(BatchNestedLoopJoin::new(logical_join).into()) diff --git a/src/frontend/src/optimizer/plan_node/stream_asof_join.rs b/src/frontend/src/optimizer/plan_node/stream_asof_join.rs index 49d6e9c43aabc..b0c417e749cd4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_asof_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_asof_join.rs @@ -15,9 +15,7 @@ use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::util::sort_util::OrderType; -use risingwave_expr::bail; -use risingwave_pb::expr::expr_node::PbType; -use risingwave_pb::plan_common::{AsOfJoinDesc, AsOfJoinType, JoinType, PbAsOfJoinInequalityType}; +use risingwave_pb::plan_common::{AsOfJoinDesc, AsOfJoinType, JoinType}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::AsOfJoinNode; @@ -26,10 +24,10 @@ use super::utils::{ childless_record, plan_node_name, watermark_pretty, Distill, TableCatalogBuilder, }; use super::{ - generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamJoinCommon, StreamNode, + generic, ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, StreamJoinCommon, + StreamNode, }; -use crate::error::{ErrorCode, Result}; -use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor}; +use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{EqJoinPredicate, EqJoinPredicateDisplay}; @@ -97,44 +95,6 @@ impl StreamAsOfJoin { } } - pub fn get_inequality_desc_from_predicate( - predicate: EqJoinPredicate, - left_input_len: usize, - ) -> Result { - let expr: ExprImpl = predicate.other_cond().clone().into(); - if let Some((left_input_ref, expr_type, right_input_ref)) = expr.as_comparison_cond() { - if left_input_ref.index() < left_input_len && right_input_ref.index() >= left_input_len - { - Ok(AsOfJoinDesc { - left_idx: left_input_ref.index() as u32, - right_idx: (right_input_ref.index() - left_input_len) as u32, - inequality_type: Self::expr_type_to_comparison_type(expr_type)?.into(), - }) - } else { - bail!("inequal condition from the same side should be push down in optimizer"); - } - } else { - Err(ErrorCode::InvalidInputSyntax( - "AsOf join requires exactly 1 ineuquality condition".to_owned(), - ) - .into()) - } - } - - fn expr_type_to_comparison_type(expr_type: PbType) -> Result { - match expr_type { - PbType::LessThan => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeLt), - PbType::LessThanOrEqual => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeLe), - PbType::GreaterThan => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeGt), - PbType::GreaterThanOrEqual => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeGe), - _ => Err(ErrorCode::InvalidInputSyntax(format!( - "Invalid comparison type: {}", - expr_type.as_str_name() - )) - .into()), - } - } - /// Get join type pub fn join_type(&self) -> JoinType { self.core.join_type @@ -332,8 +292,8 @@ impl ExprRewritable for StreamAsOfJoin { let mut core = self.core.clone(); core.rewrite_exprs(r); let eq_join_predicate = self.eq_join_predicate.rewrite_exprs(r); - let desc = Self::get_inequality_desc_from_predicate( - eq_join_predicate.clone(), + let desc = LogicalJoin::get_inequality_desc_from_predicate( + eq_join_predicate.other_cond().clone(), core.left.schema().len(), ) .unwrap(); diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index cf6319c894a0e..f1915dc7e5748 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -657,7 +657,7 @@ pub(crate) mod tests { let eq_join_predicate = EqJoinPredicate::new(Condition::true_cond(), vec![eq_key_1, eq_key_2], 2, 2); let hash_join_node: PlanRef = - BatchHashJoin::new(logical_join_node, eq_join_predicate).into(); + BatchHashJoin::new(logical_join_node, eq_join_predicate, None).into(); let batch_exchange_node: PlanRef = BatchExchange::new( hash_join_node.clone(), Order::default(), From 962703d890121ed42a278f8762e0b4bae87c9391 Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Tue, 28 Jan 2025 13:53:56 +0800 Subject: [PATCH 18/25] doc: upload new diagram (#20338) --- docs/dev/src/images/architecture_20250127.png | Bin 0 -> 114969 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 docs/dev/src/images/architecture_20250127.png diff --git a/docs/dev/src/images/architecture_20250127.png b/docs/dev/src/images/architecture_20250127.png new file mode 100644 index 0000000000000000000000000000000000000000..703a38e83f356520c205e47aee621ce2f27d4968 GIT binary patch literal 114969 zcmY(qbzD_Xv_4EpN|$tZ3DPZ{lF}j4E!`ZD?(Xi8lnPu=Vq;E3XPEb%N z*pMG+sPxQt;D^vos?uUmRTIPq;D6xEKPrBNf~t){c{F+h1>IdGEAdg?9r`#6$&dQ8 z&kJDQY4Z3OwQbb#xEHx?sFrynSv*nvJ+cgzKRxmQwOGE?W*i(>uL$8USlBRGICPj) z`kGR56vv$JH4)@(dZ*A{P;CMq1a1}F7P~sD4?lO5S9O}Tnzwp(=KM)NE%&@W{OqEC z_TYQ%lYBxRh#`iU_WoyDMIoVTytj>(ko*oKI zYwM)KUZ4`ItjKeM@00BPwMd4`X;<0y8UTqRFt)I_jg2A@gQKTrCj~MnoI5Nncy!t~ z$WyPAs44?qX9G743xMGSHx(!XNYAE@1PYS4Yjz>zCs#hTCJ9^G7qLzlRSJ5 zS7hOuOr8F&CO_PCMRPJ^rn}M3!eB~bnk@1&M_yMtEe%h`6jw=Ox7;?c_RadJz8+%( zx}0{Jrp9I6Dsv(7Lu_Q&Ph+}>r`-gMD85}Tjphms2Bfh75jU}&MSaJGg~Dx;jPl0- z)%=I)aCud3jmJb}QtqTk12bx{fMW~)TWxnG({NI?0xcCPDk?0`cFV=D9H3+(NZ2PlEJTIfhA`0_J8H6 zN(`3s|9K{(4Ym+9!fdF2@=kp7#^OI2Lxhh=@!9{M6->YX9uL+b%*IIs^vAwkS^e+mmVP^r#|GLiPLV<1T zpFAK#6f*u>&(K`2*;|_bUMT!q5aj*8mg@gp4hsQRi;mzFz>&0Zp)>$%{{`|k?K>wj zq?rt#mKZR0e0gSZk&UX4ii3lLaz4N9hjhGx(k-hjhma5w8=&_Vz!mdSTughgZ;^5n z&HZoqr2phS5&-R8TR2PbyVj?Klbd_!>R`4IY*x$>RzeiAFj3MFV_I3EbK0HI*!w`y z00Xn{p$i@%Uc#d1BiLX3m7!@G5c@17#6l7p}S_GUH-ny`EWj0!-VFtGa zK-OgZpJNX366}*D;z>6j9%s@9#%>2VwdB=)y!U(vdR=)W`hSXuC`9P~T=01AV`pDt z&P;I)C$Yc5=-EPt2`{R|{rCnfBx>yc^t?r#_34M^EfU4GxSCo4FtCbRHU-_1`PGDL z|1)8*tSAdrD;On>9$6p)7=Xzs{~tlmiKtIO^e3CA0(rzyTLYs&437{phcS5k`-5r4 z@#=3DTiZ&jRGFGJ-T^hdju_xcOtTMfwhul+;}7NRAYW@0z{?O{62(J(qv0gA3s ziQ^Q<9^ch=j~~m=H1c@z4GcGJ5HI=Ps5wLp#eTYMT|J69Y&MxqlcMKX8<~`0*D8tR zU6SML*yD^Lnd4tyr_s`EsM&G_E|E2a`&^Est zlq`yo{Pqi1b9g7kPzv)kd=|q0GjE0WZ~bTq5gk(up9#|5lFJESO>N=gK9O^id2AbT z#1T7B`8z&?_U^VYJSlX6*i`QA!pXSX!wREPhQS_Z zPLnUAGSS$`oy4{PJt6!sro5I~(uBMDs6|<#ByeKVa#Z#32n=a*lR4=sm9abRYxWT| zxUw$`a`cuasQHwLderZ`umd&i6X5gR=flGK8_dRVmD zLBttK;PPtDU6k3l(At7YJS< zA_W&BTuVu|rxaK$8q#_h)e?MKLZULWqT~|&m3Ro4+RO9vpE>PAoiW^d;%Oqhr_sKW z!Hw`anUy~V2gA?1Q^ahk`fD{i?vOAN;;n$U2S!bOm#7>b35g183k#V_qRVkx7W^!> z8_b)$HR+`gtr+;P?{D(({Bu058p0UgeC9dMwK$y2vNFe8oMAfm zVAkG+aswTxqvZ^R*uH28C>X(Csk(f_D_nmP>JcfC(sN;!%w?CgRH72|jRm+TfH!VT zl+VsASYr^SjdqNk*q3Sly&c0S$_28wfITDx^EKC?NC=F+=VYf6h0d33z(CAHTkFKw zCKncKZr{&Gc?2PldFKK>kcxM0Q|d(GjEkiA7GD z%AAQsbkM)_%ZINdFb72`2FEEEWX~INT~tEER7)?@>u)G+S{iY^j@j4ot7@M%w#?JK z?`K~M^K)vu-bZhZ3rfY}Es%NwzOFJ`<0KMAAusKD9oLE$Nq2f?oq7g4MoW@}Rb6HZ zJrL8y*xz4HFMZYN{F0MbG41&G-FK*Hj#>0q-!gIhZ|mR)laQ<}y$*;*o4>?BMIpz6 zsdEw>Tet7}LQKiTPs7=AH52m-*#U!wVl8bnAOKBgH=-NEtw3ei@Nv^L?;0NqWHz3T zOK5M4YOgfs2)l(rhyuNV6)6v5wk}ImK{Gwi`LMDJ-^81>V3C; zf#jB>KZ)$QQ2ToRAk>Y>NH~X8&Dm6oQA_GdFtC8Q>dioGj$z)ARTEVY1%TksU%z$s zb^92p5bRI9QboTSMFm4r4Td84@7q$l$QFVucQ3yCm~k8UwVk$Vk`SfnyE6PTf7QC| zzwA(KgnEND-Ss4qY;)aR&{7fe16pV@My4VV$sv^5b#F4fqF^4M8Z$0jX4X4by!*mg z?A@PiRc6{~-mgJqrcpyX7I4fXFNZ&f#72)>_Fw=46pU1Igez|hzaH0lt1RAC(*7tA z)Kc#5l$c#xOPH*r7z@9mZ8aG2>Y#dSoK5ogR$g#f zRrQZ%W!}yvHSj7wbSi#5%DGP98@wp6jc{SqxHvnPSs1CUSd%LqlS%v?NYY(N)$;|J z=6wQbREAG>rLPzwCZb~Yf%=zKr>Fb+4Fu|K-nLPiNbMLvW2t2KEz@x3DP<$ zv<|9u!ow>_Z$Fr(=x?%_ONkx@BR>gPJMSgk^N2}sc5bDL3U65wQxLjtLDHUBpwWN9 zB>@qxXI~k4dG8%&1))S#@-ll?<7M~#6$b`iLkz25H$oiN4N_6=Q|fr$9FUv?Z@Njx zLgM$|2tBkVx+Qp)(Da9)RoWA$w+g!0i7tX9WEvy70+_~+`gZyO6s*>ou-1UHF zycL�C-CUg}lYWDGrq>zFckBL==_1A>D{kyTqqlK)XerA(_A(=VE{dM#A{|HF!g1 z%KI(2R)cmTfCg}?TTPHc?eJ&kW`;z4L><@_i#yrn7pIj(EO$CpIvCWik9l!U_{x)u&G5dX+ zM5Q4{5UR@WNBv2W7uZ_NmmwloVsP9{2rti$q`$fM{}=xl$VMBn*WZHii%EFgxEW27 z+^M`^K|ui++fmD!x?5=J=;%_Hy+wN~r|XrW9?z1|G)#)JH)@hsAUZfRogV#;OOj9F z1C%ggva4{#0 z`e5;8db-}}MbYl9#!Lb5WKnxlMi41}{r&tMvZbkEz9w+-AvA$=IO8ni;Ovz# zF!tWa9gJGjnXFU#SMVWvE~Xsj>IQ>B4i_WD--Wf-nmJdu*FVlqNKQD6rAjd3l6TDx zPEHb%lYfgLmZ8JL!x#0nOnk;#>!gioZ3bLGfxE^l|3h~UHP2jp&TeP%!qO%|^KkI$ z+g@47LYN1s3#@4hQb@$F`KX79iNrGYs?irNnZu_hgHuEAXYgmltN%j{h1Bo}@J_|o zx9Ertq@}71X##GIZ_&`?G&N&9U2-tunSW~QX0@%XFPBZ2X3iE0m!>@6v*>~4K#ewEq@$ zWRYZ%CqeIGGMuczbWG0Z`0KrnRTRtYcliUs(Xn+#k6m6Mb8G2m;OfbW+rV1a?kRU$ z;bX<7EHX>do?8<8PSL-T?5}GB8GOiC=>VM741LRBK|@a}zNf^pFEF&cIlPf5_G3d)~7Y zVj8DAbh=*i%W(5wXfUv#PWvB38i=8KtK3Ngo|_ioKRgnc7^;XA5!>I(8uDuQFP4i0_v!_t|L55k zM4tcm83XQNhzVoDoiG&m%P}|z{==7v-+{y1UoY`f)tg^l2ON_4hh8DQX`x15AhV@~ z|7%T4EQbO4|8?a57#R)*XY?iD&IB**?)A_{OP@-TOIOGeCU{rIB6QwEH20o6tu z`i}Y$4(_=RbX^?56A&~oB^x?F_PxxP40OG8D}NqfdkQPMd2zpN+XJa^`EGU{r%zpc zcws3R0Zd*}(G(e8rLl570~-|-BfFj=`pO@tK}lV^vh*+ORZ6378mJ?|Cj^j%Af{O` zub3%ef&Yw`q)mteC&RLgR8&+mtEv|BcB zpgq>Z&G<$M?t^X9ej7aB4|+Z2*u?w3DBfR!X@b}?lvgSQOq*S8bC5#AV7jksb5Y-V z+YP{lPoxiH0~kXp;rb=Qo9=ZfN70YZ7bTFIL8P|{47qfmJM`4{p9Prbev>Z4O_OJ_ zF~sYt3qSt_W60W&S$M(5g529E%|SZgc~w|PdA8JL8Q*XI7g@MxBd7kdo25EQ;*TI3 zQac_E5J*7GIo$q;5cFyH!d{joY$w#)7byS1RObgxTE(jgsAow*#1{{Lec6A}Cmn36 z6G3w(U<0I@Je2QNL|!_${*w0vPJ}MJ=1B^O7@hMC&N;ruc!17c{3b2KG1yGIgPkR& zos%(#zp}D$$Nd4~AVXST;9d#CayiW|mYmL7L$p6$94j48V_w+ukTbU!8ZfvrdzG$w-@I_bAY)F8%D(41gnv;i~C$1F%^ z`64}<&Uw+|I*Ke1y^>98gu_|JAL&E&zlx^1;Ml||2UI#sMCDYI#x3{bahogPY@p%5 zVsdIfh_tXp!#(gFWPPJm>k1e6W@Rl->gk}~>2w?WPX(~9Vlv=0D{!jOW8B`srtggH zB(48NSd3ycdcdEkagf$f6i>XS@ZuVIK(kiOp<}gWjF+DQgu=N7nIq&3*VV}Q!qpcG z^KWhwp(btHD_JIF=l!SMf3zx=^d?s>1CF!Td3Mru`D(Ke6^} zl0?oOUI@0)GfU9sNyUB8-t)=j_to;z^WB=)34_;Ji#h$cQy1VPn(u?Ze=jz~Ix>Z0LY({Wo!VsL?enn5^S7bt!BPCqM_j=3V&^pp-*f$k9g$8z$994F zB5MONkPuyKknZ{P?6Yr>-xuGWUjPQ4rz<>Jm<_Z?`XU9+7R=S*2h@0mXSxE|>82;2 zovKAH`5B*82t_4J`!5cwa@_|rX|8l_-h5oQ6W@{&R#JQ8SAqe7hdQbHEHJQ$9No^Sx2CCC)*8Cx0OiP`{|QU`h}mcs%lz=2ve2#**nD=fqx zUu2C&=j%+{A6J9|MkYbQgTYR_`p=!*9K2&wVl}Ky%QG`#k=$G3ztlfQA z&ae|(^Cou%=kvWWevpKz!lI?Ku$CrHA*eT{T6taM#VzM*^n7SbiT(#hg+Myl!e01p zN%_yiGpH5EStUU6LP*bQcC(`fd~Ac2&=^Les-9O}C^rdih+b}vD7d0OA@w|H)DH77|hNWF|R zRIXu73x0PH83A>7uM=!mrSBp(VOUT=i2(2H=Qy=&$SDaZx%%~lNH)(FJ6#&E9BKvO zjx(Om@RHyUj85`wxybK&t6&e3;I7*Fv*&;S;$y=t{AQwK?0~Hgr4e`A(HaZ7dRV>-oz6 z#-snYuwgvK;`nW+Lmt=Ra~CI*iq-S|n#^ya2|z$J%%u4_ijuqUO+zHmEBNklX3?DS z0d>hCP=GRW5Z{F)4sVm0GL3@qDAa^yt@k@DB@1N`#iLXu_Z5eZAAHS~Unr`GB@6JY zFr4l7)7koSFyZ=Mx45wYZ(wQEH3b>sQfq+(;!vnBn37`M4w2cfe!kq1dndA;y7gpn zT3U+gpd&3J?w!qU+$z|9h?aEvJwk+avZF?W5hZ8>oErfgk*Yp6tRe;+H9Qs68dkq% z6~O;!^3PkTG+SPCqE{azwr@$2GbV+d)Z|i6No$gLuJXLNh6bySQoMwZq>#m%S>J?? zFUyHbzYz}&dS6k=GG<{hHv2qv&z!1lcYIU!&}-fk}#BO*@9OSw;E z8E#g&`10jS4?w!G5k8ZV1QqQ9*qlu%^LFuVZTBPq!=36lghuq<*5_zb_SsRf z#uLo;Oy&?IqmwOKIrD4rFTa3(7uLp(8g&tHqfQxDT#);{S6`Buf(vPKa>1G!BA19?9hx=fw@(U| z`c$pPt*A!PFO0vwi__yeUtBxlVpMzU@pAqgNHbvGb^w4MaY2#LBnXE?77|?65z(ck zm6yDi&YH2oLlNy8U2o6KX9WoNm9{H?eMLmq>E;Hj=lAL-bT&0t;vpqzl^>4oqaWL) ze05T#aCo{K)l+KDlTDyGhv&r4tsJi>Eqbia2I&tx#}W5sgCXcahQspiC*;&4%~1;} z`s;{UUA!Dk6Dn82u#B;B0Cm!pg-&|9+@erv6KeAud^9+=;=|(sO46KnJBfI^Zi3iev!on1wSFI5-`H4s za3*{}O+&r@&>g^7d0MC(@p}mNi0|t!BgXFpFkHv)?VdnvNp4qdO8hEC0MNYVJl*m8 z8nimScd=)fOvw>=4eE2(C~VDH4EzFb4W{Vp@KW_xV(>Gs4Vm$M_F+UsSh9_u-eK@n z+KMsC-+5wpK2o3QBnrN4-!H#F%QAYY^Hy~HM2}u}Jp33`y||dDpST0S=Ti%78mh~2_3D9S zBK#Hkmz@BIko!5|$W3q$iVB`tgMKa;E{>LmEPShqtTTK1s(O(dA9I6!Vng2X@P zhKg=R4EgPTU#ppOP~`^L${2tGjOo&L5WolIf1=5)J79-)DVr7?t6mypX3n}~0nf7c zmvmt7t9`{%#pK{DJf)0UlnsoHJPwZP3bDxZ2|6b*QjA(0eviXiJ^F-jjViI7T+Kdu zU)K<~x+MUECw_S!r)4@psSMO?cYm9G`I9eU0sbV)R$kXEq+D{ol`q0vbZp4{!!T9F zHwB?6YAZYIFLnAqH*~>*`roWy&hg9u>^X>OI3t!V!EdCCMCk(FK}qb1lTh4;uOMuL z^MItq%Uibe;@v(b{_JH7C+Wlw?S)2N4qGlI$wnTSr!uU=bDlUBPb#I*2J}$*6JqcNo414S-=}cpjcP5Edg}A7e zBB0x)e1nxz{S;_&ycgDN06Eb$v5G%tb%tz7ssZXMbqy z1y+p>QJOx25^e*ic|3(z5tSnr3zu77dl#<#c6B zACC5GT;)!c;O>R{>cs2W)>!&<1TG?gK(8DQ@GcUIWU^Q#d~kN8xxON>Y&sD6smxs1 z(R&WB0IPZ|hAV~~hJCGTaja><;H2rPEBQq`JjKO5ER4ce-#CVlwdnZQjU>ZE`T4!$ zNKOLt`o)JeAHy=Ifp?da{;IdOMbTLoS#9%0>3e|APvr9YlUkEoz~s@6*Y6HghlivU zLpZQ7`;C?MBMU6{GD=F-v0(*mUMW}CH)WsP-;0Kxy7XmXtPLY+n6He~Y@4WM6yjt0|`)<3pLWeahKBCn$wB#G`^31 zNYkB{BC$3>rEJEqtS;s^O+2l_ZRZ@Iz#lfhQ$FJ+9 z%x&>z7~NSx6Srn-_Quh-Ym;!O&ukx*-v2`1I-$B?7zWG`vwD^>0)!jLg{zD-Cbo41 zC5MIgUx2hdE*ZHymO5+Y1vTXD-qBxEYHSK~PB!pTbm;7L>6b1AO#qkx&!A!tHT8k! zE(s#~UQ6ocsio6v&s@NJo*#&??2}lvZKa7Mnie3rZQKtWf_j28D~$YkcVpqZMq{5! zW<691Z#5YbwJmpQMs!$ta=b;f`6e@xxr&p(zFhVZ!pX^yykA3bX3i-L@mB97%!qeG z$>6RO_w?#ZCL$p@nVCsvURMl7KRL0bDf$C=j_qG5hsT`~%Xi|{e=oGYk$(F=Sj%=0 z_UuGnQgc?C?Dk=b~V|sk&eVUo*IQ zf7OC?hjlBu|HO^So-coWFr9VX@mF%pLoJKQx}oLX=+8JY_q1=|TnkRKEw%`on8A@mPa#0j}`rIrP{*`s#s-mXVz zem!TV3>LB#1{VCYt6izUTm6p?4$?k7@wYVj*=H* z-fSUA3^>gIxP70Cvw{YA>ATtt1^sQKVJ%!d%<74{qbn$Zlkpm;^6jNBUe2L zcq}E_AR8kKi$o6r&K_LfzmEp>+y`6*?HUt<^g&KMI@md>_R-RN#*g3)Y94r}z9r%| z7_bxK2yR^f%DB94o>)0THNFaT-n~yw7UO#Id1*Max18+ z<@B-ZDmv)CxF7Ge`b-^Xlyp{L5fb|4(7kgn?kjeXXR@+}FUdZ6!eKD;#N%DtsD6Ue zn-q-_G%KZFxcf;>LpXZU&hgHzfR(P~ct@f(Ehc@}t;5?!xF=)VsDqy0z{Zn(%+Kah zZ4#TL#Y(>ZQ6^UIK!P1Kpcs>Z-xNjJ;=k$Y-+sB1m$2Nj!`A#VDB3_p6tHC|eM`GH zPSRkX323yBz)WmgXf%@Y+5oY zF_&ub=Q3R@Xn_R@3hjjtxfm(xiVI?BRbUL1lzpZH~orZ>OWDu;{jzxv-)2S!MBG6700_ZJ|nI)~P;<$YuZ0 zw?_TI<K3sOjtmnN2t>NqoUFD~#6g3*T!%E+l0T{Jj|x^t&?xi5J>y+2 z_peLvSa_{=JQaa`dk|gA1}PDMo$-yVCnuR{-?lPSc0MRDyD$sI&ZDwlT5QlIJO$2i zG%FIAvGd81?#o6W4c`hz)8rpCq@rRlAW(ecth(_^{T9BK8WvMme2fpvYZR1B&~)kADx;l}8BUqY;|Mf?@eZh&x zu^Lt5yd9Dae@;^>RSfsbFqt$$q^0k~T_MP$hEa3p7T8wUJ&SJ_5S@eh^+4M&ka+%D z{Ww(kUzyD30`Mmke2>YhK~yGbNVj3rOQ1e}ao^x;F__vFR$92ox#D?eMdm5GC~V_L z2RZw?ON6kmu^bE?$a+kqQ>+e{Q^S!^is^4cXsw=N6Q8QxoT`F?sK4(!#2zpBFj73`TYO1__OM5s@iLY!(|1Px2X9+|R)b=Uuy+8XY& zad#L@kUTlNoT*xj?4dCZd5wlHd5{7+HLL}wq5gkL*fl|NaNYjE7bsv*cPPcH6rYOa zhDN;t%YQJad_p34^}1l01`8ap76W4$Rlu67x~+WBcjFOPyv2R1c6d(A@%Z zltj7dgOaOno+$1WG|J5DO@uWQ!oL4Iw=E-V*kmBYLid5@@lb^mAj>4q-*lQ0%-v#~ z+%^9xQ;y*LXs;|k1y*jv>8bwA;z~)h*ISHyFOrYxv5xQ0^rw3=NmIDN`%Umyk0>{i zA{(Wa(PWOm9Mj|chbC21N;RT?2TkC4wxCH=bAO@wnFS6=U{*R)^YPOj;FfOdWsmzi`bF$_dFCsL4$}gQIanv1hTwx^?h_E4>60)oN=lh zHg$(`$^4{Q>ceK)l|5*GgjQQ|i9vVOMk@Csm>1@1+_xdBEKfKPttj9*^jcjJTCR0n z-P&+xd>GF}*gWC*?iHFAGN8Yftj^e4al%4n0Afy(svgx{Rw@|{ph%J<@W|Mikjg$= z-5KY|WA>a|W!vqX$+}KAjkfC|bsJ5>ebR`_=t`=K2>_Zzoj8mh?^4AN2L}1tl#nhy zNZNa8)igf-oM|++0((F#T*9iH$y4{bUu^GOMTv!&(n=}5gX%fc7p&-1Sp!&6&aXpH z3L|EzxJFW9N}LYDQ>tnZP~=TWfC4_{l(Dy8axCcRP()-u`+>nUJ5h5oPV(~yKoqs` z+0x!NNys&WrENT$0P9eDvPuM;#i-H%J4O;#^0n?<$eax71N3r2w{NXC6 z_j1Dpb3UJ0HWzv_s3<`GCG-V{j3LqAWXr!1MOBp{VC_jCyL6|Stoo4J^zZMNw%^Z; z{&b&)N!wEYV&8VcdhPiVBJ3QU4(LN|DxdtiWuD{HN|o3!TG`Y(s%iar+C~=7v$n(v zt{wY6mci4sxfLUPU;ns)PPDURjGppzvr*;j(wWSEaw4pPl|E z%7Ow*ODj1$qQj}F8ZiAKqv{_!Gp(XSGvE3x#Q9~=Ce7yRhi~4w5tDXS+M8;vRxKpv zE|BF~wN=*N`Gx%_8iKBm@wXkcw|Dr%g|i$T1uBg;Yx!{XrlZM;-nNv1%~!X#Ow)ks zjvHn1Wxne4jEo{e>L*ZwdbGv%2WAqD>W6@_90B>8^sctpvO(*WwM3pqx8WXp-HJmG ze!_GZ=O>7DV!eMeVzD#ZVYFPE1#9tAP%2t(=oRIukv1x^h>4FBNdab&#Kpyz$B_+~ z|1CkaY=};jPO%$zc0whdRtasdEa5Vh3?m7MGFv^X)PZJNM${qDkOPgZ_Z&VE5e+L{*orkuchbV%rllfsP zB^;;cw$5{lMZ49)-qPM0N%GevEkEfU@RMpOAG!q_qE5n`(=>}CeWC#7-Si|9%`Gaq zEF5S}PY$nfc|C*M2Fxa>Cw_*aCmP1t!{j*~e4^|!|sV~hg9|3C$tZy4<@ zuYq}t+35wwj;8=QFG~xV+6g^#Wj?AxsA3;@*6-FN7X+Q^2pzVytshmrX@k1izb_8$ zj;GOBFWRe4U)ou-QmBO}3j9)gF^Cz=Ov%%s(^^D|XTJ5$QTmn*3Ofg1D&U=Ew4-HopGPDT;!SL5%##9Nz<-vOv zB6!p-%0g7s=*3;%__N|p{s?gtI1vg+lA8t?!>e5CuB}!LK$I0`=-aAh1^wb(S1?QS zQ40$%E5Qpj?1;0mG3<`{w%Tf`M z%j-E9QNpx8e;0OjNGgDXC{}=F+eWmx>BxfNgI?Wl`<6EetepjGobTLFVyF8?yK?=X%OSq96pBQXc>(9jcFI(XgnF@RMe3~ zf{w*8P3->HH`0&xb2l330qbl2FZ0S@)S$FDIB=Z@^>>h5EKy~+jD;()G^6+cX0F-< z`~1**Q4YycTy%>kj2|-jrxh#}1-K1?HuBA@!`LF1af!B5xI9k8)MXqzZZ~6r+PfI4 zWZl690c`&9jF=rFQ9-CHbrZ@4K0am{G^4k9#2TZ`gFz>6?2yV}-3KtS#J^F^;R`j6 z#gD``)K%qtW-JP{XxxqWEYT3m6|eU31UPRisuqMLsN2WaEDKm3=M*Kt9`#&HX4As%UX*9q1 zok$>gZh%E#@|mKfA)_kAam7L3FcCu;;4id3lgdJN4SdL{1e4#2TDr#n7fe})wMtbP;;QkyZ z_mx{B<7qQ-G{v4M(VWM0du6X_6xcDE(8zr}G*V)JoD%cw)`gWPYilIUZ?|#z)dv#d z-zlJ5aIq$BB-byD!_ra0*EPVz>P#+Pzxc7|=KHq~WrJJJbjX zt*DmYpCqF8UQA^Dkt@tv_^sml8~ov%f-f>OJsmjqf$*eL#$M!i(^P%HVKv)c1~pLJj4FC3{?|1XDFL8soMIzz&D466Bjtwcb8ifwC3!0Wg zNOtoQm1mWan$NUdl1u1MI({GlufF`-Vi4nQ1~;DNF>40`cF%p zyn&BP$5^(Iyd<4w#wcr>MA&rF7w{bojmlOfr0Mhsw$9wIAWg!$%WovSV)5u;68|F9 zHFEotQ;5x*DtDCIRW(rW)A{|;&QL+Ls}Av5d_j!xa!%Ke66w>nTLVx5RZOB#R()Zo z16eEcmH@@IY^8^2270|LwRE%M&<+)b=%0P_CvMaDD!{aq6^|8@Hs z_I@*l|# zwJ4)gmN4*UphC-bX44(GtbGaFa-od6f1dCY<(^Q-4F24}15 zlq}xO*CmCp(?fHEKU=qpn@}3tkxUZq0#mv|xtHxXhffg`lcqvjfNjT z%NC-jIE&LoIjL2&)a5zdw}nm z$j%nvS`+wt67UIWZvAP(ZTk^rB4y&BNN6da2Ae0M6n!3vjyPkMLl<>ZuxRs}z1(dX zp0;1c8aSP-(D^j(edR8cznarw97)0$TPDgPDUxAJ#R}J-o>>r|W7a+_@ma*#@;IYQY#JS_^Vgj*pMmT7qNWW;ck($I~&t zW4IqUZiqTEt^^lF$Ri@myy8{oIAnmzk?BfkJJX{(V22P$py}?)xcd9J)I@3)D2-B7 z%r0;&?t>4*BC&4KaSg^wW&FH;%BYy{4xF=yvqx(>rd0Y$@eKh_QH-{Ep7qusqafxF zv80qdq8?v<=NGgRW+ znNv$tl#WSgREH+~@2*t>c2=%X5%M2ihgt8#vib5pcptUCs7J6}eCAPj%b(+kmwO$z zZ3W$47Jm=fjJZ@^QS~k3_{eakf~nVjO)Kv}b_2X8f5kBqd*{+$ z2tLAyS0`t^Jv~&*y8Wecwt|l^(S^b$sEHW1m16C~=wd35am*mwQPyAZn3`a9>*1lqzdRfJxughdC5IS7-V_TKkNpmac~ zjNdC7Zx^ixlQLgK8832BH`y#1ROqhwjiuu*-AWui$pl1W*LH-)E~qRTe?+iu8H|kO zhm_Q^H?!0YSd-zF=6?Lv`8QyXp_ZWA3Knx3qX3#RXR-7Adw6v4dwp3li<3o@wK;Oc zt>8jkV+Z5mJ;|UU(^b(cYti%k8DAxkU{U^G+IJmgd4l-*&i<`N>7Z^A6iY>W#?VN9#%;aw?Nkk3Q4jFj*Tk0w+*|{ zBs5`r`q3{=Fq0vLxO`SwI1QUJXm&{=meYGn*CZs7zMqf}oOKAX-;q>&O#GG?eGf6( zzaZ(&rJ1ax5}v}(iB90_la~Bv`*@ai{kM86~()|o7xb5 za%i+7*sxV^-b!d0<_bR?D7~mzwB7_Qh`u9553|ely z))QXZcJTNPwiA-(2QDXt=tMfM@#ZNu^xdB2G>AJXO(s+PzOKd^2lTg%gD2wjZJ zz=h+fYJ_Hgn`6H}oM(Z+8u4$|Ja!Cm)SJJb`J&9GO2-PCtOU=KFO;19( zd8&!^qARJ{@A?j@vM)KAZfgjMB%wzR zJ~@#HQX-4^;Ow3n-pAOvYqa@jWTnwo4ReZ&_g0S&Tx2j;4*(r%?kp8V#}`EN>7eP> zilJ^n^H58mWn_XcQB(4~*VS8THbnc#n0pbWa|HGgTos>~9uZ+zT1` zmSwA+a$tDIeC}nN*}SH{oF~@*H1@%eZsth%to?6;uJdV)$#Cc4F7oPCQz@dI&4&@c zjOvLMA6|Egt9uOK+$hx_TupEAF>{fCX}6%DR_aR3*>p&RDrm)jka~WVe&L&31-z*! zlg0=Tx0X*MVyr>Ow0e&7cbic43&j5Uq0RHq6Xo zs8XS^vGq0x;j}{u96D*q9xpFZSog?YmerX*oprr2OGoHAP1ax)_IM1|4-G0=4QGTr5Ss}j45i-9{5}C55IvYa-Bd8A>l)bX zfD49Av4J~v5$l@DG0w1+z1Mo~LIu%LP&6c26%PV4*+cs!NM9S^ug&2AE%xhZWjslW z)-m3h?wjo8o#hYuy+09SXRb~32k{+pvTD%O{1*JpPgi( zg#J=QT6fr*@f~kPSl2WFiCYld38V=cioNfY=7X_*KsxkljayeACS%j;%w)N=FRwnxHUFvc?O@i?yYsXo6-XdqZ$G_d8wy4x+O9xv zj{1pl&_J(`y!N2qi6o5`P4&k@0^Ne6s3z;ubD!qcYs5p!E_GvWb#K_@Bf15Q!7BXp zf=``oOd129gJnXvBYcjB(hovl+$DSpCbZDP$W6&7+t1UZn$?8BQPZD2n8lZpynVe} z0{G5DbV@ADkNK=&zL8@SFPyEyq5LI{X`=eK%hp}hr@GA|M}O&2?`?aXo!1cbph2*z zZ}rWk-ls$}a>JINBe4_vo041MhqZDC8ig*TVxZi!HhUPMf7QzvE2DBDZq zp*l2=dgzUN^E-L-FwLg_kEXMVi~5VUxCoL`Qqm<2BHf*WfP{2N4&B{IN_Tf7-93nO z4V^HI$X?6dY>-$i-EEy&IE(j68i3(>~Vb3ArrCIdH0H86q8 zr1$3!pog%5O^Um_eOWJ~SyBAL&zN7mWeVU8h8t3O&C5RHezB}L1C(0W7GcxB#>*nc z*0oa7FqsJ2x%9bHHzw=A5&QS!$%#E3O&SSD%wwyakCd|D zT@L6X4uU#Gn(cF*IbZp$J)R-tPzqb`gbmEr$~r_rdt|81GQ@8#sk0yqiyqJG z5og1|B_GT3$}mTf?=gNtZhpSQ*sFwUci8IK67LC3;_h>qtgW~)lhfp>crT7vcfTL< z4dnJVqTVP$>A16VBe68jn}>1_+Aq}^UOVj-z)bzf2$%uY;@G;0ed0~HfcO4?zecmwk@mr9J zpd|}>zscOh+GF31o{Iq_kCMAD7a+q;`eH;o4t!IDKhLat>v zH-N!rgeBnY(Fe{n2Z0sw`DW zHuXteIYGocU!Yog9}HJI|6E@6u`^TOH&%x$TiIZ(1S#>B7!gQOOc`bwo_)<=DMa=U zG|oZ_g;TMMAF)fKYbx z<$SWu-HV`N?Dc@2t|t4OY|o1k&SP%o=d*`JCZV^ec{pFG#kU{u;zVkswIaH4VtZ~J z7DvmTpSctmyP<&A;VPX*a;Mu*cQHv=FzQi4uqgNSN8PSrTz9Quyg0EM5}xLgfyP+W zhu?lTV+Vk2YAMiVO6%$`!QG%&YrFan0lrVz$!-h6a%bH~u|g?<(GAp4#GCcn3z6S= z3DocU4iWuCzdx%yL+Amd>}p4xkOGM-2EO&G)X-O$s!umMZT{VtJ&%PQD-sc! z$4k_~u~chNvP!!L*3?);toT$BLIo3IU(Mp1MbM@wnV8s9nEYZ)EXgBQf)&HC^dFdm zbub9I`PTQgP?ksI^=xF3TGeuT>WRb+#SdlVga<@f*UQg}l`j^CqMOTR!dtcl=25$P z3Vs2;EyUYKhNcV?T+#UdjDT;>L?q20SzVF#P$;~#Ywk`wc27DrD`Js_L=>^4r2=+e z44B}Bu#mRugdyPRyhkHbnJmLH4^^hVmCnQ0Iu8o^)S;Kv^uRhAg28f73 z<>XhtnTc5AoDa3)c^%gG9MqTjJ=CO^j_y6`7gpUXr<*q%!{p>z?<6k)DE_pRw(Ba& zR?O^bx$&c4@4ua=E;pro;P zMf|ESdRIksQO%yIqTeA2yC}gN&0|Mj0qG5z?0)1HbI(=9j>sZRQ%T<3r1si>M%i%? zgIUpWnv!;qap3XgkusBqBFSps(YG%JEBGZNtVh^ zEY`}Pc>e-g{v%71OB95oI4N@6_a<^t(hhUw!k7A=gxOMTvzjj}T5}CErl1-*_ zs9TI@GCi>B;y(_q-)pXWS6(cM&P5?PxJ$R^ZMbK>%SX7LMPtY*2GXu2#SYtn8JH5h z^9FE0X}`Qs$l6#xh|1lV7FapWDvAuEmkgnZf9O;p++puCIZ}sCK+=-%Qg2HxSeM!&0*Z;wTL(qj_TIItc}R6tE| zxkjt*gmNOmS~c=kZz8bPyRU%;JN?7?Kbbwuks-vm-mGgpLrBZdoy31CBFulk7iHJ( z;k1@H1BVeXo=&|Rh=uhjT|8dofOV@=f05sw>(rK|^Q9ms3>g@#`Fov{AGk?#kH!i&p%50<49&ia+IKqE^K_HI znR)>Ve@U|F%{B_ zELvepPxLcZfwqeMxwH=5oS@{{%erUdxa$cb3ah~$T3K}h6eF0?Slin5^f-|x_6(ha z=ry~vIV&QyXD*z-V~^U6laG)pbiR}Hcx3~|Fwk73ymZ#aJ|Sq`c;|{0-F2hWqJ@7XKzblJp!#s+TK896V6Kf# z;nU(n|C-9j2&#bVd&=SYjmOl6J$@40b&%-PP~ZcGUV#greeNC8bLLiVUey!_nN`53 z%cI_9U@i^Sir1n;USdLyP;y;W(17fbXjBO-U#j03*m_(a{;om@TU1&`qReLv_>B_; zSVTP0bAUduSGSk%kKaQ%JNDcHR!=Bg!w?c;W7G)!dfb%{5wq1zO{(Pf5k%a)9#fhQ z9UZ4v!y;uwJ8x?Lvel}Q>?-a)PeNM3>C37H99*EnKgDEYpI|DCv$eaXoE@NZ9K zMOAlB!PrH2p==*DKQK&1EO2xK)L${lu3Rvi>9k0*aWhe^x>Q1IjxIVQ@_#tZ~ zvq;TZDvJDp=QO!(*}Wc#?h6GmF&mrl$PuV9K8LF$dH`1TQGi*DghP9V^Dd3C{n6`< zjF$W1(Gk1V-HksVR8}G_m@O#YJRF9X5I9NZ#Qza z46(eW3FQzk4|N-iI$9VFXTLAUAC}e$5HAjWAI)7Ii_w;37LqkJFn0!msiO&+joAA)?(9$|C-=%BlOw|-Ct(Ni>o zNHYq_3>2Nt$Dn zH2PhK=~P|*-wZAub#pP}YQX+cOW0Kx>HIG8#iJyW_kO%glx^dF@Y_XoWe@Ftvc>c5 zw)xZj6?_(s5>{Bmka!Lx)EI3jsVd6mj2=R))Mq|l!H@kwX6kvZT%s?8m2g4XrEImX zxT3B!ig!Ya$*53!mRR8V->R}@l^C%~m+Lqi*H zuYhUT(mf;a@crTs$h4GpcGYEq{A&MFA=(~F^DCGU0A3{B9yL4QV;*PRmp2R@vy&A> zp=knQ-cK(IcE3+pFhP&Arx$Z3`Gy;c(TNg~h2v-7G?z+z=55$5n@ZWDzl%@{*$t<( zu&~b;V!q&8!qo~VH@?=SoB8992$_Yp)?Cio>7Tkc5yhUgB>zUwdMkZ;uIl;n701 zIvBG~1^e4k3(W&()dd`NMM4)y>33ZJYt?3{ta4M^;1 zT#$lFREs9hIsaDJJ$F4`GcUq74Ap??2NnBGJ6%?DXuPLupU~vC_dlK-u0FLmGs)ds z-k#?UIJ|Dui%36zP$gjU;NnaQ`vK;W$ESi>46J|Zxo*>QjZ3ides~z$1aD4mY#s3$8Odmpv4gPX~oa94l8HlMHozrm6&w2{x8-2=E3(T4V?PCY=vp0q_S z!fbQiY;Iu@r@QvcKoCZ+SMQHE7e+77l>Qp$Rsf(}$;RJ^NqAfS@or4)Ok$}O zLC!7ba*I#mc4A1LJ09agvFcWnf)Fw~@qnAG4b9m!Hoa0LZ`Qb2v;gPX3DEU~mgJ{B zpHAsipkGGh7V!R;!-$dLJ(`Qh$n+3-(dhGc@N!{S+d4w%t8AAZ3l7B^#K21jYO)yV|K&J zbbRz-8Zl?2-c5hYv01@2P2;li&eLUxx{z~`q!GalxDP>?wa4hCgUn+|f>YD|WR>ZZ z|Ngz#%Q9f(Vs~tJ#il0Z26)qbvCQskI6U`vdVX>&DC#CLb&uZ4($X~y`Ltk+M9USq z9|D%+9bVO_^sTsZoOd$jaJ1!vuU*fiE8tFvAQ~%1SZdfaO}F25B^1G;b~pVa6Xfg- z2jC8yXW!kQS!Ix;11HtRX)j%~NeG?6A&_xLJ*U=p;39gD;0De3+p?atG~M_j>{CCR zQPpZ4=r_*oXYfVV5QSDDuk?v?ZFL8&xb)K&Ug1^njR+Ny;iD=KWee#_E}|EFcIl@I zblugGH0No>{!UsC6;ug?pm&RDAl(+lRk-5(-fkINb!Muee@$DA!!k~b?oopl|b$k=Byyy+G1s?R74o6 z$i8sj_QChe^Jv7)cJ`>=l&;v0%b6d$B<|M(u~4C$!C!0U!eFA^iGpa_NIlLX0Xp}^ zKr?b`5~w(46tYZ*eR6} z>n5E=**UI9KRG$M9BSxXvk|G5LDn30Iad$4n+1d{Mz!i|JfdcJBC{2K4A(w@;#HWbQBip%#rJx)rM(a~+EUIZo~$ z(djtVA!K~PKYenqeY}{>xubPcYy%Q1>RD6NED;bC47EHuC|7qMX zdo9;H-z~1dsTgVsa<|khfW~pyJWAm}(L*bKB)QaX?hC*DK7v7BevC7foseV`7r&%y z6wIecUE(cC#Ki2i*l&Uo2q=YK&QmtowZkLLw(}rBgJjqgv^8UmI$5{<-Z{*_Gx^5s z*VvOD%I-l9?cu{^Y4Y)UW@7i=cs$)~mRVRAy?HPQrL0?zJ~A>V;H{q`Qa4;%&&wiZ zp?o^t0?sFoBrg(SWhiL>DtL{ccXF=gP0L^L4>y$<{yR;9`MS<^w4&@+&=-SMzJmFU z?1=XB)vp_;5mQDUNS4x4Jw;Z)@p^Q*#S4zOFZlnEBhAuupw|YE$^x>S)Pz zbIl{K_@`Jr2So!ycf8S>zRfC^N!(lQqDLH4v5V4TdS7eqNHoEOSC&PweDlN}pCXI`!!Ihf)!)g-7KoGUAZd&Y5iicy03RVWmD zs6FDkK0=yVaV4#T0$v=gG|BN~)M)Jv8Tc7l12v`ODJyTQbbux@>t6fXIED-P8W=z8 ztIB4OhW!3^GO2YvZm*@KgiuUf4S|*U4O>JmB3w^b=&Vj>Ua`cSL<{{W-?xf=zqUhG zL@Zanes$?E9~g_I&ZcCYNpIa~{3C5*irN@ps-XMeV@`8GJQCIqVeA97h<3YZ$L4IJ zcG1!;IdV_9e3XWawgg$UZ)T*K@#i>+8l{MdImtuyiC@|VT$$oNLqPH$FWb@BIG z)!%v&Q-ee^AgFzf!Vj2u(lR{XyA~3;*na^OFc&l+Q zH?1XPkVR<#BxZPlhagCag4hq4&w##}ZTKAe1wZdrwP)_N>f~}U zTKnCJ)JUh;fR9$v;X8utq@`F1?EQPwPFfNvZDk=8vrgJi<94q7TcyeO>y~5L|h`y*+jTX!)DI2czqYAL<{Rj$)BTM#h{0c z#E|jQIt>AgITFt7%17WflPkG7d&Kq0Ge24GCKvPg-YOQdtZTB)3)GGk3&gEve>HZq zF%H%IzFyxhfN|&X?ka`D+I{9_shl}%Dqpy>QZdyH6klXP&Xz+C;t;zPDm&`~VF^l} z_**p!-|0TT3O?z5)TG=Fg>1|fEJpdC*j7JwFL$LK^2J;Q3SJv_P9!X@uTb~V(l$IC z>u?q&xszRCU8X!Elx-S=vpC^+BcBsdz4-(-mR%dpliy3r|24qK77*oElpVX(-6f$w z4YP4eF%~^nln|P|NVpRxyh^^hCd7oFIIO{e5wjtd02;U*do5r#IrKdnj*i4Pm^(Pbq2oVOhT?uWcRJnMH#a1`pqi{CxxmybMUn(@o{zyw)feLc#={Au$da%14W`9 zp==UPzvnmllpLb!&EUxe03|1_yEWWD_ZP*DjO#R-9R!U%dW~t3(XXw|TiIsyd=IMf z*`!avMw-p_vxtF1`9qQR0ptnybp$LLUVEqY;)8#}u1Jelq($;4V*L_vgFar|ZmXyV z)^J^m9Lg{6b00@9NhEU}^U9Ldw=5FtW^a(IPP2hrJfmE?(@%U)T}tCtPEv~L=oHEq zQ>n#ChX5kxs#94kTCZ+Kq+i$HiTE^)j4mS2-A3;vzsuvpz8dWlS|7*FQ)}SNU!R zQX|)cklSgy%DX#B`moq=oZZ!D?gEhjeAZ#26D(81Jkhd8>d^LY`tZ zt8jPq_U!x7z1mYRZvLiH)6I{y(0Daff6Y#B^N~My)GFuJG9N(+n#A#<%bO8X&;@R| z2Qdi6l}+0bDFvTpy4y;n1l1B1^!|5dnHGzx=k%8NtDyJM2vjER&CT<;F^kNqx@e0a zkk?lI(EMXjoRH?ves2K+k#4)JQS3tARGwb`g>c^l3-ChquVJoiVP3%>=b}+WUbC!a z(OmN$jjXR!r@hH>#(3nT_N@(S7;(V$+3%LIJqGM@%GRHGCWVWxV+4Sf8-8m|0ov>~ znzfqX{8k}PsKDy0_z&0MxGk6NdRWi{`ik9tx#f~elTQwQp}Is;737o*(1;roE4#vG zV-&x9Dy&1F2yd%)wjVr3{q(|jcHUH9gg@JRc|!E)*k7`0$ejn*eTE$j!MUwPy+zk6 zN)}DLip}M7cL0ox!w|OWzp$un8oC%mT8zEERD_$wWQS0zD2VR9apiqGwO3?-3Kxs+ zvjW}q-^Q*>=9}_V7B%hKE52S-$j8syF{=R>Yms)as2b@N8RZY#y;HoSL`+#a$7kqQ z{>kz`-sKTrZn=A2N#vSDoJlYQTl!ks+G+$-EN*dyLhgfPnM2<17_LTR(D+vDffA}6 zpz&#Zz#3Kp)DaYlY{zKfn+xydJu)sIP!6=|tn?9eKR&v)P`QCIyuL1F@5s_Kzzh-EZ{1>?8iciO@?kdvB^ z&7u*1h8^HNHD)!eM(9N2&ZNDd!1$Xzo2{7R>62`Sk$_|~OO5ICsGhz053bSriv{r9 z1!GTpm!k7-o@tzjzP#VL>iJ?d2&MUlcj0;)_GfQ~=|#ACyk|4BJ3NctRp3%borev|o_1yp zzQj%<^BDxUMt4>z&;ErZsadJbe6P|gk@{fItWB>S*V-juzf)0`1-CQvc<)-GPdYd* z0{oG!NS~l?Fw{}&RE)empfl{eg)8M^jBwp3X8oCC42*xArx7e>JgueoTh196Hy)UbGl$v8&NQ=YDzrpPU-N3!`y>uG7 z#RaV>t`H=%2|PvLlskPVtk#Mi0 zD3+*LdkPXV4J+I?F^@Po)wG3rVBMT6AKY1G;)QAhhP;mB)5j|h52(iU#L`D{bIb%C z8FL5lo&z2dfKJ?0Xm9eSS|OFfGe6UQD2$S>j2dyE48_&2uqay(F0jZP^WqS!QT~iP z0qrc0dn7{8mhhM6(2R(l$j>tZyhx1e180T_(_-nagi zKm)=qFN-}ZSq_wqW|227v>nlRgo*7nmP66EzhF+)tbZid_Fx;@ri>TeB7i*|tIC@1 zU{^}LUeRzza#RqLiqEv_REIEr!4bkQ{E+wQY_@I09Ym5h$UheN!S-i<6{W4MzqmKG z3ZPwHd+X_i2W&4ic`&%1A?w3cq9y^{QyTjMv6ox&Vb;yPC2h|dY^LI1zjjv^cx?5q z`W>a{Am{(W#L+T8CUA&izRQc6&L`bCg7N5!8aTONjW0zA^jvk8`*(o<>H4AvBc zd7YqQEqAF@HS*mHt(ZY|F~c@Sj&O|OFRuPm{VO(t986;pSmXw&%|9D7r3FLC{wK#_ z)yy0QyTzO3y21r zmurnU#oS^axZo-*`K1%Z0jCNk*vS(Z(&Bl7eb(h{ZW-c!F@LL zPA8>Nx}MZnszL(S>vBIVO!DZHeYXGdwu5I%zd>4rSFqiCm$PuqW<|xAs~sELPt$zr ztwcs$Tl4R|OXQ=#)-3e9rJ4G;f9MW&^91MM*5<^B%Bc?%h`K4?jw5jY?z~F4xxC56 z%_L;dHYEu$yOoi8oKNUazwY0TtFU783OVtmiYw^-{f^c)4p*#Z;CMHjP0NH!8bDwG zO@1>y`CjZ`jp5-gV8@h*sepXe_#lY!$d8aMu~-WJVA;;c_W#=Pe1@|8xRV6Lv6B^O z^--O!+p$zJ&ZX<@Qbv`~Q;>WH-F4+t3r~t&e$9Zqy%o*);$WM)&bg7FeU#ij;N{Of z0GQ%w`b>`ww^-=*%TRP8e1GR1@g7rplt2*nb8wTYX*3%S_(FiMf1@7F;*d>MsQ0F5Eb7GdcjW0eK3`gx zZpaf)rN)-~(qCmv!&8n$rh?tq9HPE`Ki7rFFVi>2X4bpq9J-Atj`qCcoT8`xlf7pc{rJ-mU4`v-gVCGZ~0SnH1A<(N<|UX)8S z#IVD*LfCQR$$4w!^s%y& zbMkaGyH|HL+vi-`uOCk?UXx9v9^tMfICtkL&9qe06mN$>4lsyDtNd5K6Fu**wnonQSz~o107_7l7eJE$^C_d^g-h3SYRup2i`|wGK@_-3Zc&dC$)V4 zge?@74;LLoXUGKR#uX6@U+A|V)Y^#*g?O>Qfgvt0^GE1v7^x9FZ=Yw8)6{UDRnoW)e*j zT4FtKu9nCU_yPc?r!;o|Y8;htpEY7$^WX(>8gQ=%Y zJSxHF$1vu>ObRMksUqw2ZzK=j!%P5?6DK!S8*1-k0#m;>)n(Flq-dBt=5E>7oWHhm zlvu3(IZdh>Uuzi9mYCsSa9>&o8?hURi5#{d{=-+`7~*xFxD-;_q=H1|2ImA~}5iEXI6xl$A+b4(buny<^r zbabpXG`GY;1%vGR8lEo(Bv1?A-3-V;+KKfVIZ8r|Mua7B&0=mGhrJaft>vt9- z2VxC3KofxbaA_`s`FGc{FAf>(F5t9sV^(}Hkl(&-Q?vc-kBfO1+w}fop9Vr(gXy)ZzRVvr zxuuJt--XCsiFSwy6N(-Lv?2)EYi;-WX78kL0;pyHez*oIi__;4{jDK;8T}%0yBHB% zuzW@XKe&HRo}!SbA>SOANC_%Irx3Aluw7`@moVmkvJTLgi(Y3Ourz;b6u3Pjvq&`2 zI2Afyb&CwPN3Yx*^>p=f;=5I21hQ>9A6Tux1U9ZStk_t(fWAMJJPq2xhoGoQF<}bG zxxNihvzZn>7E_e!!#_v8vbT>tAmoq%cFv>MIF97XfHaUIUBIZFT8=I-x;x1xEU0J*NbyaJ?xMp!FPew;e+McDb$ zcx_u=?IE7r!Ff3tTC9^*-La)0t;6vWlZYSU4A>}F3F|W<6vVakqrK7WMxZg0*E)Xm zpG8}^zqRyDu4>0SYcMA8RR)W$La%~jyo3Fvx(5F|Tk0NnW5cgm>BWMysG6b3rniS) znUdx|?c@A+NAw3D$6LF=k;H2%?YZjHTIu6t)^O)O40GWri?b7L%@4 z{O38Z!z`(VQ}W*vy_uvnA@Xz!Tsyq@Lur`Ql4`LGi1V76UZ9VOo~zaKzQ5m{AYZ$ht^?A#|O08QYlyQ z%FOHw{(#01O}LEVf5CWc{1EPV4-dH#&&KQrKwIs8M8NVxJKC1WK1ykK@yPAjQxNm$LJx7k3IQzC?pHDjZ?7K>lHx|XUZ zaKp<_J%kw^Sv~FILR(Fw{JUYp%}EOx|69=CbZa;TcGt!+hVK`uomf z$=HEN()wq;m=w|<9^{Bg7z6~LTBX@5N+pg(TajCP+JUn!9$83t;-79(=!>R~G!%uv46Yp^!2Z}3~;)WRQbZGbhZfP}i;R4Y`Pm<+C*O5XTJ}I( zMk>sqb(En&wZW=XjcRXIw=}yks_{j_jzt?L{AxJlbBnh1#=8}35es+N8dfH zu@%8_25%g6;b08)7+BorL6+&}ST7|EpHn)ikl5zGs4Gf}lS7uaF}#_Ftn=LD8C9X? z4&Dp;t(Es3G);DpRX#V{M!PCKrAEK)w{#!C5}UqGI_6NO4NXRs8{m&lgJE2$4ZfIu zZQ+u%mT2T}4hDPvj6cuVCN@foH5&YHgwUsoMKi#wzru8pA(U?{)v(k3hi+Hb=V`fw z%2a)GfBfY;J%FZFJJ_d%!uFUag8sO;ZTh@{XJ1_BjzT0t5pt||O2?pPOoEeLOBu5m zy95Rv`WqBJ+v01<8k%3HJSd_70=%Yl(u3jxp&p%)vEt z_^W9j0JNi)R631e%L_l5RJ>Iw@+gd1VF{VFKiCscsJ&LxAn%^qJJc)6Kt>jFt@jJ4LVRh(%2S{Acg zV^an0B_$de!?%vkM{zYD@jEhqYuuj{xbm`m4h>*iKRk5@2@|1qq>@K8>7o*fls-6# zqk}K{4fTHX2;!OIc4L9juiF}?Bhe(Zy5Fs=`#N|*J&j3o-e);1wHbp=!pFsaoLLc& zvq@;Qv|yhHXJT|M7pJK-oCHHnF1e_aP$tFHu=n=?V4M1|S0envRmbm#=ZcG}zj;5P z&?TXHA5HaI7GF5lzEfk+P&ljEwJfiX+(>FxiuR<3kN)OqK>ihb_0B_zOaXJe3Nm63 zTy!6Bj^rU#&*PaU&A7X^KGPFAdU%@|FLCR@Mo*!X^N>2#N6E6 z@`6bT#fhH*#T85e*_v=mJQScu#@0yXb{?!z(La`_O-mnt(T* zoJ4Y!PWRwE3tQBAf7b?vFIhhtDetQKt-(a4vpQ^QQS5|(Xv@Pd3sJEPCatlB3uF7J zkZAIYFj)eA=!2M^Aw`0-&#dI@=6dK}?)c9@tWdbWCeitE=`v6mr`nEPT$_fVJxe3y_FryTRf>4&+o+LG!H-jr=p|ql4;B7hk^gFY$5T#5U}@ksSXm} zIz3^|NkK{nMo+;GjLj?Pa(q$7K?Qbfg}D9hQzJ$T(atU1@&F@!ER2~JF>I5TU3LKa z$Zg0oKC8o$yBXW*>~*MB3Sck`pcfQhE#s&Xv4A_R%(=1Gw?=%D`>DIyfMDY~iicXA*eo1iZWbmLf(8hd?-;J!T1BVzu+dffnxZ{{GqG zVk49nceD-3%kzxZ_`kT&0p!49|!PgMS5Zg>p$8Dsb~%y z)(_y#un%dyg!r`xL)cD{eb8?$L9n3vJB^U z%^Wm+$y;ZNiNREh`kneOnbu7*l#`DepB?ZIYmpsr>N$ifd6-A;cS3HeTRBJ9V$crw zb%LO$DJoJW%7JfU(McEOZWF)j!#7;m-t1MKeuxw&ZJ{n&@d3W<@DKFN^B3U>KG>DT zZ-P_)NnB5%BT|a>2^9mZ{EsSN?NxL^3T~Rp#&;1DI;V<*SV(4Enc7Xawpo9is9u5B z)4s2=iOonMwXi|4E#X_91o$g0G}FVm5miMkDQC9Jxu|wtCZ7zI=4QXrE0W+0K*!~Y z?+#8);7(k^$reNP+&;RR#y`4mdQb)+I|VW;@JE#wET%R2)KSr;k$W!UD~5!7e!y2* zS=r=j+4e2lZihUyf2VjYdK25`XyF%GoZ1nf$hhLhTIFjYEqcGM#ML0F{ze$E9+k z0wMMkhC=Z5sG6q%o(ymw(&n80r<`3c3wN@m@+_15%v@R$I7#e44Xv!2%}SUN6x6Nl zYl(TJv$ZNHjO@N6t`$ygrsNgecq`X*lBOas1`LpC zd{^|@F5jWC^hys-Y;Q*WzhHrw$-HIyW^w&f$TsfxqPu++>1+gz$8!U*uFM0fubZ8B z-z&wMJxgGcnXZDQy0ywjAjO9}{C8914Tq05y^$a6*LwSU_#_S{?33#cslNU(aVbyg zt%8HWgDnBOSGEtT(}I#KEr?Fw2ZAQ*uzP!nCiu*+DbVKOb$dx)X7~Pna=Pm9I`Ar0 z3!LFbTc*BFHilv9E@Rm`+nY>;R*m#frInb>@~H?<>dPYTG^6||=crxyA|Pwa;|r4a z%xmwz5_&%wzre_5Acccui+TJ*mv@%Ad8*?szZxI`f^!F!YDM)_fw5Xh)#h+$faCM@ zkD})wadhnc_E|NzMKoi_7%Y;f!BIpTxg5Wg6tL(r_1H~p+T15G^cnwyDqewy&TeZX zh@K`FKYuZD(sCm}fo}S)$YsVCo`7s{eifT?m_lcn|7yfZ1I4{Del6L>F<`GNF9?Rze4D#&D;RZ=EYWg_MFkm0_{WTv1E2#T8IOC}AFO@B`Tk z(m?NZ1I{wrtV-tA&MiNA{DP@mLhO{{$s|bn-ivXBxTkQY?b}Wx-KO^KZ!N(Qw~BL5 z@hr(&K>&*F-AQfu?j}*A0@6}j5wFMF!fmW&7P?ctb;eVp>Egm;>yq<`FYt&d5gt#c zEO3PXRPNQ0s;w8HZfZ*6x#1m5=2NY&xAf0(RZSy~KBH3nqu76_APVn&4kI%6fkho> zIND<5pe&Cyk}7zEetOkJ*SxhV9qXG4`ISEgILEyoAKp|31wM zu9a@Zhd=|;+Q7t$iA8?_Z7v#DvfYSjix0H@fQX5-=gEBGduQ3i-!&3?6kbf#VF9#*pA5?zUgrIK7k|zY-AesFKRUDq$@361Bkf!2_RK)0*>gC;KjfOq?_Qs6gt%`kk86e7j}8a$tTZe!Ep_T z`|PI~W}AQBP&?g@Fytk`i8Jbjm+&-gmSX2KOVBsyuw!;VZhR+$bT(?a+BXG~??i&6L#ib^UxMSS2>l zwC~9A(OG7qJLTugHm93SBnTr%~_~^-}U#Yj}MMW2|c`LaP z`_vwh5*Z!8ufFtn2qO7Md1ZWUfA*hYq=z7X#Qi~eSoJUKHB6k{XxSZW$DITg4gAaI zMZ+@S(f+=cq0VWoRmDx0?`gj!$hW>wd`J!m&p_@eGBP@4jG@ovxb-oRt6 zY{vL0;x{jB$d@(ljuCIKh-~^Ui_hWP_P>PW9<1?BI0`9;ld@(36*iO_yeCH%6@ zCiRmrk7p??pvz5L=9i6P!cV`Y7L&65ic~Fp^37u(5t~|3I$=HFZS-Wo!&Uy{QBLrG ze?aEmf)f zvM=Y)>MYdF*N1-)Aom-gj=tKuxLVv`blDr%ps~}^N?Y}XvH5#PbX01tonkb>e=DW+ zFsnpxzQx?V*R1K1`ngQcqP1ReO(bo)lrH{i@p&4jySC$y1kk-^EsA6|zPO8+4v!W- zGF^nn%`vKoTt(}mv?%HMQN=;>j4Ef?_4$q1On*RucYAu@ylwfaq^Zo{jO8HW(~6Y; zBCXW`tagriMMmSU7}QBh)!yPZk=;OwmbJQFG0OL;wJ*P(Nj!i~ixK0Ntvj_U_SKO? zwyRRU`bhTN3c47Rso=(3hPoBtsf*^1MJo>IP~6+fVTd;(7zz6n0~amlig^-VIal0I zxy*^x8lmpW5ltQ2QtBb@H-65As|x3Vypw=6>V1z>;slle4w|dMlpeB#404(PABD<| z0&1E<9b@|WK=^a1fRN2)YO10r501ZZ`cs$}KHEa;-6{GgbZN{EDWS8cV7_>b2OJ}ItBel$fgB>1 z@?h|+$p5I>htcg7n&y27*p?AfDBN?hwFAZebz`1Z^_n92s<_jIN(J1n4rY|;%=`4L znBUR3@_w-QOfcM9N$i&qXDBfYwG+IW!$XHdq)6r- zHz%6jSs$gq4}$qrq)GKUYq7Y6G-0Y)SBC;FkQ)QulSuBk+qojG{4aJ;Vb#)}c;n~- z{;(MU^_+jTFZ$d>-Jb1wms&k{x!=`>ye4U$w+|ztIfJjj%jeNb_T}CFMcenqIr2TZ ziWWY7J5QKnTCacfam)u;jvTdDy7SCF9+49cL}kWg<)B)ip?RN&W1&R#DET%l<<0*9 zh(UM0zDGL~G_5iKt{GdlgrBF-@uA5<2Q5<(%>bfsbUuV-tW%5NSTD*2ff(X51n&Lz z+c&ZuL7f$4g;1`==eTmh#siVxdKnPv7H+t;mJC=(HxaM^rx_qH^k7atC5FOp#VtJLaK9fqF-=_GCsOMvrR9_mxeN3+t z>@hPX2RWkmoN1}5MqZo81iqt@D3eJ$=WR-Uzg{X1uFXGuINav|e4#&be5+In*la9Z z=tlm+V&kEQ9`md@W+^ci))g8=2;&+6!PvFtuvD~iRERDI2 z)lyq?qw@;2;_7_!wHC*%w-lle5@tEfb>(X*Hq(`|Z^qIp9`~?Q=mEae2cJ?XH@&$` zq2D(8o5lTxvixjAD&)REVGTs)~r9t6Q6^KXhc8X{!INwx@w+U zpOzk{;y$y@a&0$sgt#X{XY6nXFA@D&Bte!NCitE2zWbqqQJCiWjRDcm|MNuUBh6}! zhhEA`S3l0#jCxxG)XFm!#MzxYx>OYIy!pG;j4-P~OmKU9n{-Z}Cf$yRjIz$2Gppos z*8BFE;QTxN^pi?XpRfK=I7;7k8ymg1U)*TK;Jntyw?4W4aStOuBHI_W&zGUzDB zR4Bio{Nk@a*oJVYWbS+`XL+*>h3b;gau=%Gn%^rSia`io^}N$K(dUZK?_J(pb?XgN ze8##?TG}KFW=>3(WcljO)dIF6pT21zXZoVu8r1!Jytw|&mMN~uaA=oZia|9h<$=uy%816Xe7I5A zTXV_~1wytKv$0M@M2}5?j&x92?D>?82@22?OsOrO`EEjEX$dZ+<+WIvVxn&$^5s`- zUR=6T%28?=OWDq`Y$ z`9f-k>YQb@NiErLdJxoWBHYYd)H2#-{;4V4$GzQZdI=j>y8N3E5*)dgBJR|4wj53D)l$Xx7Rn` z*j`dPoUagme(`Mmi_e;C?j4nX`eO(=mLo2vYT{W+i?i(=?b6vfO==kkvU40^wzs#h zWRbSsw=YVm=U)^Yi$kMX`TZ7KwsvRL;JW*ha+uv=DSM11yIVPeb-CyI7Uws3VBM%M z1W$zc47B(>XPUdtmd44tMp=FXM73DHxAXk;qdBqj9w_s9B`VayW&`5!MHgM*M+D;& z;;R;*%?<{^6<1s(C!BDs@0C0-Fev4MC}UV&XisU3!*dxx*oN?{J0L`vA#&^Wx!W=5 z+i@Tw8l4VCEjUb&C$S!c(xmZ@RpOtDI3eXco zwBm&aW3;&9;YGK+MsFm#)zqAahXuV%bbdZzFHY2pw$c`>kVkKD(RG=v%jT=iaV^#P z_DfvrZZz+>^wH>Ty*I`?8ed%KV+oJgOyrPgRao!y4u)UuaSrOr__V zTJnI7yO#OaAy33J!Du?;zRG{s^HWF6di>R_s+cK{M9aM&6pP|2WnSDRK6iOnbXLvJ(y@w`;}ttgUA`0#TKznO=aFBmssA??G!MU=^|l?K)})Kips%-I*@D!%IBSBk z!p9kw8W3p&^yW+U7pb!2aUIrA-qeKELR!Qw!dw9yFQav+aw|NvU^2RE2;HAT&eZh45=eBP045 z)*29Xfhqg~bO5+=ARvd+X0&FYFh;k#s8$gIQtwm4jlqjFwruZot#$jFE^H;=DFoyi1|2Y4+usUV7~)JnH?;STBkkJ$y&Oq76S zHnyp<>FbR0%auU56WbR))0wpDfN<$Q4#}?i?cXlMUb8HD@}F9zp|L?-C*m4$LB57V zvr=plh!TMan)AC^ZZG7v7M<~#b$^;n))kvGj=+XKzA{QA$MwvXW3Jp)*Z$8Y5Do6W ztWz3Nh-|b}g$a?*7{_;~xj<6Rc*;?xQ%?iSa!mq}J)_>z6lobjxebL-B$8?(N@T#w$!sb>82!k9^$` zpzj%dT@GuQ!wv~+gsF5yf9}lixg7V#hX1yRw`c4zn3RO`)qB`3kVgD<+J-O}%nLd) zvz=k8h|l1)E>zMoVr5g#!?&IHI-E~bjGOLH^Fnl85!#Bp4J4l0Z24lM`Yxs*9oTLl zBpS&fQco4}89+?HHsi2OVLu@>Jmu~ewNI=-Wmf6bh~7V@2yK=0+s3`MRVsy!kwTI4 z;c>Uzci;VR!6;03dt7lL>wHcX@fmE|=?W_ismgk{j}-_EXk<-dLpRFY?9|nUhw?Xc z*ZcOxQHlq&DLzB!e))|o#pfyjA@}*$GmePdAkjB&(Xn8k^SrX7FDu)fVdM5}=>7=t z_i#sio`|St4enVw`gyX%XN~1Ga)a^2VJ@26;E2qPa`mb~+4xpQ_L-YL7%xOFl%Wo_)zvrYWqoKlIE~c@nmK>U#M|mA{ZS_%e2l8Nh%PRdw2Dz&imr1_w9=V=ih%lP^cx=_OJ2o zJrp*AjeLHu600|5C{6L_8m~XM+-dCsKIUxW3<_MA}-w-9_WSKK|e@ zI_1l^4ag543AeID$4zFoTS|ET!CxfDx@W53J~!w$y@XT=3eZ)=6rtbR*)20Vr%`++ zqzE`idV2e1+gn|-aNe98KkB40)=F?xh$(O3@KR@{_L7Yay|N~=L;iK}S-GtFU`K>* z@%_*BGBIDJz$mt7_0}Y$I#}o*{8(qP9 z&a{-WYeI*A3uaEP@T()R=T55?0orSTmGNCD7de9Z4;>--hZK|R5(dCk2MW+tMtclx z+ukMn?!AW)LW;B3g571q=54ZIx7l820FxihGgii0-e9rHZ12UXrOFTbwQQe!bM!g+ zOw+-#dt!Kg24zK%;xK}Em5JC<^lQ|__m9XGDQ^kN>+=JNk~ z$9DPL8Lg7f+dZzVzTj+$Z_A;-+o5P>y}r=W{ycYjoV`G9ox}W%q7k`MQOwecW${BP zDo&c?!FT`n!0%|sM^w8NLN|KXlHeH1dct!k3f_)e_tZlN?5KU0^7pprkCbJ_9P7cW zrhdY_zoAH2)y^+Sn^DkaWOS&!65l~{p=+&9GjHBpDTHNB?Q>Q&#$?f1wd(~=NF~Af z_wfb!4=nu#LRP&wla*{HEjK(Vq>6mxxMpc=Y*c|dXjHI@c>5CWk1{ z7=B7VXz!;wCnPRW&(nOW7PXczmyHT2W?tp`s76KXLSlH_`am%OyU=(v|G4dp zTlRBj78lw>+h%HFW*Sf2(h<4g<;7UY&NJQI55an)< zc6)hD!4POy%=hG7%RHvN@sztCmDPr^oYC|XUZ&u07&qO@Gq>;!+mi6HROlQur(t_< zz4b0R=bY2y^Ht}p`__W_8tqiV`?>-`)ci-_&Nf5GVDJz<}s?Tk)Vz*lVqH*A*VV}fUY9Cf^OScA%v8Uey!2$sTi&M zNsWT86(B;x1NtY!ugX2?*X1W8ugat8H{@?x-#%r1p`*faH2N+q_=Y2H-PfIi&eDQ; zfjDdS7_WVWwY`ITT!)41@bclvMg3Z}&O@iu7Bz8^gmGopdAVuB^P=0FZyLl|y**~x z%zdZlBl=mFr;wdaDd%-myoyrj`rvM7ydP~~@gc7Gnf3O+q~N4GZXpMvGkQA*Yc{22 z`Q8n({+l!9?5lfa`Q)j&*TC=GJ>)bL^h<4`1!f7FU5xWMP72lnWmNpv zpV2oRK=cg{^LxJIAnxsve0iDc%ik#H{`UR-1Co!R9R3ukHaG`yZtL?f>Tin9ztG`S z_dwP;?=$LqHOYF>r|$LL_8X3p0>tAdo_InD$vr(i($do6&x4@6bm>whG_PB?PKnD} zcpe@emU;8$DX|&T0H(kA;*09@&_fTE<;#~#tpVYA^~-O{>Q~>Ck&G?P?b8dUf3rU$ zue>oJ`|Q>xi|4dWxDZ|F{L`4{u=@;_ z1>Dz<0zty8q!U6)2j>w)x`qb6OIk@j*s`B2wA$ns>DN>=^k)ZtBZoE2k#{%kDN7s1 z){F30e*0_r?sJc*?;k$kaQW7;e=N01^dY@JJ1X~Px5$f`9(gI-BNGA+=_L0I5BHOy z64N^O!9Ap1PLRFikFvYPik6k4ocwaqMuK)>U657N#$QAE!C!PJ5$c^^-KBa-hj4ZO z7pBXX&Tb!*0R3fWxi6g+A6Z^8(bN++{ZY%xiAps;&FT4~;>`-6n>L)67My(>1vPl< z@r*gd`?Rz$ZMpNt1H|VC){TaGtv_j5lYGjVUb_Ochr0v3k3iUWA40wFDw(@;)msF{ zn*njZX*heU;P-qH%hB)AW1^f!D1AKSi+&E$**VKp5}daezN1AQ8z(JoQsd=-a~};* zE1w>b2iK)5{sx3@1hg#OxR{&FOJmk#yP-tQuwRjCN^o z?$cW~zb0*MZF1@#pCkRlSv7wzXW2Q<{6+I-%ld6Q<%t(Isr)+IX2@87s&5_ zw_2{g`f9oG!V8sfJ=vJT^YEy93|J+4ru2_ww(Q0t*vqc8vC-S1KvNp(H*#|qvR9h&5dm_je*`8PQ3gU6c4Wc9;SUaKw zD1b=FA3i*+B5&;<&GFMgochkHA^A~(9@IbL2w)JOzq4x45wAx>^B|0U*_qefpOt@J zIVj^L$G^8kK`WoQn9bS+(|Ve+CoX%YE4J$itt#63ejvfC-j#a9g7_TWXAl`{7AySw zSSnWE4IEdI;aZ^=-IfrH>)&xAMauE4LyK7*9qlEht+WWE>~qd;Q|BJQdA#!JVHMR3 z5gDIgv-9^KF9_6QYB+Jlt`!aiEm_-`duy zhH|J>kPXD=vp;aI{O0914(eHFK(7!uU?-sZ(3W69I)4J^4?>X$uA#Y zD_eH#Qp5A>-;jeH5&F;m?9bFXAVTXX>B*p{Z%}T!|3zPT9xJ90ee}N5Wlm>{RGVVw zpT3r4GCB}zd>3bkzR&&kUTN!?E+@X{!z%iC^M5D`9m_dVs{KYEu+r}IDYj0>f$y)(AFw)duJ@Ft#qZK!s5 zX`6ejPj*L$8rCOCegc~o_mgj3R|5*r6(k(h<#w8b=uHaHgp3XM)X_FJDLyn*7>LV= zhJJU`LU}RMEw_(ulrni-x&&@CXCTH5$e)phkTvlcACS6u)SANm<5bz4c z_r~P*VxL619HIYv)^qad>|xR-4SAzwt3kedaZyW<3Y85amdWyc8kGk*+#tMGU#r zi+=WOI8?)c9Tmo9%AtH#ozDi(sYLBxXTJoy#tsr%meDC-@K1*Slrwj^5Zb$Gm+XB|bmDKK|LHZ*W+ie(81D@WxI#^^f+K zyMOzV^bWWUdCxh^45xRK(@r~09(?dY^*I?peE#V@Ps*@8O)3S1=wGdVL*9SH9@5!X z-D-Uy4`^V%^{b=t#|nhr2k-o`BP?Iy2(>TDSN`VRa_JX;qy$?;B17=SfOLpx)}k;3 zUrZnGIw2B&<8x=Lb;gOm5Qne%%7toOx=aw1VJCu5RYpgEUbtwfBOOcXyf`mI@utVcx<$(ttkX5TzsqZ)`nwpx_>uJwE z`>gERwM+S<;S-fY%o`aQQN7@TkA3W8vj6`33n59x^?=qF|3Hc$l|@B4r{>FXP4kr{ z$2v!helgpl#ONLNkPO(BsTkJ^w%0Id@mQJ^AVNQ4ZI*Xs_Ef#MN4;bxf?`$xE^dd1 znY2A_IqPaiWd8OA9r6Q5T)zG35hXMuniAr(DHP*7Y&j6cyk=uMZzS}FxAI3FV!7pe zk3YiCQZ^(h|AAb0{y5g(C|?A0G*8?3)X~t;G;GsVBJzt&*=Bw(I$Ue=o*Re1R905OiWJqrLts7B)+QCh7$-8Q=Uw~3PXe%87ecLBHL-sk90Alk;zn@m; zJH%}av;KOgR0^Cg&ZTw2ud8DPp3Wz~`xIIEM116PXM2k*nm0>szHd#@yms^q$m74? zC?_7euZo1$y!6^z^7g%!s&!3<06jdCR^s!n{y|B#=L*oO-k2GcvCum-Dz`tiMgHX2 zJsqL1dV)!@^Uw5OS9X+R>;nm2%MftS|LcDVMKWU=L|llz5P8+-1~-ZrQPD@9aF!CI z&FE;8V!|}mrG;#OsAnBreeeOY)dmVYEywT&{^xwAUTQ7XvR7)F8cvZtOI~K5 zLq?Scw8>u`aFl$j<3y* zz=Fd!i;3A8W)!*a0&Ryof4uhILAmv}f%tXnZ%1TWi|cn>OB9QKHY3|A!J@e-HGJyE z{)&muCdd<(JFwz`UB|`>tRor(lZ47MIvh+}`QMmYFlV}|lt}^Lt1%1Lwv54 z=;r`0z#!u`w70jk%4=KSlo@YZEM)5O=FTpe)rrOjGdiAcT~`YV&=n-Nmv1#DV~Nc{ z&&?ej9diHn8M5W&XZ-1VI1gcee##2D?(PRYFGQDH9(+ztUa>^pxqOM-bnj|;_SLOU z+PBHE({{?;ci%0io_eYfk~B$!V)P=5T<(`x*04}M3@`9JGOFbwH*MhTu2 zpC^y?sa=v8aYbis?NEzew5ozqp&~(8MkF+>I3QF)D9*>4jTuJ(o!m#Xs0UuPew%PK zIy@~U{Mh6@Mrmt>_)?6oY-kMl)M;%wp3kN|iUtv}5GA8v`ASBy7UPe8zRK-u5Ec*0 z^VzdFu{_6Y?->-D8%mqioi}1`*6nOZ2}I84q1%s>hDep*2aom3r=EMzO@}v;Y}ZLz zeP+GaRFyTBJI(SwG1$hq?JZ~d%Xww+@>Tix=~8|kYFgIvgm!m5nX4BS+$Vv}PGuh& z(CH`6*ReDR6#~Z_@~$-bItZ;*hK7ceg+@LW&z~i~eercEho@iKEPs5=vXB7X+|)Q} z{?D(z{-*!kP7gSPBLrs4i+HxP85!*BQA3xrSuJhTqQj4=D5O3Q$TtOG2ENm+v)N1eLyKeb+CHR_xZMhu=UU$`{vT)Hdx%e~JC}H@z zt1c;6Mx=q|h6$!2!Z{x(qZX+(7}wjZ#dTa5O`*CF?tU(8!ST&s4f^6UxcTv6fo=H5 z-&wiyp?AnujcRzl+1l+kra^o@c1i8@rC@e|t)D$p4Ytgl(IJ0u1n5o9u=j%9g-jjr zn|JN%l_d-3t7m=;`k9wOZ`FYUbOi}V*x8|0;BxutPk$==tT<>)fdP6^B@vEE*d=!h|ll>Hmy_8DP%Q- z#6tMn>_vm8JR1%iHK7g&y%7GC761|e2I_WULpN#|Zm z_up$bIe77We>vE9XfNo|fn73l=1iGv?2*9V-%2@E@suMVH#A98`}Bg#wYM}#dU!yF z2Kr>wY4jf+>{El~EZ;I~PbZJ+dqD4;dvfeNUH4V}Yd9B)%;<=E!>O&K@LLpv&TnE0 zvJh}}Bru|pA@E{81ZQo>p+sh9eGqCdzRVSim1Tu94%>@Th>F0vQ^6xo3OX2sorfuC zBO+Rh#h4E3481Hm5QJsMrC^H!Vokp?7 zrmS-exNnnL7_8i9hg*J|+t~5LM@B01`cMq8+c;)PV@}@}lTA2SFu1?L`+3i~bL926 zcF6F^i0nRlmdxm+9Y?(azwiEmA=$L;O&J^lQWoA1!!-@J`Y6<;1DWL95 z$|ZXJQyq6@QAx6^_TWuQBb&`Q4@)Y^rMP?avHqvDa#!&p0eCyJUG43 zsif-zZCYCW_|Xw(2uGm)>EZ42z`IV7S^>x3?LT{__=(%EfFhz5xC zLt_#=*KxkWJ_8L$CJF z`5SN`I`1=YmOS@|ZL(!o96b+=7KC_y8;^OhFbx?voF# z^aWzfFQ)@SbDiKjKUu5%o8j#YF?sfGdxoNqkw1iC-5Da*zw@mvYJB!?d&ppaubloT zmlj>Gw%kBoSQmuzvoHKY$dW^QTpxTy32zcwUgNNs{Z36zpbkXm1v68gby4kg<`=c% z18&5tE8Mp>JJ;uVbEVrk&R^Q_T5JOssUKRKo20F|S&7drP3~JH>UqhxHVPUm82DCM zC_q=3a4U&5Cci__*47rg99~2F#QTqt`LjD!l=Iq+!7E*R&YdBDcJiUJ*X}dr1V@Cv zz7Az>ZuG?Cbsdv2}VeDf{pJNAj*AD3Kmv1kFfoP7o%`=*<2l`UJgs`L<*@%vtM(fK*T zw#i+0-6yM7-RGyl{zd+$pYBExd`#`{S`+XqMc`o=v{DmN4(2r(mGW||k-A>k zq>j3o2kz}p8{IV^Z~VsfC!BZGQfZnVRGQy(rgtoyD;;~zEh(>3!1}t@ZpxWH%gLvy zb7CVd^MU>GmUF*3=#yuawVc@IZ!)nDdFHwbtrr!Ow5?--ry{oZnyT$ToL4k=J1i9H zXC^Zc+KQBixcK`e)bW?)Mv5D=EI-oA>+CbK4>1j1&;7%;ta{FklKa3B&GO;nYRmr_ z<^0fx&M6^X@xY6*@WV%j)c_Is#HEeOBYe@El$^Dq_;nZpHHHV)jyR%qMpiyE;`=>+ zq{2Si;qxgOuMp3eF+;MWLj}`3{=!D}i2$4R+q&eWL-)-QprKH<;$1LWbo%_94r5G=VR)-)inNlx8;Z=E$!3PFxt>5>7iZnf#a7- zHNp8;$WuI{I(W`1i?h+uv@?v#{}t#r^8CO4sv6>eAd9GAL;~vwXA_9F81jwhLw7V7 z$BAcrSVcg??jve_fimdw=y?#tu}n05bR7_sA#g|M=i4?2+REZ1GCht9o$GXyBc7o{ z=Z1!c*eFhgb4mQK2f~evxXyq4r24Es#u2?1>P8dPF&nVooEBW%I%amX$#kb)?FJ!a zy?(vQzo(di4%LjHjnkU+wCHCv4e&kNE%W&~yG1BXt2~dq#Nh;vjoNG%0mIVf|w`Fc%wF3T3V#FAsk78mYV*d5xL=>XXJu+ z9VVy#(f%O;`hrsqlNUC;A?r51CELAD3)5YjWyOjWMb~%GK?lnD=LZk=Pd@pKtXz4A z`d+Fy41x5ahaN9WfUY$l7GHn;jdJ$cXUGXB9P3Y8x$;iA_S);@^2LLRfiDz8SjP0rF1u9XqMDJ{Q%|jS)_0jdAN%*JtFBei(-^QGjG(&iHvUt}`pI3sjp0~RrN@JP{=H2zjUyPksjGc~fecuIN%MrDY{`$vMe%0Wa zD{oSH?|Dk`dk0wmvqz+PM)?(LTkMc!Rkfp_^F2@(dbm^%pD8IRlB*bq2>Q}lt?ILT z(3Vf%I54rD){O!6cyF86;Oae$$5xQU4SZJR2>$qW4$By(8i55zFwJwl@a#&0dCHo@ z7xv@#)+gT8s)kymoa0U(uhptMtEgvZxccruFrBHQoZV<=6McSftU?EdaoFCEU&_AH zK@51_L{J2R9T4E4pE;j=a68=JSo+rv30uK_3T^n=O>ic$XzLXdow$*a)jw3=?f9f3*OO)jE0p*>HgKhc@TPwC~^&e*K@CP$@=9Q*9ZE%V_F>XxwaY^ zCxXC=&+^_9ar+^F2#F2}^|KBdQIIa1O{oTWZ)pkdZ&T+n0~*{nw=B!+#M-hV%Ljpq z;qxR&7hX<`rugl*jde;+@JLX0y${UaAe^7BYq-C^8}00h&nZuQwsPz@NadjbT|vwf(Xz%@ zNju* zGbGAF#C`C=$JM7Uo6Z|RnAP*~01ffiT;5!Nv1n_Q9ci@q3=r9T-gzI8E3UXol`)@v z29`rw2-tcX-EFeuth3Gtr9~t&=3%=bc1OX!U$}67XgjcN7hQCLS_ifn5z+wB*eDaW zTW=?R2jq{+1eVkL5X)hCvyuUj7L^U8i_X`1;sTEEsHk8*)}i-%bibPU#r-BFW1K?} z4beJTNxt&iU(0u&dqjQz@BxR*w~nn%4`VWR^0(sWuN?i_5t8Q}oh!gTclAmow7&15 ztJL^IfB9W`;d!c9j2}HRrR*)-0B`YS5!&-y6Du^5aKFO$d$_c`SVrh>kH;Bi4~w>tq~kDKR$Yg%InmyTyXwfc;UIR^Gq?$y}npA ze_g3X&dt5$2Ho0<~^-yn9AfvLA%GQwp~zfdO6B;#<`BT0SLCTl!-Qwhl0z! z>G3jcxbZ%khiwO6%yp<+xFxX-siuIe?(Y&l|H5K&WL+_N<%^y!v|PUBJCJ;7w%G($t_l{xlE&?hkU}A&XUHUwdnlyyx(JWjwD>Ks3ZdDMU+f+iiEss#W(Z z@fY(TI74g&cp%jRr=FFMp3%x);B%_CHh@5!n#q#ES zL?|u2r%$o?4XJCdf!;It{Rg#J;X=Q<3LCj*n#r#)5@u#wjimtA%f#9yN z?2=qi2?kJdhJhKv027&E;(PP*d-vY{_qV!FcXi+HoAW&%xbr$xS5;T1s&Ab-#rrvb z5MjgO190gPCO!j*wul3QuMWz{F6$|7TKJ1~fNOHpM6|&oP3Q&MuydOhgCSPb=WmQd{NgVUp#+iR8s!=p*xiT-8(v> zem{EtA+;_v7WzBT_34{MvVwK-L0>yB&BetVw>?P9KI6HW*yn>MEAe%_-#n=pLEklJU1=6klwh}?o@f|${XnxYyThKdl~)Tai3RW zGk_otYi<1g@A%KFFo+9a8^XQtBRdSI^IPlqKb`+w6+eds2s^lOn|pqU8}T75((Da! zAHvvv3xk#retz<&Vdb9|?g8+-?a47qkRGK%@OKhtPM5JKMl4?7yzV32%0KLZU+<;2 ze`lBVw@(RHm!DA^Zd9T$Q(mLVdwvS%-~Tj1zkXP|o=m&SeJuztzOr4j)QZOQbE!wpI+Co{vb^*}BSLcjGAl?+T6 zST5dI!npnQrRe9?HefJRJrs3o)cr6b&o(T0*bRp|q-|hukTS$TeGm28QUd`R{+VY_ z51)jutu-F9E!>9csaH1AKJ#YSD+VSaY@vhqog0Mg0>V zaGdyS$J6G^Q)_FRd!BYYEl#{G+Oci^e7Na2@oL9-`Dk_Xp>OktvC6N#Ep1<`bDk~k zdaJvv$7kouE32@5ty&y*8N?kSI;O1R{T=l35-SH{0bOFe$&08c=FFKx&p-DZEkE#d zddz9SV_2DP889?DPDyS0a#Xs$bnXfC;0tT$kbUOTz`Ez@lb`$~6^cFa-DFFE+^hsq zh?lx0EB3Phjs1%j?O}rMhXq^|{rxzD8VVgm4)n-W4lWD)F1Gnh_@-k8oFB z`8{PN!{-LUo39Cu)2|7Zxf3H_K0Wjx#jKB&fnhP$hrx)uW_VOAH@*B=+P3dL{{XFB z(n>;|kfTA7nh?jd#WZxeo5>WV#*@?<;WlDQh@WN_ZzpJ0y}XQlELKCHe&$;@((=!q zuC^&L+PYmK&~9;jMs2v_2WOHcir_(bv*I{%WWKoeGr@>sAC&MX}BI)?)Y?u zOOAuTeje?8;mP#Yqc71$O<;dZ6W3X|MjEi_M&6K4ly&#Qo9|S!rA{|h8jv~sAX$=%Koc{;9;M$P8=rX5AW?a zoNd+*!!fN6-5Bnl1gu*Nw@88oH2lA=afafDrzto7GOak*o=^HbL#MhA>7v#zIKOuFu&VY=jjl`4EH z$;ZW2+-@95%h{){rx{oAsq4jgpyDSzf`8x1f!D&CZKlt<8Q1U|cMMam;93R4pRjMK zzBAfzR^0-c_iC0ejhgKbO8s;`D>s=#B49QC91v8nJ+T0FxS) z?p_G9dJ8WH7!<%bLLZHY7;bBzQkS6dwQeq6iVd%`E-3tA7uqB+W<_|5IR zl;Awg`WC_>db;)9+-*XuDhmkR)#8MM_oe%vvcLPOp{;5AQM4mN+cL&6vORpe5o=MC z8lG_I5-KOTJbyxUu&p^s$djPE1&p?Z-)HW4hbs)T%Bwq~jhJTx(&1*r(j9nk`PrRiBv;rGSVH5ws@|d?{V!|#<@O}N32bG`; zYvJXVC_S)Yn=d?DDFcYQYLY@rIF1saVSU{;$uB@-9d0s^mJ+zFc;+~6GZ6peednvV z!2~N$uUOv_7Sds<&BF2A6Az+SG^^$y2nzum=LM1cAWMv9Hd@=+RR`R7pd4Ph_FkHM z@Ot}#_4Fsfz1_|-K5+lC8q^0Fr{T2x(gS+k{6GHdB{X2bh|4Y?(v5<~WZl>V( zF<9nCeQ3=^k;^ETe1Yf;w>h_ecB1NlfE7DB7M@#|RGkxejx)hNq7ersFg*R>cC`)1 zVg$eiC7p;cToBPgeOh-l$H4U)1ZV~p(WpD}Xh_t*bt98fvhTk8(yA9b|jgPRj=nT;tp4o`?C5uYoB2+(D~>p?57 z0(|PJr_z(pJWpNit;*UNJpS@}`r1FfokkNW`r}itrXBP2s~hQsH#XDSC;v!qz4aD- z?Q37NDw)U8$tRyo`NCqP?oQ*0C!SUow-6j5;G!~`iQq91B7?_-iK#5wvO5Qeq!6On z{RD$$SmI*xL!<-iYE-IW*^7!Zc2Vs}vXMfGQ z&+6bgW{Zz4pV{3A8Uy)E^oVkRB{tlcFt|tg93ecT+)%z=85iy+4{=0YlYIGM=TXj( zow)MN)Y4)L5T*DiEO^^^n#+xk?{hePzx#M<29n97+pAxXjE#;_TYL7Q%iP;;TQVfo zyE%~6G!irtRbmj-U}1ZRlM>^B3LzaoSTyf* z;mNKD4GZYym)MrlD2I3b?wiW;89$smtf+OjB|2-)c`voLv{Kxvb@K4HV8svO%&#Bv zyWDvH!CiF!J5Qy22~J4K>4dw_!bFuR)v27c?w7Igaa!=@!Y2&;p(Qf^;QVeSaQ^nG zF(oiV@PzmrWlakKI&9ww6&j_n_MFo!0jJHnKCR0CezCssP^P$+aH*p6_lI;?*VFD0 z=0RQRc{S~0Qp$RoD1qe^wp-W6`m6uOlO`Q>c^?(S@m6_5NIdGm#dk z&VyPMpl!^EZJt$-x6N=S>M4bxM5W2a2XsCS{am@jdS=h~;V8ZDpjKrK4FUN=O*F=R zgWsC1{&&Q}8F3{vPu3qIz;D{0`Y*5GJIu{Y1V7P%1iRCihIN*Q-=;oVmrh|?>B~5^apnUSdd()kdtWv(8m+vu??tFMvWL(M$ zn&v1gX;?o$($}Zr@9pianjdJkeg?6w8Qu=3t%#97t(_Ax#xZF^hb#QFw0iZQ?K^FM z@{g>x&U4TF=ZeUcOg^A(bRo}d7!7)OsK4Ne2Fp#*pMThPA)TO`1${C*)OWnjJ5$wt z^!ZAHmd{?C{%@`q-Y@>9TTpwVE+1oyqHGl$BfrROgnr+2D`LM-Xv1f5?Dvz5a4}z5 za6W);Ri8?kY30UwW9=)djjmgcd*yGY-LvkygN)ET+41<(WI5>osJ^kX> zH_`i!+K&c?$LLkfvTo7rY08&S*Wx4T%!5v(!xl}WE3de$AORYF-dOnM2}*Q#7VPRG z3J9iQ$J}#IFr44o7J@Cic0eA2E+vBL!V;I=Lm*AKhd>3HsOvq>z`SEjM^qN7M@?ilvBAWtYQIi zRk^v)EPmnU0-`X4=0WW}%$Cp3{J$HOKnxcc;<J2@cx zY7VpD&IYWr*`>&OOXPm=<3Cd82&-*|u*FtKls)QVVpkh&-xb{{wBn7;bn+p4D^VK;7&qK- zgYx4#Iy!2Js25Ppz?OJw^0Um%a){2cSnWNqx}b(-%td6_Ll?^mgy9E%e*-=K!0pu8 zJ&EQl*q6?@;HxDUi+}u$575-z=Fo>O{V~-VDKp)ARmh#7Fp4@(DwWd2Q2Wb@?J!E< zF!r&e39qJW`OiW$`;|s~wl?K;a}i) zzx&;E&mZrmw;yl>y>sadijDNrM)9K>7}d2u}aRznmI6Qv=VqLK~w5%7VBIB<+T z;hx`>gkEO$FvVwF7a0ssyofmA=bBxTAm6AEXR>5QntTrW024v%zPX-p#(Ugz)XO)D zLk~UBnFw;JdmS*zAbkDd98nJ2w|kucNGI4SkPj$tSh)wUNg?PTAN_k!s*l6>kAAQE z3H3+*Z&ky!hAvc?fROs<^X-ZGAV|M*{k^W(s>6Pmu^*EeX5#N2xSDbSewcOfc+azs zam8k&^|F-~L}6y%i%zCLyyr4{^b5bF$xG+c8o2zhPW#k5znL{H24CK2 zdIQUEIHEgGwm3!e?~IcD>&Jei1bSF~!&fxB1PMc33zI9p(H5DXd}oLH6^kx7hiwmT zb1V0xOPz)DjoXKmB{YWO=Q3=+CeW-=ihb|$@t=4{yZZ4qUuS@mr0FTm6g2hGh1FVV zCMaw6^Sjfw;~Ez_X=YBCfki{#Y`UU!jMr7mEae8Qw{}*P^>eV^T?t_wlZZaM&K9L> znXm>m>VBw~qQ1rxR-^8R(Q+U9(D|Y9z(@hdpuS$HTeWPQaKZ`n?9)%tvOO2k)30oF zw;#OsTw3w;<5nIYpmWYShm7uNbix6P=;0UFq*+Td%l4Q__doj@9e%<|)NTpdxHaIu z@T#k>qG~W{LMQDS$lZ5*A+)!Y*9Awm(FZoBYk*a5{`E^|DB=0kvp=Oo=i7h%UHaA4 zpQ2A)b_11?Q_sGbrp_pQ4v2DNXdU^D)~Q%-qTDU$!ZgIrgw-L&;w<1%q%0Qe*8i!L zXYK5{&mf;53s`h!A=VzLnQ{edU08%YeNGc$Ug+^=;>w zv@-R|m(%eJ*W`8-@H4^Jh_9V3O2^W^i^006P+mlUE+Jv!bFpUp?|8>M=*cIar0s9~ zm9D>kt$K)-FJDe`2M6i)Up+vy_es9+GpA0Zd+)r34msoyI_abn>5E_dLQd&G*kr$N z5F}B#g!rsaILMVxxYEG$36WU$6$gP6>zL#ZLN3HruH^DD%a$!oOB?Y*6h=|)+}Z21 z#zh(saKHN1Zz=1DY15|oD!$kcv6J^h)P>m0`G)9w?X@@3=RWr}l?Uu+(KQUV_PFV$ z+thjRxg#CjLO86s-DZUl_ zGQcvzv-piT5eLc;pfOU63l4V85_L`T!q)>GUP%yy-0;7fa{+#gbzp6L>RsPdA}l5k zP*a>s*=I4_rp2 zV9lB})Ya8Rzxc&3XkF*N$~PMG(bEp6m4Cd6uDRwKDhIHhWgl@-B5u&PaW2PzOg@&` ziZ-fZoGlIBpN|NdYrgX#S3lChJb+B+4swaJm@>|nLdsm{p=Vy8WqZ!IdRbejt*zDV zwJb;DvKgs&AJC(tW7M9P4`@^Mn&Z6j!+ZJC*4%wS3x44L`fnfqkrJy5bun`JuKlSM zK8CIP5SA5qzKeD7kPZDi9aaI=!`a_*>s}cfSKspaU#^^ReEI2}bj2B6p@QaLzA{Ov ziEsbzuDp-UhjmxAI6nAWop&!4b$z@M2!8Z;x?>b?TLL-${>+r2Is#IrZTXT=`|M%f zrSt)P-G?X8=?AybvunNU3?I;Nu>il&zkPZv^4NUJyj0i6IOtJQ7D3h}v1*)lCywed za81Y6vTacg$=S0C!nU~+x@h6VZklJ+Cy$vqgO1sK2F;(4zl)wqfqeh=mS`W)Yzc)Q zo>zP$re0FG-*x~OEN^^oit0GJ1cpyk4t`$%Gi8ii%i7xxMKbFmcdT4MLwPNP+k@l!m^6 zLE5stm-_k#Xum}Z)b~;M_bgNgv4AciehUoW9x5eVd73(PD)q%Wefy@gn`NC~BNy>B zxIb6HHIq=8O~lNWMj;|{DTQYyUop0j@-8&}QVGvYDawzV(LAYwyIqFK2zkQ(pK%{0 z64LKlENmHF9QOD;zW`bJXFDoAeOMsGyGp`tI;xElFHIi$7%S2BVLRt&3r#}{<=X0+ zYz#C(xLc!aO+OyYAjVX@wIU*l{~X>O}hn%8W5UNGAa^oF5f z+Auhz_Wt+lYt`RnQ+ntVOZKAo%$rR;ZIzkqG9S9TqXlwZNGP>!4FuxrEuk142Upxa zq$Uq6^!bFS{XR93VVjz0wA8M6OxG7nZdUH4uICmD+Bgu3e}Cs4Y8@l#FnJ(c-OZdi zlaSk|9(b6xqb)RfUT7R`t#M0qep4Nb_+a&1t;q)&7N3DR?paovhSe0eg!WiVJ0-_5 zTU-w98llA42#r~y^Vq28&cm>d@0c>*61K}@ZOq{LyWoWI`^D?DO(_97k%(Iz{c-xA zGWPR-;i}s$q4=xH6$b>~#Y+!WA~BZPQW=6XSbpe<^vzG7sKjD0Yzem?`tEJ6Fbp>v zzxv*%-3bRF9-sHg?}VlcG5U-Te1*?kSv^C5hP5@sXHFAwBY)-O!lT=jiv@Hzq2tN! z$ghr}&h`P*`laZpH3{k;G1YtuJFIO-F0O64iQPl2w7!EgtxlzRGjs2h5o(*B3Ek?q ze)AT3(Yn6N7cW$XZDRFYT||H`A-ekwf8Dn{Vh4Jbr%d$=&_mnbq{9zCoCKi}Kp!Ow z&zYCX18!x9?UmeD8rX|`@jTp352N9slfc^m}=RFNy(3lx|@3}K*bZneXJYYUOa_@iB&wlnZ5`;z|a?>VWxidgFVO>5L!t(;WfpT7*So?fSosP_<##HSu`)CTI!s0qv$wK`}bq?{Ou; zWP=bQgH*+0Jt)L`b066Q3`qIf^M+)TeJ{tX^|@W0>UZ?+)9DlYE>`<*e0?q5xOy!; zWQoz=eSQ_)^x8W5$SbS6QpzT>G;>-kTrXP(_{U;@ih!4ovGcddGFShM&HV9Bhz&wv0OMO$l&PrR-L z=I%a~HoUb%EgLOXzN{afO*1)aiP58ErpB}0w6R#6Iwtl|N2et`TYrt1eW6Ep+Vz+{ zW^_|I!Sgq1Qgmktck#Mz3s6TEcY9+E$zHI3Pm5wY`@X z&7Vy|;{vxTYc_7S1n8+H9MLBff(Xzh#6Jn4KJ#gLZf?vL{qvvyoF=Y+%BloUqfO7Q zrYo+vf&`&au%9V!T|O2-b@4K8E>2%>2e3!&C@}r&QL{ zhGqd>iCMyIAo zT+3#N$#AIwunzysaFOw;%YGdhAN)G2W32pt=<`3Oe8KVXHO-dOu*fbq?1ssH-qA@x zH3KZ7PuRDG?tNigx#iekq}GX@WXL2V6{RQHN^cvRLU4V_0;fH1PGz2Hy|XwI z?(j@1-USDQsry%rDe($Gkh=bFqsp4rykVLzEQHMu{&K(#8)VJal=2@N<|i6L*y#ti z`zF#W6y2vTu2Ci4yP+1&o)ZpNWHne&HQ0#0vfl$-qe43fN{3wtCZE46Fq6Ga!*9=T zFL)#M(#sAO&(ELnE@cIM)pIM=-@dT~edo}(Q8mfr1DdJC2*e=0j>uf>B!Z>DJ8w3J zZ(JsJhFgz!wZ)Xb=vtz^t(D$?(qZ&J_dQMn5x0pE%iJuF#lkH}AwIwFklB{FSKhiz zE#fp`{oSo5Z%l8;37c`4aT4<|O$ zGs*3h5t|FdHViSGVd(a{stz z^5(U$8@Z*wHa=n75tnsO67G9nHMC7nd?xo0DyPsyh4KsU zjCd%8c&(Q~qffE~>*ntE^zZiG^JdT+udSwi_SuJOjY-{|^moS}NOwK9j&===QYmO{ zG3Z^E_&l+*{7w3(=P&HrT8aDa_;-Im{n*uo?$a4yVnOU@pUx~U!#WzjFyKErh;B6< zw4grk6W&CH@I`&t@u#~J9kRb^q+L#Q3-nq9qVrwPj?s!55TN;cXHPP1I)AG_;j>UV zaLpk?6UAM3RYGME0lLK4A7OtbQbcG$D2()`vQTXxJpbR=8hSa^=Wg34KAGkj-L$Xu zcYb^Vby}a6E~C7km(kHNYHx4LDJ(aPc(M`v#U!qYM3TnG$0;$KpeN&&px(BFHpffT zC=~q6AL~-T!$zi3SG=9#82T3T+@VxjEYZ^m6X0Bnq7r?;e`tx!KRCZzO*Z!1r^e`n zC9TT$E5zq~EomW&9r*QL71s$%+LYkT+u>*Sgzxkzv8-51c>0Pc@jq+=LZ9uN<7&bl zYV24;n_ghMZ>jmd6NclEgv@{&4mSXc_QCRHK|eR%GjBG{KlwQ2isQ!D)>2Pf8-3@H z>R3W&@&V05|GDF+RKRTxI(t-y&r9RZI;wlsmjd{XONIgkTr6{mN)la4`r{@jg2 zR0t58A&ixiQum#QS;?hc^T6X`=;53I_SZmj$6*8yEI;T#sK&8DoIxG#ni;ST_sYi> ztliNdM6QdtSPreQ8e+0es>prV##8vB)Ctq`GEGqQpxJx7?FSu zpUd&=9^0XwUp#*hqv7i2IctCK5Gfz;LAN>FIRwFZ0Y)gykNY}}FwrPFpw^VgbqCHF z{w>?u8oy7W%@v~i zM{@HaUk%}ziShx>*kWFJ=Rc}tU~N5T;l9e>^v1O>)A8^6C)Yo9`SIqq;EtzS0{);k z^_nzL0|GSPr>O7aeJO^5&4@j!T3mWo2N9slj`vneB$KZH89@-TW0F2uSZ(#_Hxuhq z1N3hrFDf6;^(s8;Zjn^SpjAP~WIil18b;EpjN%tlfl^s&q^`soq3N;}XH@uD97Dfa z)VeLMrCWi*kS*m1;oGqqEEupVG`8-|q{7Tz0dO~)e%%vn3AJCxG;tK@g5B_pciN(0??>o4SPCuy4 z?M%uQJo6B)ueoQOZg_CC!%sJO{03?lv7Tj&6_u? zA2{{YQ)_DC!EES4^lO{@Xy-`gYqVQWC+#z9LdAt=CfDarpFg14fbZx+{6`l~$S9W@ zY%4)@-emoS1@tVc4*CStOf}|^5ULCN0Cx>dgo+~qbcxY@)_T6!4C-ocqru@3nPF8B za-gq&P)*q48DsKuCF5l(4JH@7cI;(k0X;7^fj-@_A5Amrr-7$qi)l{9+!C5o)*lF= z1`qK!W6q4&CdJ8LQPl<9ux07A=YHcyx^3e|V>3b#DI*!*;<@twvI=4Kx^smeM;Q z1gC61o@)tDL@;u;mW?9HPiWY6;3*%&Iz|kqUS&VarL}g=1z4>YYPnug40~!!wMgV2 zh|N~vY!U5KrVAgL_lYHY(SuvI(eKy2NuPZ9No5sXDRA#(@&WCqJU0Y{=<6-v7(akX zKlh*CV*R!KIb%H-9>BMIB|4q#XUxB?7Of#m2__q?wFw9F1w!5R4~<#nnry&t;P-E;yPD-Df)Lkz(9ddE|>wLUijOeG(9~*Sb2|Y5k@x)U($fBnU!gw(QtR zQzv#SD^->tJof?ek*E|9pMO01qFRR4^FOpLqI!ql&$zYh4viLTIq9?5u^UsSTS1K? zSZVP(9eIDiidNg7#E6&{e1eTo>;9+;kzd?S7o2r*DO-Mmt|-Eei8^28y$zj2hLKiz z4~mpu{NF)3|L94s;H(7bSenP$vXj;QLt1!ym}WDh3%tcv(8HdkQ2*OxTlPASp{qbE zk=0$c#Bkhu)V$Y6J!QH8udC1wZnqJS^dRzlXcu@n|NY2A=xv+-Ko7mOl^(Qy$LwA? z;cWBft(MrBIlX9CL!}3T@&hli>xA4~D3lj(w3yxjDF~$i&mU1w+IIauvpW^h$(&MS zw^GT32+(C>e0;>IQ`z;ZF{`(H`+)t?+|-w#r>rITUWUt`<#St<8;oi(Z34bnN!m7$ zqP?j)qrv^}!_tV37|rzTI=`6$%KO4&+uez$(fN4CQ)6l2PS_W70ko%eTngpI>wHLW z7YUU?z6fQ(v$C^aZZVo-iD_?GAMSOQ2t9w+ZX^goICw^O?&_ySvv>DZtVDROEZb5e z%JLb!uWcdyUF!m>XY>w29-<6v>oCLZ1-F0c)({XPqbjSq6|i2Z4R80`$`2+%gn+AC z3iB~RL0T7rCPO1dh47C-fE!Scsq$`+fQyH_1&-#3o9=yvjZ z*3wOfK|Wv4F(a_udC)vv+VtF+6hV*m-ulG8i|M=1ucE7-TSdp5Tp0lx_sD(s{Vmi7 zv_4X$RKh|%6N_c<%q%w;Ap_TgxlI*Zdn6M{>l+u{Om6h^3j@#JMyth6M9=7G z9~er|A6J_+G$wWUQqt9$qO+D8)ZW%fE!K5Z_oEr3fl;DC>-T1FiZ)q42-2@?PO2X) zqL1IVwJJWCS^R!wbBd0t?DPRH{_lj=X+ni@$!`Yfhxd-SKK+>+wxyK2rpwRhqRE}) zj+jad_u#1Y8&eb93Y9~z+Xw#&l|cmPvf$0a9drC!Vz~9Z=~HR_*6lPpHb%QmokBem z#T|zr094`!hel}K<}EZlGNPUp_?Y#?DfvKD5{S?LGV+{qy8+ARS`?oPxf67;3w;`F z-Rg<8hVA>=L?zMG1X|Oo-0LG-qTOroa(MdQzIdEnSm;8rod;VC_wuESzq**L=f|IJ zP&iH(v%~Y}eY^Cu{CTi}a32rMX%L<-gMRc+lbmC`=UD$$n@k-G&2ByD-9N<-;|(p# z;O%H=+Z?N1b^kdcH2!1DU;RF7#!o{`w@xQpUXB^*-sDa4I(>+Jn0Q8D1-(q+isPp* ztyUJ$ePiQFlrA^8e|C?sw&rIT?{!P}a;=-15xaxcRAo`kp~da(Z8S7AOkG`_RF9yo zW{*+|r_s+ZjNNk>?ciAz+I9E#HiM2`I81+gqmzclr4C;TAU>l^;mVAhdK6xFZPx1a z2{G$Ov|yU;UJgE&pIVbpVl+hQn;soi*3btnj4M|il>*{7tg*)urpik-T!VP;!jx7Y zF$(UMh4_ql0IvJU1mAYJUc!7InD+x<-o2xnYPQtxKY(_WiuiIbeZ6Lm>bx*x!YP>Y2ohEsc&$IURkr= zdI{H$A4owcHyu_3wzI8`=I=I*T3cG&&Ksh_{Z$Hu@!Rn=YQn+kv5uN`yHRL`QMZC+ z-?b!@rGV%3MM)23@!q$q?epvl)-1f)Z8zbY}xB|j_?nk3N&!hz-|U$!N!l1 zD?~f?;ry!545D2zKlolFqp)WXg7bUk&Ze8*Sf_;OFTZUGRf3aGJ~ncn?aNaw5#Jy4 zrVy>)NligdC%X+o2dBNQnholX&UV_dz1QjxNV@YK)Cq z$0V+e6xE-5d@C?qQ#5% z@a?<(_TLfC=e+aI%qgxNJ9g6Jk3U1Fp85{Tl@%*i(i2ZSqtY)I$k&xuevf|egKtqK z;QB&5UcGN*Ga~|Y2?_esWJ@x9989xC-ROAX^>laAM5`Cc_M2%sL%a1Jjw*2)W-{&!2ZMKj}SB`b1V?L(`d8T2R4VprI8Vrv&ZE))4(TV41y z$#zrV7sPm5`29v9=b-QNmTkc*=2%t6TohaXg*aMh_?6qAsI_zSyxp zH6r@ElRF97V$LvpvW@mG){obpQF7kx_A$Dc?1{a>v`KR<hDB7e_F`E zGiIc{gS=IpOQ~J4%-J}u0Q8=p;i~h!KeRggw8*zOUp-0JKx$y&+D8i%_BfB zokeN)d?|3hG2UkPgQ1t-MI`eeKOstM<*A(mVq6zO&WNrT8B)m`W^Rh(*!qDy}f-jefm_YC;Iw!DQmyB43_Z1G4kJH*L*3J2F(@nS0=RbdOQI^QCGKQF)j{x0h zaNEZ=a?>iKLd6gPx`g-zVXqR1CR03)?QyHuSv46biQ#dnlw&RCd=>-8nw~C60i!YR zSusO1lCFACmT-Y*&>#wk!@BSo5`FL)*1q{1m%FU*4~fr>#*I@uc@|6z)1MB7q!)IY zhF3W;`5Y;vSVH2?c;1=9g@j3qgVOUIEW6PAz#*;D!DVI!mUGRM9#7!ftJH7e=-_IgGmyn3-^50{xslRZ&G0iCNaI0#P{=PPRf4#Y;*I7QS@zB)t&SkiGr2pRS_V1dv*FdFT!EU>US+1fFHe?F zv4_e&xMf?#+;C*_0gZblIBDq7iwt`BHEIgpwTyh<6wg4Qdh|@5q^yK@?HZs-lO|9- zfV?Kg6Et;N_Mza7ettpl{9O>?13Iiu#UeAm-(h%#$qC)?9N0AdGLULZIBh+>{iecF zb|TwM{x~y6;nt5L;?@J2UWZ@G45=|1>_jS%FMdg9Y5yDKJ=eOC4o~jLb;K}ZpCAyT z&pf1!9(XCCCdY#a4NK@siO{gdhWHHM)VDr4RQ2qJd?`}q=n;=m*9CJe%_uhe7SjKpJ)q}M+E3Hpf{xK;ehr>81K0jr^h9hg!r7& zD$BaRI_*Q{#u*R*$!=1xWuAVl*UK9ryheO@eZ9uy4fghq*Ha~5WkHWuPcv99x|r<= z9qE?b`Zms2ro0F9$T3mGF=1uH>D%jO$i7Tz8)+hro_8%Ih1Gn5;le6DUOA)-KFXrf z@L~?G2XdR1hqU?>dD^v`H&eUk0{c17xZjm;-GE3CmJdU#Y4UJ>Y~x;i_K)jXK#Py2 zk`hvk&m^|Ft~cs^(!pFv7n}VeX)fYwSbVvL4Yz=%v^+!|A6yT!>dtgb*!E1v>dzkw z=$Yc>eLwy{ch+!~Bq;8Oy;t<}=aCp6Jd)^#f1;+~#HIFM!+AfK;@ai&EcyCOHO+v( zIjVkQx4G30eDD70sCN3vpGK$|IB~x=g3%k-jpAC}O>J|+83D^bHNmP@c`N3-3c-1q z)p&eiM=z}#9HixK<@eo~iw|g6UGlb%9c$9R-rF2?hIcKqFKqBJV^>qv0QEQsowlor zgeB5WtXX#za7)p(%nT|UxE$OX`u=bj-UPOAE)^J2`!x>@YZ}a-n(ni91Tm9{0JOA2JMKs~UI@x;NzHCTPC3Pn19@P#9f$*A5$EE?dn)1j%{Mn&Y5rL4 zhu|G{9>@o~{ebv<&pm%p=Y?~)ew(LH&eV( zgjqKRTffM&pO%OT_Bs7o9+1z6O1#Z6BLxA_GMeo z6i?LgVdoI`P0+s;ncfL#Ae?t(JJlz)nuj5-le-N%e2)}8_C{mBht4|CqCa4xH(hl?w(w4WjQfF5uwYRlVyhemt2(m+i!!$NFrrc`e zE6z6h`RQbO{`7myYvbDAPTZ52@6etcGp0mVOf(*>KX~h2yGoBcQ#~tK9|@}m86oXk zZ(04D7CwroHdwH|7FItCJI>h~+Wrr=k$A*tEX40~KF>O=k%2BkSHdL*TVb!*VEd#- z`uOF`txt}pZM*)#F`|s8U>hHp&M5R$a$JoFIAjjhJP=+X@J6}ffXMsTzdlS~|N7@$ zaT#JY#N|T|Jy40(SVtIGCo>RxVO4$HakfP<_CYL$=*w2dh+B^rqA|i?Kk|V%Pe1)V zu2nSRxbn)Y-8f*)4DlI4IMdP5p@ih1Z)t>O@tMyN*44|FEphXG(M9LElOq;_Y4S5) z{gFb=hzQVSAZT4>zw8XRr;>$d-1lb6`+#|1Ij${-3my8pov!*b~A{?YTN z5&hB47Ea%jH%)++b{-;12$WGTTM>ftW-c8n#i(8aWW_&ZB zUXxVol>kCF`=mztm>_Y-fJvi?N`1R+vER`k>jsFqur9vordur$_I%$yh_n!yb+;A} zt?`4EbClII#9j!~5Q`by#~U|pcGrXP@;cJQUsy;7U2-g2w$zOS7S9mRb&w{krgb2O zKk>xVO6=AJcBBX4o!xKP>E}0zs(xmo0CZ2~IHR z7d%9Kt_$H>hra@C@r>_UD`pSQF+MW5454_awJq35unS6K7>YhkKqbz zPGiiTDL7n|<(?T{4~m4IDC_m$gbBRi$9vmUY$2keevE41MZ{?zpFWXLSUxx&{{tW1 z0QCgZm~UT{@Ew&`E=!`Q$+e=z*I>TGkBJUz=gYf}!7|X}2`gj$oP#2~er&e5^kc%} z5AF-z7ydHz@)8WMACrC#6mpE=C9qgvXKbk-&7cV7y)!zf-(?}3X$zA_EiE4># zoxUL0khc9Trj#|;2aih8uHh8@+XIayXnooe*5T(GK1?CD8){NJuVGvI_psim_He&B z&U#8q-xX9H5t@Z(EGs|A*20Cyv|HE5Y&d_=5T9{wF-Kg)cbLEXO5WiDGdyA5Klz|7 z2nDVGi@K(Kv2u8(ih-ZmMT-_>45#iB)T@;U9doQNON#FldZTjaelz)C_ zfd;WsxZ9)M5>{w2v_&z14ee7=+>g!=`$uO}x=uRWp zPGZlGUYo|>jtQm_%u`gG)_=a?#|GPKUK;+kdoUjT8Rqtq7pD>Ww-7J^;Xw;qlsk;4 ztRH+d*8}kNjL~TDCk@U$vfUD&qet5XM*)RhlX?Kc?z!ilrL3T5&z|L40<)!akexd* zvEWgWap4E|8&MY0AQ&%Pa=_|Y_bZJsx<6;0Y*5b|te3;~GdQn94}HV&7rlosKCE*d z+2?d20<>TAFx9(As4Pj<>BQ`pQz4o(+WE$|2931 zcxe99uR+{*ycb6IyJe*1*$bn`mtO8%=D}n2a`U#Q-y_~%Bkj6zI4>@Ly59Nd_fmSi zQDMAz;%VjM-J`Lz{OkE?X^A-khd&(Wou3h1c$P*~`C$v_`LX(#$z7-)9(<)Xh9XA^ z&$$WG{Qk4vUDof@?=a~rrzB}&dwGSoN&vSTU$xHVGjAv5w-ggc#hrJ&NuE2TQoM(a zw4u$x@;9}{pl#+H(;5rhLCFFuKt&b}eovUS#PpLo=qsmnR%BA8a)WcgxqSM>P9?0W z?{3UH*O=iKCsf5=DH>StT$S+VAi3QCQSh}WcMG2Hy`zpgIBOWLH~8z-apk7siuWXx z%Zhpj=YVth!pVlJPjQ_}AC52eCZW`1dj53u?}fK)_}Z5IYnyW0!w7_n z``U~gmdNn6C%t&$6l}}s?JT`L7i`=4^AgtH^uK@fww>3;H;CQlc=e00cAqyHyT4sn z+P2a1xp8W1G3d?SlyZqtFI@Vr4*JViCaGU0@i`-P_(_0}3#(vQE;A5kAtu9zGGi|r zuJFrlG$0-`5TD_TnYTe)hJ`iUYv^#iuxiFK#A67=NE2y85au`@fBYHcDue}MxZi-E zXgv*t54!E(^T6bS*Is+08~#_n`i*kW!5|O1#dbc}me2KH52$Cv0=fc34Rs7!n|i&a zUIj81OE)lKA8*_<>h<6m6zzs0>LV3B^gy>j)$I1<=?h&@PuW) zPF`M(tjj`)2|UZ|%|Z2do(RxFi8<{DyQuf4Qz>d9p5V4j1jFa*(>AQK88*RD5byhD zxlk~Dv6Tn*ueC1p(OfTWPQ4#?;K zlJUawcRlR-n40Je=~8#U;Xj7I`?}K(KEk$y##B}alro$>xJGDLIN7`Km^M0bNelhk zpGN79r^l!sIc(1s`p(&%$}Nhrcy@$#V&T{EkZtQbcDVqq`7pNo6C%AX6m^~a&x2Qg zA~$All$fbiP=nWxIDZ+Qb>B8N-R5i>{``a0tJhkM(*+cT*MaTC7_&RP`S_0>o1jyc zjM4Z0)IzsBQ=iZ5k$a@*`)Auj?imflVp4As$_Adlz4zXO3Lz*+>+5Uk@A1 zPpO`FxZ+r5iO{fueq&p*qHZ{42lhx-4N<*F)MBh(J^oGMJ8BG!C}cpKr~GI-fA7jy88{i{QbhGBXpAAY9(db>nZ1& zrkrrel&M^Dq&Q5<{=31tK7U*}PJdpRpqDl!Yj%>t*;b=}jwLo9x-VL% zEuk-ILu++0M~_QC*0}}zO1X|-$5wg%zgut5>I0geSN8LnOeWQDeBAzav_#x8JTZgYRvks z_VxACjNQZ)myphX{^xBOeL!bIZ};Ij#JyW+9MvMK(>qf;D&KZiw)THH&mZ5#n;soi zmeA)O(WXS{M#J;}%-@Fujg2Mf#W&W`(FZQeE3PS@*kN@9#_0#=cZKf%#_dD2W?PD` z`$&3Sxp&B<8~$c@j_>^GRO`K7+OlTLVSVC29rn$vPxOavwq+N+1@{RU+FB9`=!qI;K(&OE8^$@)#9Y zwI1|bD>1e3A}Vzrf}%h%@t31~v|`?tvMVB8hI|r)(xZ#dx?manOJs%!kv2gg zGu;}=Px!knx)B&EmQt`|41Xc=m`e2F{Ye9p-zDjTN1B$XJ8p}6jsV7Lo8ikrz`T>! z9Ys!APq_z~?mAokdKkKEUrMtc4J5>{oqi%n*M#|D8Fba((W>#>NExRUp3a)!!vvBRjghj}V zczeb4Nq*JghO&r`ql6&bnzpEKy6gTN`ECRMWwWnEpdj4hQy0m$vd zu%Aov`KiTo3kxpxQCknb$c2NCW;NuV_l9!umKfjoea8oZev>OhuBi3dQ8n^$!SUW}w$sjkw`aP}_IN~%&d6kS6lVF9-FiA8^n}FgZLt1N*$SYc{6*r| zLWU(Uq+%&$eFzaN-jcFK2h$UvUCqxGL%9%GZq_8e5aE-YV`5|rRZ$YT8-WwQVHkcD znryL?$OugIWT^b}BuL~6bhhXfmY--m|IcZA(mEJi={)_OFv5;=1Oc)wvTadbsjw=z z?fcnK*3QbRnJs!_p4e-q9MRcyL_ZbIaIcTMoixpxw8n~QTU4uJ=1vn9f(`Gon!)qd z8auiW>*>TSs2lEm;$8zMxGOx9=2|0W`W`4l|7qV6+qbfFKbl5BBBdffH~Pu=?&i>7 z{!Ly^LWj2dBwxG3A)60qj>|nt6_M#q_{HT45f{%aI$YKLq&jt4D5aL;ZfB37#lz>} zdEfSK9QQueZUEml@V*wS8W^Dl#-0Z$ z#r6ROOVFeQ*tAPo3XRUcPuV(hX_P);RIFIvl*bA%lK<$&G^@+qO+>KP;hkoK3s@Kt# z7;9dw{7-TRLEOUkN8CxtkZuKeVxT3tk}Nwno;c@CgIz`F30i^BXO>!jiDD4 zljSwuhwbqvc_<+ol=B)~VGsLBbBxoXt}^-yd?9#Do#|$(dm`bSWj@-R7E{d1{PuXE z{R18hh_}&j>1JuJnxu&$o;6aIQ!px2P5b2nqC?-gp57(Me53lBWKD7Cv3>b{sdAl< z`ql^fd&%1paAToXOWs(f)pBx$<4yzscY!&c==$4bWCzC2u@TI<^u6e~ zlDxZt&Vqc!P5Ik5q0h!%70$f%k^eiP6y+}Kb#|@V&8Ny`vnVt<^O2yAJg4)G0OW-z z!I9tdRo7rE_YyJ}=Pmq^uXIh=10iuZJ(Sh6^4j~cY`iW;k_3jT1Cn6@R8o*OvnqsT zkGRKqM$JRYWbmogPyG{5+&;wS*Z@wMAMYJ?DqXKW)o-tqEY>TsZPbfX-qQ+~o&*Ts zNw|X@$7FFVRP82!$*8fxG3lQSW9P)<5F%gks4QEWTdX#FR}@CH2ym;XxlQXk{FyC( z*8gib34pSLZ?i`=LKFxtI5noPp*3Oa3#E=3iDbjD|9h(e*WO|)=Tn^P$STiI%2rKn zpfsZ7eX+x+=-6_UE7MRuQ^s-BktqewS{QMlRn9!My(V_)Nh#kb*vEE6kI5uY^{FD+ zIrhBBxH@{U3C35O_Oz2$MGGAACaUq4zXbJK{>BU3-#G|(XwFh_*I1{bE^W}b6j+uF zvEI|>DG!3_$ODWaTJ8W8PuAV2Mh|6HxRAy#p7M-V(x2mcDH3$9uiw94*|9R;&n@%d zMxj1p%r?CuaBDNraorVknclTINhm31QqSj@xwDXfsHpk@3-@jm$|d(adSr(94u z{R=6Ae@j{YttzUE_W-@G_2;cdw@Z71s;^atW$|8v5h3xFCEu7Tp|$A`9VsU^#qTiQ;F!9J2I ztg?L>a#I-UA|uBg_ZG35$=^qj%0Qq$W%%;6xnW#2W((h?ii~$C3A3VgwxgG-r75w4 z=61|^Md?{RWvx9A+`5K7gyZ<-k&R2LpY&23J`ps;P**RlnNrB>aSb#Y8Clg_IyeU6 zhz)_$_MH*fVMjdQecAm&7}j_CYR0IvUHlbkl6`T7Lro8rC;C0Cm2)P-=l*6B%j!`B zrnLWI=Ern0(jP}1Az!$SFa!$-!eY! zG4w&efq()M_`*|tCBiwhYi`UGl-Sday+vH(SB{(N=5C53{z4qfYnsYlRMl|(t=XYj zkE^S_zQmlosYjoqy+HeWxCZ2GOFSaBJ^? zT6g+^4T{tsh)-UiBrT5)4N{{M?2iErZDssty6g*s+isU7wqHb()@w$V5sga7yU=`w zdV|QQ7GXw~Dcf+zQ2)C#XM!O}%k%`L3lg_`$Qb}v$XCyE`alb_3; zm=ME-sD{*{u%tr+PB{IImoU`QxrhW>}N_$Qp)3Y zjC|&83R}Kddp2Dfv&fw6D;d}JwWZYYa)2=)(Y5z~aKVn7@S;t18-4WE zp%C!YlhY=(-%Ee>P=GiZ{w@Cs*k|fi$uRW({l%ba45o=Xh2%G8JU`iEF`&f%J~(&clewXp-IJX&tAB{?&I@~99jdm#vPoQ$t=GpR)Ul3IYx!T#Gx5Z~tj8Fl4Z+^_!*wem4)`xvLF_9$!H-2?EMCx#m-N z!yfVA227+6HfR@-0`V$()amfidHwj{g0XVLI(9@Mndiw<7Y2TaHg&Ec!RK>1b1Hvx zfl=BN9TP;5#ocDem+-@|0bEi`0Z|hH1~yK3x0^~$X}C3*D$ew1WvyeFvWG^W)Pt>H zX3A0C>YTd;dMc_;tWo!Cob;ITI-3$JM{MwdeMZpKh57W#D}q&}`>}UM*aQ(?Ieg3z zJtLSioFW_y^C_}3k5`b#vH~vg7rFKtncyNyrimsUN?6Z$MOz|o%em4Yq4X0Xy0?wX zx7=qp6Xr4bX%0=nUujty^%|d)`$_WMY*O!)UX0}lj!UYc+wSx~E$wV!ZM?%>^bNL| zC9D$JZEOzvxeGi_LB`Wmfke|M_&uivwv`suIpI?fm602G|AuW~G88VMZi38=ME&-H%jiIFTAL(jyiX(z6CG#-mJqgH+P&pOcWk1Pqy zFB!^UGNXa^r`zAQRZRwch|Gkk3v3MSeu0|ky>M&_`?^b|>H8O9zk_WqM=OjK6d9cw z$H78{qvdZ905QYk`GQ%MM(dho=&S?l{;Z|tv~(J5O0eghx80XtR7cuIS&a#&(gaR4 zxff1~4AP%`FKlM!8zS317ZYmj(F(T*`B@sMEAcx|@3S);tFD!%m$;(=*WaiPQr0;{ zE;}e=1~^ipKKST!@vQiYA&H15xoqa9b#mYC@{!>e_B0n{*x3m=G+58fE#*Si=oCi& zz^ZBlo!QT+I79z`73T5(cIA~l+-6^nVDacsCg1oueKSO>sLS52zQf1OO-0bPbSey2 z3odM{p3BjB3D()%n@n_UU7O1PN;h}Xe$rmM#awWI#nZr1MTY`GHiCubm9|je+n-A@ zweHBP>09Ym94fV27 z_gBUX<3M1`rnlHpeB9nb9&w=&oc4ZqSHFy(%lf+;IQ`+iX2$sR=mZ@bEqRXHe31#X z6t6P9$TaL|9C=*bZFu)Hvef5mE*}HE$KlV= zmVI2ArmhQ*_3x;k%rh*~9t2-qe_HEz*4OG7dkB_aUIG5#-Z5gO{9pDNo$&h0=0rg| z7vvGbK*YZX`aPbCkH=m3r*EsKR`UAgspBFnVK6=P)cGQ*&blDo<=uE3)cQn)_NejG z_q~x0s*)NE24ZZQb_s$6`E9@d+s_orfD`8=*I<4TMCRFDCM&bwIC;o(F`3rZ?MP!S z6f+{z?YK!vF65c_yovS$djy_Zt6XMhnDxc zqAZNjf~B|7BJ<)@i%Gs0X`FnfPBKw#y%AUP?7qD>kv0|i&gKW&-3l|h@rfYirwUCu zw(E(Q-Td`ABVa4T(3?ch`T34WV)1(sHzFr4DM$z%32slLkz>uda^c0J?uSs&4PI(V zG`z4!O~eSVt1-0QKKR4@5IN1f-HMcDmVbG~N#94VeOoRNZUpM6L96d-aLZkpYX3+LdrtG(B9L^zoP(_3dKWK)HUsj+!kz%q0%hJR+dVl} zj^|f^J)W z-YvyabJHi0=Bkosavr%|%HRtc)(|uCUTX#mJ3G8I?{w16=Q>6g6%T*roJNE@?%l4Q z;bt&3?Y-b+*GLq|zva!gbv(Wv)?Tfo^6Mg7bvm~F3E2hTVftQF4;w^UKf<`?fYWx7 z(=0cYF6hyF^}5~k&$lc1`}e2(Ia?BRdV!9qBiadn4Viz(zBoU0J3Nm+&(tkYT0}cAfohu7}!NoH?vPQY=z_k%be*}K`xI>0wO$hkFw1cK-m?#XA`(_Bob67mYu*;d5q z)?SBwyF3FX@BL};5GSovvfp*P_{Y}uiWmtcf1hmuN~*cp%8)N`n-%tP<9nXEDA#rD z?PGlh&AADav34AtfJ^fse0^qG28R4gAfv(8);4NsY3V->q6E)-tQ_(^Hy~;9@@mp? zKJIUPb#340fv?-?HS^^al`gzK0p^XTN=~g~#&420YE{L^l11-r$}py)x%=k$ZzWHfrBj*!d6g7ng^A)N+7 z;S7XG$q<9y3hqL5+O~&L|IXf*q(P7N0*4pUvT$|b^uK~LFQ*dsqHH20if!JHy_}Yr zCB@J?B}>N>I<%m!XOsX%9LfsfgL(Cni_lHGuhy;w^9Mc;tL=p?yBee0ykw$D-e4G0fsH{`75fmdR@FrQ(j@^c&n=fe)asskhTiL1OvZ5Glc6ztCy{ zwF?}APxQXrQhMFL9kS{q=5=A=SYT8zUn9x`iJYuPADWJ3b!?Mc2oN+7I<}m8DSHy{ zg0G2XXT=QjAbNRWD!*cz5Nnwhz=$!Ws?eA*7 zjBBGv-BFiELZ;nVtm@_|F%-^)n&Vl_@o@+-K5sxD_Y)l$ZfIT6piG4zx`$q zngv>{DdizphC&v(e;tAG_$ zd`mXpCp39JYA&`C`x^)DJ%Iq*{!fM#RdG|0OpCr%E zw^BkM71rznox+NWEGjuy z^3I+9z!-`0URxb$|Fdd`KL{-YNeZY%<(Lyj_kJg`|HJ|PgAderZiTJDz2P-o@E*~9 zF+RP%p1p)^s@ve4DeT{_$>B(S402(6UztfK(E%hn*SZZ;Ay@mhKlq%qHQ9y&A@%e}k)*8jPu67uEe>pr%&&&@J?P3<9G8GX~PT%T1j;rC*N z3?zIgxMC)sxzW|R^7Bnm|24E5xqrG*yPK1kO4QZF^0fEuYBk-IsIj;`ixxQhTX0PW zA;S4|6Lx(EJK5nWC7ev@q{T8GNEX~@9_TY`51W{khDZs&k|v9NqC^D)VDz+pn?dfz zFBTk5lr>-c)LO0_8Q2`ysAO`&N)nZn1g)Of)ik)=-A!Nlc)f)-9@!N{#PpoG6OWsU z*rVz1&X(ZxNk$;kovvlXHKGx5zF~gY$__oByBJ=+vv$&Dt8=_RM|PtselW&l3QO|P z_v{p&kQh-*J{Yx=e@As8pzK}exU0flcDHqp?h$S9!GV4F>RPiOGVlsH3mLnZ+;Ue zZTzUatMV(XqYIu)MDNn%w#Lk}Lf0`e{owMnFCLu>-Ev(gOWP}6yqes^T-Ac_U|n6? zRFvgAV`5yJCqXotS|6QaqPL8VV0t1g8*_GAagVz#u)8anEOk_R@v1*(!aUAi7LQ0nU6XyvFG2zyY_nkGETwif_f8cW`WceMuYCh|KC@@}g6@;QE zwuF)+CjYZxx3fu8@s0a3HDT|gPHU%?I5B}8T*A$C8lQ}qHFg}!uO z9^gmm?AkRmnU7n#Mp79^I^%T>yF>hp`{{+domT3(@Sq+pBclYcE?{tN*eB8jVsN2W zhWbT(`{VMC%jIs>YhcCV-U-y|Y6kM3bT`s_yhGDA23t?i9g$=jXR7NcZAF;R{5fZE z;a=VvtE>0*V`)`41guNprUKli;dT!8iwbC2*;*TuFf3^fB0~lSA)kJ&R8U{N`mapw z9GYT4K42a>Z(1=E!g_a25km6i`b$VYcd2a(93ZW|*@uosGvg-_AI{$xvS$|E2?MzW zR(KCRwrE1|L~=5*cHLn`@^=)Tiu)kNtnW_T z&)3}{!LL#Kb7+eTC&%LN@JFV}eP7}${K`x=qQ)NK&N?nujo+h692?sYhebb?qhlV_ z87Pt^$RCRtc&?N-Y5fhZ)KKY5Y}-%K&}fow=h_IQdU4vv)D(NaksH&pyuP^!_A>#S zSIV?s_b_QuJTgN5sCczD3PyExOnC4wuJ5dA#^Lv44xVqv&qx><|*{a1(l%xLYU*CH}3;ZN%t^tuB?RSLQFRrorZ65Lzs zQjGlE-Vj|aXGpf3w6`2I8r6IuDyzpj*isYuUT?d&>7w}ZvjRkl;75=V)7hO=iR8A2 z^@c8b2`TRYjT}C+p;vAoef$2wkL}OHyt^Qooi#mR%M@#6Omyex^)(z<)JfIY5X|{R znS1`%Da4LoSb3pod3gc$Hos{Zm(Gw+i?Jtu%0cYyzo_PyF!?1kQ^?lB zMrfYWF%{$_@-CIN1YV}T?IO>NLv-u=&IqSwQ&6kqFB^G9g4_o)YfU-i&0pnid^fB; zn@zI1psUb>z1=JjH18)|vTJ}^Jzx@7w)Yt3cu6yu|HI2pf2B(}xC4yEFlu>jZ3vx4 zg+G4X!1zf+RzghXg!HJ?gqwN6OYoxP8*V^s#{WZi>LG9S^y)oxXjQdrm^$qW z3>VX>Y@&w>)OEj3;s_PR(-Gr_T4wt2w!ydK66zQXI1mWLL^Z22WaBkn+yeC+b838_ zzjYlN1TX3jD3h8eOc}D&#NJ@s8Oy+smS*D|dnfOT)wx7tFTLu2xu_HsQg^4w*UV8N zl1n=)uP^;!rznWTD&q;waSZ>Nhcx53zRtqh3Z%rj#c8RSo%3PB5sndgXrf0zJMop8 zSHQ?;izv0>kTg}o+#_{r_dV4ctMIX!Ej&HZRq2+7hN%LScFWYF_T}bC_z>$O))dR;!XOu!pn1eH@r*#6Yl=jQsh&#(rGb}pJcFk z8{K{k1A7>TY%A#)DZ9TPG9XSLSGbZUNTPl4g{oOs&q^cjh7u63N{vy1Uq7$M++?<# zXt$@5_`_Nft_jJ<$`+^X{lVl6&{{>s2;$7G@bSxg?E^ZR!kpxzUo25UtRb!S-f!Go z-KT)CB{iSproyoCC(J;<);Z5VE?{o3>&7);InafEAKvA7yH+WI75u?)eMa6ul24jU ztKl(rSJQ7D2yv!zveoba{A_OkYt2&H<}+HNdvn(-GGM2)kcOLStizaI7_wQNSx+4e zfg6DctDnGhOpLX3{0()k_K6VT*udrFk+ZWR58#t)6?$v8{4R2LoDJyJkQl6#`nm{? zYBJfL`@Sq06;u4S%04F>p#BxmGHUIcG_0B5`h(QE_`@G<-V3J>OnfYv2hFx< zxtbu*cb|-U2Svc_0%xjyrF5kP6HBF75NAL> zy|v2?r%;AB|0bQYtr2W;^ls*g_~|sgl$DzzDUiBivE9~T@NbQ#@2wS?7=}*vtz_el z*l~sxH#5^x#T+ZKT)(f~3Q11abUFxlQ%J^UNk=J(h!1`PmJ$_M*a`aZWujHjal27gfps zB?O=(S1^HYuwS#dc#*&3f}YHHoiq>6M%%>f`p)O9+ZG4NYdo1^-3Cedi^AFRI$a)# zz#H}V&)c5mo*Sy|FwH(I$LSvGcxvGg3mR-}<<_hH7SwM7oHc8mca*$g;q1+R$r?OL zBAWu(kw{$`cvs>Aq4CuRvv;ZFYMP1?3VGr2&afH_$M`uU(PZ;I*3nxx>{0lBE)UBq zW3h?~&s1`>cyV4{Ae&l>OhyI(fY>;uG)FAhT!v>>wYgEdkp$%^NAEd z$&1@g*z4C$v)elIX0v13+S<%s)B>4!qPX~3*FV_FYpvnLB~T%`meMTsHCNq}HQnjL ziKybZ>C(hJ++}J?qoCbe8LT}vDF;`R>kcS^_JgeHlblQ|CpLq3dgCU_Bk*@5mxP~E zlYX@U2+f@RD&kb?VLQzxJ*hvm$e!i)IsFfyY`BKp$d?pNk{O$Jw{Gri_(utx{OfyW z?HU;i#}fca3LrG1fffof>*r50vD>{oUzV?obk;(*Rf{P9Pp+w@lXrB*Bov4cY?JW3Do zLHz|~ec4q()^3R8IT!7Fj~3&?|0Wl%McNv@%bX2G{4bAZ?A&PUdiyL?C8z(d5}68V z(4PJ!e^Ff1)^>1YoZ~$>y@KW0Lw?8<2{ezj7Dm_=VE@V~P?!xTRO% z#}tW!$hxm+1GTj-h2m_*@NA@1n{l8@&!@=X?U@9^G{`+>Pb`735EXpV8kQCz5fcCbhGfYANpRb>W%qC%j8{x5G+SnRfZJ zOG|&I4v&w^|6CKgMp{6dX*$cWxzpOvXPs618(i}FfWx9$#D1QKER|En@%P(a7!=rk zEqso06uw4Y8$7Iw9oLyeHnm<;;mx1aT4|1k>PQKbIYP}*VoHB|cyT|<+)PBc zSPZAN6w_|@*U#Werh5VnmWuZw+W0#gHx9E3Fa^mLwF&E&)bz+NXmak$+aVvClN9`? znCHf&$cqcUa;>-k-ehV?j6L93#pQeFV*J}5@}P^D9ry1i&tuAqF+P_+fG_8>;c>@P z@|FJn8`teiVhr>qfL=!$05ttZvI~UXYus2c($Z`BE0$$h&&I5$=Jk=MvyUXpe{fHG z7w3^>fj3D}y!J=_0DL8G{l@=NJeC?Or#pG|B8)PVsFC*09rEMW$8}gG z^^4a03l8Jo5t6f|k zH7P(XcFRFM!aDAdJTG!!ik9|#)a3&WArkUGrneA)P78+LcPR5|>>r#3MuP98o8iYO zEdz&T2fSGVQu@nfhmHx~7|F|L@00nzyB{sdx6v6DCT@VdzaYOJr1OW1HM8r0Tj
(>W z5tG}wkM$>G@KMK_=Iz)lSaVZI9i=Mv{wQ$phcEc1&&^(fG>ZwcEl@H-xZ@re6UaPz zELs_L1W5KVTJ#;L9;SUJ(6xZDUn7?8*(PVIgf)4F&m?=iN&9AB!VE3~5Q(Sql@mlH z|B7|Qse)37!FP_SLQXH>`7?g9dK<6JV`Xcr&wlOu}nA-VB=!FIopc_cI-YgB7wF>dQO^q@!bWm6M|jB4JPe(xYbXNL9I3j>Sb-Z zar_u@bl_oV^n0i8_M{xVQn(pl0<{d!1w{H|K9dwkDyyv^P*67B3N)t?s&YkTKZ+K5 z1kvG}-;@%e>Zc>h%#M}cm%LmE%=A8Zl`6T+mJnG%$3li*D*pRZ|T2)Rph)!~ivpjFZJ^arENw1g)CKNXVf1 zF`mnraEI!O*60v2jf(+WF#+!MTGMSPBJH8=NBgZ;<4$8F=;kLuTW#qzS;%)c0JWj~ zq}TP5aP9(pt-gXaI!;s+oN$M;Dq6X_7{M&}O==_oxu~^<^7dA=P+whr(mStxp zb5GA8_rpCT`p=W|0OK4jv6Dgg!K}k}yuHpdUQ}geU%YHRB%bS7FEC4@KAnqw4RRhM zgl^`VImu#bkGr?H*4?@C@-7Vot<@VCIN(y|0NtzkmTz=OD3ximGgf4CW(4Q}^lF!W zMTBcRAWw&>Hd)=VX-kZSFl;t?;lw-(6>$ES@a&mi&I2Hxaruu_MuZf#8iX%|a@km2 zyM2k4JaDWV%N4wn=uvYi{^o{UWAswFNl>5-0}pAoOm>p`Hm*!q+c*@y`cP10fjcE; zXlVE{p6Uz(O8@^RHYbzTmZfp{qNWMxhG4Wk{3_!-&%;j_R2GhfRUAI2ObCS$M6+ej zNWRmbuV5iuV$#1zy(QAgb#{j93w_D2@p6sITxm_XOqU2(=sxGD-F`R+-1j9PIL(tu&T$@jw0KSN(VJr_7D0$*pMWdv39&X(m`2J>C%ABNz1O@f*1vQ52)e z5j82~m_Q|^!Qndg{i8YMA+ zx&2+7Yk`(=+36%5NUnsOS+)?TjEOQec8$l8CgzPIG^>+BM&y<*dGq8~>~Zs-$Bg3w zuciP>*S40*u_`}?{V6}8+gHq3C1s-+E;c_Uy5p7`o*T=mVK_IFET*CY>5RTvJ` zuTmN1xOUk+?7CIMwVzW*w||m&EDt;Eh*w=~LGZFOzmTO5dWL&?wPfQj3v@TN12zT+ zE6Iy6)z`t||{o-i)rHJ9-$gAH-qdKPXhl}A++)+~Ia zLZT1cj4TzxP}j5wx&Zh;!eh`J2Qz(-8(xrhdLHjg=yy+D5gbxejW5^x-9kb0@{xzt za<^A27t~jfpP8A3)J8h*JXX$p4h{|~ySd%Q>0n(Mk2vy7Qdk&QQp2cs9$hh~^YIn} zr+qhx%@7-kZR*Y`G?4mz*%xY#aFK$DFnTF~D#zN(7xg5j-6qCGr{9c4z$2lKy;hv~ zxTDJ>WZsStL8wr3YP$%60}UT^sr|ebY;6jzyy|+1%lbS9RsYW<@x#ne!vVH#AuJ*s zEcBLBdf}>IK~!3b6-5D@$n3vo>zH)RljoWeF}%&mWsXn%;(wn?rX!K% zE=5uq@ddMj;rS%1;8|10jOp1O-U@XEA1`o2sN^36k7kazb3B`zI3)C_<8O^EPIIAj}rxF~37N-FbW#kJA~x6~h5 zvgXX={`{E?p+%SQv_O5<{#{u%*;R32`?Lr56k#*^n+YBUjkSx8&KDMXpod7%&5kMy z5h3KV7<-VX&PbZ9rhz@)VA$``lSoV8?XIp9i4LBKs;wl%pthFo^t%{+U%vAuvhwz( z`%T0#)e~5cH4AZA54G`~Dd0YuCXIFSTY2{l?lko3#Bms7KB%lx!#$Gv#5yC+a)VIW zA-(XpfE7QTpqy{hQkrr9>@}9_HrSJaSz^l|aNcw*J>`j7>UBy^=&QGk>@3m7BaHnF zH9in!CC$^w~XA9 z6^&BZ8{uGk_`^a;U}4XH-Xmbi<}jH&XMkL8 z;AC0zu?KyolfkRz*AmEyQAKS0!=h%G5z1Ldk zllw#`34K>gT%h)k^sS`-az)3@ANrl&_9LMaW#S;|_n*S?%JI+IY!@}>NrcW_R=q5X zV+aYRxa+;=wtb53o{wLb=&K7#&r`XJb{rD~rVQUmVCd@z4q^_VcGJ=fol>Q}vJG6m z2fOQtGI^~0EKd*)yikDLx^;&BJSl{f%c?f!Mhf3IO|X-jxfCKfvzW4^F3c&cvUEs} zC>n+zR`RH)$D=_RSUIWx8t@z?5#B|J12Y#%ibS;cs|S8s(?%Ldctir1cD~L4b#Md-x4&cnzvw!%?u<7dLX-^$-llVf78?<(>yNsU1x}geFgh%SG3}vg}1X4?Du%GNsHc``shAWpHyA{B zd{WlnrQb%ji`kqx)M4}l;jCFKs7Mn@q9= z6IO4?x4<-~MaFOl@Z_ond2{JY*Y8(Dfk(F?w+Tgy&&0c^u-gQP>+8T_8$6sRjU$sYCTiK@f z|9W@3Rbo@*Z>#cNPUq{wL89>;sdr#G+XN~Z4>uMlpfX#d!yhwY;wE{Yuz^&hF=)+L z_18D~2$FO_h>5_JzxUk}Wo~4`D~3UMS%U;vAE>@OZ+sysj=uUF#T39fr2UY+wCtbP zYrIXCeMQNnfQI}$mb{WvsOXk{rbzQt41%-C=N*D{HtRA7H4Hg*=8x_nPTOprlv=Ee zB%;T>qBEt$SLh&KAutMiMAZ5Y7qdnePX_Zb(?lCMLus~PwJbF7v3J|YIl8U@@hI<9 ztm&I^yr#S^w1ukRC(-RE|QnN+N{%IC%R?X9BkYS;(jpJh~# z{y4GZ+))`aT8CkoT@aB(|M~lbTD9Q^L#CmPnv`efW3_|sb~6{`BQV(M##^O??F3L^>Y1X6$5Fq zJ&2jPBSdea9YO&RkLYRg%{rAy%>*AzMu*!IZl1(YGpfnq{)yuos{*z%ubnvV<#?ID7`VicmaC6?gk@O4#j(^ z7seta2FmUwxgL+RR~E*FnPO~)V3=p3S#T2qXwy|93RmaRkO6^A-UnqMo~*6823JpN zo^Fz=SBqK>ZxrbwnF|ej%4o4O#&GkLsTL_K!B{BApa!76U(Q<9{crjw7N+*-6b+f# z)E2u6b58}Z9+)fz=R`!2d-_@Q`Jta;xRRbN-FK)*=yC$5npSK;B1^*tz0_TljtD z;G2pz6`6(WuqUo@Ci?n+@~fYfu3QblYiG%KepNkVs4?L^dSgM(S{AWrSfc$QM@lFN z7nhlLGuR(M8=}0iOg~0aKW~BJ>iw3v>m?xP)6^N?^i=trw4cn++>H%&t=xG7j74lj zn|=P=OQy_(_XEev^I|lw6ycynH6%hcaC+w0_xLHbu}p;8VfpAk|F6dlyJfx8ZT(!1wlQg4R*a6r1-F$nqbc3dzUeag+^e$;7xLLf zw9443H_!)z$rg7O*8$~PFK@NYv=O*7^4Pr|{tbOA8MiO?xku=4hAqaR6-&?FfB6rr8f84=YsB}>nD(dz;)KJ}#11Z>? zDVH6Oj`yopXVPBHNq!e27K>WC)RQ^m>X2rjoBQy|hKoUbzCIBuEs)xXQaSLqsqwKL z@I%4kv9UV%9~7leyMx7XeM9EfKfCEDI+%e5_DnQEce(4=H?rTe?lo?^NZJ#qjMPZF zU(F%6XmIz^Klw=3#P&GhSeDAuri5{xR9+B^8$&DJoYP^!FwU|puZc2Yg`flwnGAns zk&F4^b6K35FZ6T6qTFj@DdmBF;|s74joDvfg~Cq2&lp&Rycv{s zW;a&uQirsI{zyO1^;PF5(FV6hJ9W5u=MM_EAr9i8R``+Zjb#%qT*fJtvl>o24&GG6 z7K)IGITD79qgrzxK34K~%6m~CB)?0L=x?6SSWnJH{R4~xGh{&+YZDTSYiVt-mOi6T zJ-AJ$I&t%LrCZz{U$tyBQgWRO;!x8Q&~w3HbFBa1OnWu#9*y13K&ypWGj_rwgk^6TXHV65BRH3^}m-caY`oZhXQ)z#Fem-WQJxA6xqabc`H6j0t?^wOyRfSObu~48pLMfGEuTEw?%;; zZn7FC!RYCfmop$cJG;4x1aBXd)h*&+93z&**SAH>kn|=d7Z^wWok2f>!}vEENGE3~#$3DJrg_?Y4jU&+`Ej~6{lxg8 zipDwBU1fKAcxQ#>{F#~jh`(SsTH~?pO#2?<;{Etog)a}$$NQ%*WN`TO^=@zLm<4_5 z1~&Fk>GnDX_a6l-M+t+j=P7t6v#EqHpUD`v1f*c9&N#^} z`gOl@%QyVKtJDbga7px$z`6t3?f%F9jghsABFIbfd5YWj$4|VCW_Tl-a;Et8tBXt` z)u}n#WlSvi6f0r@SutqU_p29EBs}LeR;aHmeTXuZ)Oq5(oLR$I7;xh9w^E8Qi_*t7 z8?I&*dog&E;(%W70!p+k_MooU=48p(rrEuk-Ap2QyD)r{ElV4WitAVNbW)K@O+y>s zk@8cNo#1BIC+5xMyf{W7e|)v9QGFzv2aRNRAQ`gZcMsxCa!mG4gxPB1aS zv%l{1Kc_cTM4&IPChm6l<#9r(BGOztO8+7sO6S@UfS z=-E5=c=Xuc|1jfT#E)=(2LuQ{ocB8Rx@`MmY^ILGu9}{6^v!%HlH*7xi(?>1Vs%Ed zm&*EJMM960JsQ|}>TP|!nQPmTC6hapa4pp&;(G_I*L-FcB~@ncQ&@7=i`aKI?rwkb z1|xRo$x7P?@bI|igZq0=*5o1`l&~Wp7-xYNK-DKn8`}%qv(Z@-k z>so^wJ~kl;d3=!#$$LTbRQX%FG(dreNKGyrU6EY&Mm$2QuFiC1tKKX93IF`Lpq@8j z&f0~)Nb*lY&k@PGOeh~WMb@WK;&xvL50>$UcDm%*bE^&O)N4JZFUj`;0lm!Y*-#Yj zfRGH`yLu{AY^JI-`*H4)D0oqMzL0OrU6?X?eWo!I)SM?69Wg;bGa1Ie62yQ#k)=7ZHgxPW2j-erkQh9gS8eZY%Ks9BjpcZ>#7i}F@;cD$OeTe(#36}&Huo^(fY zZ5tvYCRR^EQ_0}cU*BDVOMKB{p-kcA;J)4FY}HIJk2w*b)U)*V@DMQDlnTSOK}g?u zY}e;ds#xb0ZrR$u*O%Nk^B0z2?`g=56Lld*{ia4t0r-Pt&~hMb=jhl>@`|+*_tklZ zSX;`|*^aTA5^eE{xWAJ?!b@##cp5u57q)t!>+`|aP=h!`<_p`epq^S(Fw2uWy?BRB zIXWgxxR{@2qREpVTB}oVu!$n_kx3ayvb{%NX*~@2ZGB518~0VtYzM&c(xK`G+Dh`f zuWxP$0EVUj^?Ze>-`q}W|I@D0>5?`z=wT1IcT}W>W;OZ!mIvLspTBF-<>H}sJ11o_9aBhi;8kqYD{}X6GM0&?o-B)J-Qdg!gi*F4wF6~ z;xk_V&QG$~&%{Ks$FH{veDwCBw(5AgQfbCY^gU25n`yMizq)~*7D7aYdslL`R;e|_ z+%4mUsLezW>@A-PSsEHo7W?>wz&xP`SL6RuP z%AGGYsQP}#f|7!`09OCXl+=*Uc?WP2f?mI1#&fR=W(grRI^e|s3rVR+I8eJOotNLwJEV*|q^Qv9o^#SDC7@C#!;VQn6i7ax; z+25J#Uw-!arh;=H|6$ql#s)ER0w;I-GQ-#vWq#1z(Q5G{4TM`#3eNKmF$b+_8bI+{ z_>!)uQJNzwM4~SeSy34Is8#z0^1{P;^U#o9#+Q^f( z@mY%ECKi8wFs*_*9F;*eU*3*!`DHGTF=p{Pvh9exxQnsZ z`Okq?V84g06~mM4W>hW4Mo74E~1jOutmXl%!0TV0A zO?LBk@LAZO!nbQo07#AF2NcWxYIs+-8c~aSm?Ki&D3OGXFnp-ZWvOzAwC}K)uVZD^ z#>FQTKEELr@EihHn)s6y8G91h?P6cC=bXs?6oM#G&T@GGZV7aUQ>&x66J%GUmA}(VT5~kvkYK?JRIdUP zw*97R2UdYXtgiuZbw>E3zhC}oPOqG^bTWBmkIjqszFEKGmf3mF_3gi;PqV^YWT?T% zu^*iaQDfHK$-r*@@89nED|!UY#O>{wCo;LC-@XlpyEJ1Qe0;S7)~*4x?-c(e3eX7i z(XOFVr-CO!;Ppk7j*^mX12yFwp5V7~vNmw)#WyvbvzU{=kg`$yjsSv~2V`B*E=dtH zK5?mn+?sqyqAagnMcHE+nSRZcfgIwZ#~}MtU`5Qc}N9%PnQTw*Ka!6 zbU;O&pf|cgZ1^^AeF$r4LTX$bQ-Pcm0pPo4mT5WnkjN9#( zXI2#a*bFRCzna5uobPgbV?377-&gR43wRCq{mmiw0xQP$0IX3-|DUbnA3s{M_-Keg;yZIJ*+^hhM zO79_5`FJJpe11o&xXsSQu2k#0d%e{zmm*Wz)^`(;6W#h3?7Y-&B^d8;A3bI+dOu0O zv36Ur6ktht!@nrN9X9U8)?U1iiH{zyi4CU3q!QTGj~+I68jc5Yu{DGe9LM_`xy%(} z9|8jZ9BWG-cn2EazP`IKiO=CU+BcnR{6h4;HR#VH&?DPsaZ5ntYAWyWH@E70bY%$! zT%0^;C)BygyOu$uL)>a8@7;*YHRN!P@8s@HT<*3A}u?BvzB{e zgm#k1JlVn^Qu=kgZR>J`3e8&RV|to{WuB(!_sITn)raQMeSjr;zeqcafK*WUH8+ez z&pY%;@{$#0zBF*1{!WMha=tv?Y~Hkh@+YgJDv@fDw9^0zl03VxEq1Jma7VekHFCHYyIbG` z9M;r$#LyX#H$Gxi@>JH^1t9<6$j=vujf>N?)=E<-XG{% z8k;uR1mkTubxw%(7(s3~9(4hC+hxgH$h}>25NwgFcOqxMitK!k=SD3Jk>xMj4|Eu- zmNn|Iv9PAQd>fCN7Z+xrfwu=gYn(T_<~yuU3tFrFNEXZIAn81pf3l#(i>9WEim5DY zs{pp!ww}0Mv!fU{A`+5Pk(+erg7LV_KaX#wmtg8<^IfxnV>@ei=Uhu})K{c+o9aeE z!j_Z6k_=71zb%{?jTj4A7J0l~6atekrbIsl6UAA6d0LAk1yQFs{LN>_%@hH#gUz73F{3L_pj^0?09LrXBk; z4u{q$k2JV`F{j0m!MfPH(w6DOs5`FOtuE>CXH5nQ zxcjKfnY$yP_Z?De%l{ojN;1naJTx5foZU z!Nv0SYQQCgmw(RxHP409Ge}l*=TNgNyw#?mt(MnsYT^ta;Ffp^?mBh6^dj2Z?=TQoj))$8w*vK_pQs!t{bvTJ|&F*?1uAf+f zZcvOm(Q(E?sZ-EiE<-sb4z;L4=JH&a=f2JYs{X2x>odyVSAGxet%o|X;yY%m8y=`t zK_NeA-b7Gok@Fpztq02?wzf7;^{tm_Zf_^x%=`VYa&TRA~O7N4igSUMH?Voa6;jpo#j~y_^4}4m}nK ziJcvCPnG9Be#LcZ$S&u}ayPpOp3VdRUNi)&&dD*d#Cn*e&(tOcR#V7EXLLb|)1q;G zH+p{3tX8f^tv3S};^%AT;K2*VpqH>t))!dFW{yGGyBtt6((eL@Vdz_#bs;O!*~HZF za&?Sml_iu{RM$aFQ)*CZiE^>7>E_%9tc}>38XRhe@x$CxmCU7Om7DStEgQ1-ImVe) z-}|J`c?|oU`}s>9x}_K6P6DND^?xS0{xkww1`k#knG3k!)!1T}Fv6jXjc5c>&j}n( z6+>=MUD9QiSdoTRbbR++%PI}_$uH)8_)?!JKsYw&qj%_vr&m|AVWwWQTXNv&G~I58 zCw7?@hd&&-C5_KC*B3>UgfaaD^xbK-CUNyD^l0B(U>7K8B^}y)Y2DqM8%}vit&{s( z`(h%%4telFi79eGmTAxek_B+hzjnj-XxR6_Ij!IYunNyEAl#3wGg~|Ty@%?G#QU$$ ziUA~W1qlp3k`WFUEp-3Bzo<+Li3Ce>2UD*%J80|Hfz6A$q>d|}?(y^M7EvvaUU9@U z(J&G&uer@Xh`i&ys9G2qM2et#7cmD!GLGh8%(#-a0ehNBI<~IX<-R%r#+O2n&2lcy zHY2Yl#c3{x$?soENws9!Av_2V5*!$1yZgPAldF`!%hGy?N$M88R0t!wtNnbhnh@BZ zzxWoX(b^7&r-kL3I?;2%J6-Z$e4p3Yqs0=y%EUdo#_=O19_AVY>$hvtLqZGW_4Y9?a z-I+(y?^a@UWcUas>A5BQWz|!-uB+SV&#}?W3tKG=+qAQsxVKDDDMhL82HY8kZi+HRuvQ zH`TOSx20YhRzd9x63aoiWom#Ww3S(kuY1dp%O}e|Yq?cpjX`_?Ltnjwk0k9&J+WU# zmabY{I6&`kQfK?ER&os~W>@Q+oVZIZOPgP2FY)H+FjyUp9q4C}F7o|G>(z(N1mCod zx+?n?O0)s42M*G+;*=CtjQ%a-L&>I|$-^GblQb?+&|g9Yra^WL&nBpVAtE5|Bqj1f2juhr>+l}+FcrlUv>)dMvn<3*^EAy%EZ4^T0cDx`TQ#krG znG{I6L{P{X$0OUqfy$3rp{3D$(q}~E>!`1)gh^_N=in{pIE!jubpQ~BotX%&@OU=p zs>)#}CB)81;7xWbNG5Xh^jUsU`Tm z-IMPA(jTsRxYVtG#Qrm1o0lC@%kGoMHgjYaGVBg8Lu4&;^Nk8a1uxHFgEd`dIu3n` zb}-PAd)>_4d>J8&Y(BKh3E&Lv&bta|Rw;d26!KJuS@2T^>~NFB z$S{qJvDNe~@-9L>Skg>N2lL#4<|B$0f8RsEBN*O zD{bw-o}!&EGc_gMm(6t&zj5=P``nirB+cQx@~t1xkUa1>O}89}NbXrKd^}5(sr&29 zNQ%ZVZc3g1?5I~<$5tkKZ(H9OC(~s8Nk*+ykI00&t84&De_X7=!svksVQ4kabb#CK zch{AloNkX)89K|wyS4uP&a{-R$B2rO+bM4-NvYJW zgRMUti5XgnzYsV_|8b5L80EoiBf1f-*xEC{S%y=`ZI5=8$IiXALYuDg;Ct5}#){l} z?y7G4_BLYhed?`}t4!EpJZi443p_FGYld+`)X)EQc_X*`gq7!`Mn4W(DULeN>`gtP z2HW=K=gtIk=Fg0mDU7vW>x@My^fZ7Am2k180cu8!5@Zd+5?4VX3i zf?F$q&Ti2K4RI}|U<}Wfi87h2 z?EaG%{09-HN{)p$)^dCZQ8QgQz7RfzS3=WpKPz%$^fue zU`ED~jnLVNmAmHmH_!DgzPcwywbNy}LVVx-zbP^g4Mwss$A&3?w14@H|M?Or4Iifc zfE?o9fliY1(l*9z^SHWX+Ql#rqgcV;{^Xm25|C)w}-Y@HICv_p%#T$b%13;MKQx8Po!h)fa<2`}YjkzVm^NEbn_3c&-`1_o zlEz+7Md2F2WIDE73d>QhH=uZyqq^v~6W^~tKlJ*p^OPtKyehyO3HVry?@xG4)Xn{1=u?U?k?ob8=j zu0^y`w{~Wp$@WIg_!ro|rpuX16m*OH3~!7ohPzH-xDpBsm=gAe8>LWm*ad19S;qc}v1oOWpO%p6XwhezSHh2#)W~3xHDjeosX)@^eWIf(mE_ zg5C2&4N?{WOfz^Pt$WGOD>YeD#O}^i>wCXY=FcVKU$P+P+n;X$I!~O=USNk^Cyc!) z$Hy%{0v%6YEKY|BJ7Mdc;q!tpf+kp}S;|XQAity!vKjfUYVWyBHSOlBd$#=t<@nf& zryara=lHnF3w9n+hGidovAfCuG2y$hww3tDpGFXd7f`2_`1(sbMf?HU)~Bi)!LgU1 zaj%M}O8|#|#b$6MrMn8IL45CNlKIu?^J@7j;DzAtrM@WWzy19#$ZTPCNfXY)EU@`8 z`eAT4<+|5U0nP%isHq7Y#>B>sSF^T8o@YjQ4&gET%0JH>(A`;Xeet1E7d{D8QlUw& z+5OWIf|V(1&NR<#*y>P5+^JjyDAC#n4kj@lWx8iHFvC5aF@$^ZuVC`}NB{~0Raq`6stF_CcZ=oIdIwr;FsdOlY-ZaC(+ zrx|)JFZ7S`*@-Ov{BvzIp1J7GYHtG8S?J2O*S`eRT24UPQ+eyGnGENM)hOVpX?3%( zC%%S>A}Lt!9|-kREhZOw zkT1vjUnoBONK2s9CXuIosDbl~1?!JPf4x;ymB8r>mf;EfOA0ue>6R}i+xaH-Z+9UU z#NoVFDQC3cBZlv>$~TK|RE5tSj30*yLlwSFW@=aIR9VZFlfiA)e>7xwYP4E4#GO7t zoJ3(&_`1|a`nsZT-EH_$7eACN`27`Q85uoTDGc}Dj{WcIUn5PHsRUBNrFw8;e**0V zj9ml)A*9J^J&UFoRWesRbf}S5XDUVNpWXfU1__*t{`$cWQt#;geVo^?9iJcg;gh*f z!IXz7%%*hks1+US|NUiOvp>OqwxN>K{SQW_7!V8fTY{Kk8XE!nA=>|*G59ZowEe z!N22GIg%(JexY~YyCi}oH1NWxOFp7V9ubj|6>20XWbg9!Nr?pM=gFFM=dPq z*x5Q3+_dl=ADNg2g@1l={Ww}rb;6GotZGCI3(0luUTmb@# zYsYtVcG%1mqrQtLWK+`#(PAo*o`wa#_GH7+SEQ3BRB(n1y8IkaMk!v$?OUklT|OZGCD~@+RC&YVm!D1 z{{!mdIGMisw|+vi6S1l?-{^paTA;<}18sEjUd+HYS@K&ZFAuMBkW}gj8ZN-XBe_)R zsABBXiAU(lf3TxOf{v0m?LmWQcXt9OOVd@=X=Qsh>zL=nU)iNM)~zSY0mw*sS9H`rbk`TUZ^DsXJ?-P>pPv`o`r_2fbdkuAUV)i}+d1mTZiLI&CyNPLV z<=FBr>i7}+o_7xzsQ3Tr5dC}dVJPDBvbEx#jOzDU!5~RZHK=Oco~aGbN2S9;$))zj zGz}{S)WqvQPPBwBD%@5|nmd;_Cqfk(;;+aLDf4YL%jP`VX`GYjZ>P3?ebJdFSlFwH zgB}_n`9`-6RJ69TZ%UyDhKD}=AJ)E9@`_7>A$Rnog?)UZ!4)qQP_VK2a`BA!EgNo3 z8BN3AMzBtE77QH7HUO|_38EPxORf+hUIDd?s6SK-ePUc6N2)v%RP}7IEz?7>@v-`x zJ-~aPwIIK8P@W`d!S`w-+#lYB!(rd>qnu_C3-uBU%uP{Mo1swpMsA-dg4k2qY1mrS zlKa5eyr59dY4{+0cIwxsG>>q68EqhZtQ0MZw(M%+x12>RZG=4!2S+-qeCh``&;rt6 z$i&;Qp|FY0hld2ZHkGlcs6`MPQg}x>6P8OyY9x%^gQ=~6M$ z0l>>h+5jo141OA2uJ9{I@Yll=v4=+p8D_6=KKR+X!w|B;EzTo5u9p60~La!;cUt&h7{%W{EYCvV^Yyv_D zXZ3-yjjs#8HcmN4MVEKR#l`4@fkG+xda12*iS_ZfpWSqu(BlC3bFQXv-H8#1P!<%v zT;l!dPXVaRhZbC*D3rm6 zcDMhG-c-!oy^VB+ba@BQ&6fqi{N4zB^lSJC1ilqG|8a9o0qg|T9+%{%f>KJ9(_LJf zqDLaW5qtJQp|giZZ!(K;p{=!8nux+cWXN}B0ZklAyTp-bLO-2tObKTxLc z;Q%P+h!`O5zf*@=RXPX$ltA#Def6-!7}fi6sKDaz#sI1B zvLS6vUaafnGo7?tgXbdA`qQ1&!^26P0gzvvZuFKI4^8fWPd|9K_V70S^Mnx0bi+3UV*H1uG{~42WUG# zrBY{1PyK^x_0O3Cyp~NY^J3IKv>q2mm(iv&TaLWcoPUeVRv-q}2kjF3aSN21B6$J1DA_ zZEgF>=lxyZ4N*m#i()ndKI?DSlbPZnMPc2P+a0^p+RiiP3*Zi!!_ZyqB6qB;&+w&7 zl9Fiv2qz)O&4+u2|2azxC`zJ{n+VQMCD9Z<-<*W*8SqH(*E_ryUA_l`Xg(v*?%);T z{Uzc#a;{6i@(`M~eazx15Lxdqv{BgtwX6_idM@ffQ3@oSiy z+ahW*9<)&w7-D{pK1JGcGb?f{3aM>vZe?C16NBw`{6v{*KEvQehtH}1uh-=!8=6bC znBx-p)4%6AmOq8AX-e<#^ut!q8-g|vP?UPd{XG#G-NX9jE>C8{C>m0h-D91xIcf;D zGYOM|Kh_)Ehp5(Yn_cl;Fe}G2-ZH(Nq)_PI3m7l?rRh`WrbTG`Swl|k9R`(fe@#?^ z&<;X}+tbU*dO!l{eh&Rvo|P=g#G{QB#tjBK9|rZ!P7taWo;=6acm~}vD`|OW|8f{v zv6DSqg-P3Wo^CENA%Vv5ec4Y#>qRF=B!Z-V*zT)GKJ*6f{^)K*KE7?bo0Wv6hEdMI z-kUD4{N+buVBP|4GvI20q@pCXhf*P?I8fcYiq|~gdKS8!@mX40x_NN-o_bfcl1FjX zV9$`hTJ7MEcc7=%M!Nt@axk*#uLaluN*>VNOJ(SiQuF(F2Vokwz5e!?5K#@sBd96U zJ_l*+s9y-KFVKg{x-(WXpXMR50d^Z+`Iv^;w{sx2ShM1+XO`b<^!Kv*2x~+$532-m zCE2juM5;ORH4iQ>?mR({Dm_z2UBHgr{qb0prH`Gg)5}p{_>?x4@vo>?51Xo&X*TTz z2Yi*&H3&>xp|#5VoBl;@nA_Y;$zm=AlEfkUpRxD&XhUHU%~h8D zKTW8eXP$Fwwi{g)!#@rt^(C3l%&mQ#-&!lLm6tPh!jjb+kE0aZw?it!%9FD$v@P>S zJT4i8Ch-Bgk1=!NhkOQ4vpFBz8aCqlDo%uD%Zl6%i$3cId~W2K(;jYz&9pNvUz+LZ zcq8rgSpNAHQvPiSDIg!{W~Xr-5zad#oXPt|w&e0pd~+#=Q91GlVIBd;>Q2Kekfa8UEJT$~TXRH?;KG-GX;)GOpq$m$Qa;VIxJ*BR-{s$`= z>vXe*%|KJJv%1SQr&_;Lo#5-LH#8~sR*WS;*I)R++`ud&SaaCob{0st`Cux>{uw3b z=$!IlVfN?CAi3M!MA{5n>k)eod&mK@#S1t$6&ci{oSmMNlLXjZ_cyL);}riepKJQG zqpaKCXfBo5wC(OLm18uItM>Xz(s<)fnt6yOml z@~5+{Asc<7s^SKc=-w*BIgXT5&5WtQI( z2nl}y9#aE_OLj=!PucVq)KQ9g$%oAWvN+gl%{#9gSyx-ER;-!r593~uY+p)N^-8qL zaf&g>i}F{8PX2|OMvc@0vvd|gZA?_Ap#I1CCa3EZ+Pc9rsIKK$6RB@eFY zTwaPtI$EuOy~#jZbb!zp;Hkq)yC!BzUn=P7UYC6&yeHrP5_?tKvk@!LQ!f^RQh=*r zPQp}S3bTPV$a(TqmN4?)1bJcq&q|_U64VmTTk|OMKbR=`xyxU+Hy?a?e9_Fmk3R&73M!VV)RFZW(=-&p*{yM@R6$)yR5~xDh>IOS;zgZWNO@Fkmw7WyzY*VCo zSipLZ0)1L_u2iswG<|KF5YO*^87$Kdih*<}keHpFl5eIxi#>ksYK~wpF(B>bF0Jdn z418Y*yk&n0Sy>3vawAma*+A&1o%A<3t}Nzvs(3oH-j=?@=-f#E(IT7p;+54Zb?CkE zi3#2r^S~#!Y9h$^7@wQx{SwQZCVwk$w_gLAtnR_Q58yHrVg-tF&?Pk<0TW{a1!&9> zq@q>@hE-!i4<-ohs?(eUx9!fkM9!!Tobw-uwSTYV21hm_A8R2sVdbe4$6~+cWIuhp zcND2SUZxPOJHjgWtZli9E&aO&z{LIEeV8=NjR%$!wp@CB6%6}l$HB-57yKYmHmC_% z>WklTQ%AX)rkitNuAH_vjbrqS5)Y-C?r*wzi(OE<3DY0^M}+&Gc)ZzoB1O>_6O&Ei zMu$XptBso|7%?|+h&gnEQPll&w`;@P@Z74+`X^KW4C#?9y~#4~H6IME_6I!X7L(|~ zR%M4}`N1(!k+zHNrvNbJ~98*JX;kLZ;=)1aYF(S175-;5-$Fyvl?) zT}g98{*9Lj=|Q)}LX-KrPwTrGudh3j9VO!rG>!mQkpssut(QH3-FJ2Id9b0zqbkkb z96@A_$@{zZYEw}>v9jvAwmZm(aGs_qHs?41uW<-W2sv^Hh%Hv=nY@DZKLtr?EM)tL z7^W)vIy@V+Ius3P99-*DNNvVrcOE;=Zv!6{q+w4l*^0h@^?qNVMM9%Z%yv8}qhAc7 z^i7=4&Do*?6da5%cx{%Q8XQb7Za;Tm`|hk-8Bbn30PP%)`nk2lpuUd|E4GV9%Q6?e8$Bk&vU?H%li2rVoCEf|MgvGs7sZn~nFvK~J)j=T~ zBekBnXPeVryrhUBk1l3r6*6m<#v9JZ&q9DrHStAM6sUyPTddb7n|H0J*TPhK>14>T z6TN&eiPg(pnf=;~tn*0 zO<_kx32OuPl8Kghk)5G))#-sSu>B_aSG(48Y0Xvpv7*1qHh14aIa+~v!yYmpC#0R_ zD{6se{7+&g`pK<3L75u!4P=I%y$E4ONF?$SAmEQjUh&><^^8TP0oX(AgdeNHY4pn+ zWxrz!7b2MubnElXLX#;O*tEOMv_ZL_vcCigI8-!61sP zsL__qm~mCn^lW}ikzCKmm@-lFAM7uy36=hDwN!RRIS$TCn$CaahqvX&Mv&CkXP9}g zIdz4P4E&FW5or{XN^08kHmzi5?rbX3BV2HxYDjjtq-LEo;8Rh#kB)?gmT?~34}j}} z-j{9enJi$0d6z?0d;CCX$u$A-xN|FhjI?KiUG~r}u-=U=FZuhaqkLAmilcnu_sf8( zU)7hx$U+|DTdgp)r`Mpl#lMJgaN+D086ue%1 z-1!W0+HE-f5%e_Q;7VLyjqND>woiQ-7rTm)TSu(3Oc?YW_0Oe>%&z=xRKs-7!m5mi zupZ^^z1FC&Ekzr;(48{<17Q*OqOgx>)+62qaXvk~7iD~6Z6UAfM1A$Du}r%&LFu;_ z#b`9XN}BSsmH18FdD1fEenXX?B)CM?*rk)?Ew%K^iiY4_Cp9rqTyc(<)dw0%Ww`@P zCOwQKP2BtE`^Jl28-bIn>5*zOA#OGkUcMv;=E9J@GST-6q2Tg_z=yrizc7yK(wA{Z zk>hWWhd>NT6bJ$GJWq8ISR)*^u~e<*R1%1(j>nw;@5^)z?7k+u?7!GKFZ0(!rsi2T z^xng!u01V_(IK`O4puk_smqjb+ML<96MSQG6$I;jStUClauhQ_*imk>c=5<&E0j|8|iV| zG+K_E+VW^#i5UTWJ%4LM8M1ZXcUT`!x;G*mOQsgr~Fi_mo#1pLx?^ylp=Xd1io{HA< zW^?&yod8iPq#y72;#4;mnH}r1ZmTzF=JJx6=M zDI~(0N6CT$<8y%srKJaZXOD0SA#>V?Qz_D&Gb1Iog~8%kltRPU0Y*dZs|@VPkx_Gc z{9f)mk~g3Ft$m=?-i9uE1u1X8?MsDlBeBMt%y@TuB_d!GL~b|w-JR3IW5>iYjxP-C zfxZcL$UV%_e@7f{ziceda?xv<ab_8d1_8_Y@;eqV@+RCV zQd-)2%ITQ2OrF;phuqZ(L0k9R&=LdBodHJ1k_P$HgxeI;zywct?Z%%dZgi?z_2VSQ zXyP*{wISH-$o3?8qgHY6yG*y+eo5LWZ-8P+L&fUDeDq&PtdCfbtX>S76=n#^>79rC z0_)^@CD>siQSdC^(>OU#0~F2>^)JBo#X%6Lw8S78`yP>J7ep*duj)navE$|m&gMbB z^KLWBL{M+1;mH~~^avB2FU@l9Q6D?GYImLgPMdWjp5xzB_t}5e{bzZNCSXOVq^Mnc zB{P$b2n8;7^A}49zourBHNcLlS7#nXESC3T)Jl@hVM=6kX6gj_T)j)mynP(N9$<+ zMeEo*Laa1}1_PV0)3d=x^j-S5G;@obL1aU(V1M76`JQMcDNUXcyJ{wh*^xa)0w{h* zJDVW!Jc=)-YTVX2qVw%(BgN{FDLx7o*LVi*pK3b>-gn_9^N}1>fjae%*;U4j+Uf&` zbxwMZBOCcl*9+P9O@<}cF5B(rEWg^Rgbl)b^JdcZD6@$(c_fd9km~sH{{;;HcEU)A zp07-E^z@r~F4&DXrKAe?;oKr5MonQ)%tVB3ksV9Xujm%^*7T?&%X&XEm$7~SLf!N4 zIoaWN4{`x+pIT+V+tW0SG5)>5pwJ@&SO13}`Tpk@R_{gabGq-ZeI`rz-z6JNovIFb zIbx=eZ>P|FFu}89)4#*`e^!1Z_U~TZm88B zwqPk8Nwk8JwgP>s5=+`cLvT-xlqSSnKJ)(X_<-B*q;ohc{mz?0&9|IjxBR8W*X3a!7q?o@c8!<XRh^nt?>lYh1Al0pmT4zMc7FVR839^F z1z_kWiW2mJ?RZ8j7@TCQRG3O7|Bn+!wryx+fD<3_LHMOYXMR{p)#w!h{^Z0&g<=j` z;%}4bk@!EzwG}C24bt)sCj5)^E#fOr85kY`6%s8a9Rj#Y^ns1lti`v((ous~I z>IXOm_(N2Ur)$dSo$1q5MPw@;zldR7A~-(ZRrI#%0i3d)bI$R5FV~E;kPfY`8#f3{ z3%_Smdz|QZ`6MRKTx{sL9uJJ7U;KIR7`3X5DY~ms*7s-B=hcbkbH1U)!;{a~9^&2p z>hp~DE=uc(6zYO8Z+%rmnn+P@;V^nJOF*|Po6+p*Dd4O+<$+iFGsb_;%7_S7WmaT) zSM~drau0&M4=ciV{?k6Dhl1{z^1?wa;k`$a2JSfkVtX4q?6}KiQNKHQ2xFEPH+=O- zibvfVAPVVPu*`8%Ew!w9%!P1P!<}86*eS|kQb)C=myB1>_RzK>kN4U3sbLosk!qwQ zc`EJXxv|7)pNR8RFw>rG`Z-Aw59g|5RKR2RXG$CIrveJ{8yx2iUmL^wn-{Cksm;X+ zuq1fcN_ecZtzki%(_H`UK?EDTe;KGn4hHHA48L@H9f_=Y@N_wroOvaGL+zi5cjX`Z zjjvylA7^zjR?B4OcMS|J4#y7X?q#Z3d8!MzD)W+FUp+l5wqhg?|a{PTvzpkfSFQDlDz9AJ%uzF<-SvxfK+f+^)y-8Eu7XkVRzA*h6Qe54; zh32JfS3IQ!D`4i~E=~Uat&rb9l<0W0kLD24E1Qvxq3G_O zDR}M|p8fs*iu&q+D8BD)8YCAG=~P5OQt4P?1qCE@DM4yMkd$ta1}RaxLu%=k?hu3p z>28*kZdl#{Ki}WG|2d4eX6D>;p65B|`bw+2x^C~~5||WW&Yyc6@23mI#;Xi#78P-o z&)URH;mh*8wV5m)IsfxYh^Q3{XHW6(_?7D$%$1Q={I@#J!N%s@QLafw)a^?S(?%)d zgpiv67udHr}t+msh`PQr9^_*CQEJU7kdQFVw}{~e3siPDk^bVt1~V{QFmJ!u=A6e z#8}=_qa;a#!8xc$5x>lhElP~v9(Wy!a^&s!{JrYt_Z}O~E|zbOK%$xD&R#P&K#8-tg6mPOoH zDQ3=PwxdRq*C&$=Yx3AfC>V0}dmm3!j$0a2&`CF9XmV2fe7)VC-{r&?E1cx*;~%5K zT*=r;v^H8#JyWFfAw^bPc7L~B6A}|+yYNeVQ&Usfy~pjYtnIRiaJ*;9zsmJen%e@`egzCAf-ZqL(guFwTs?u=-10BFq*leGczREyaLyM3_-SwDN>4{c#aV|Hh zE?XR5$uzR87!h}%^=qe4MFeMTXK>5L?4^X0n@*wd9#d-tqUnkCSr=uAMq~QnF#;DH z?d^V(nHn%m7Kvriedv+tQGzL;dxrU=YQYTq`qLY-()&OI^`vf@MeL%Db?0Y@UnDFj%x2hmP2=H5a_|6-ML8H&CYh!#s!urU2sIP~%3 z$DMt96t8#)NLDnl*RW(Bd5l&S6^#y!>k^278@;YLSFd=EL=HKgS-x&8@*XH@JJ$Ak zJT0e%Y5CqWSrnwdc5?s2Jgn*myj)pmA+7jYrK7^);Zzg8q=3 zfBUOi0a-?ZeU-2oJI-`E*e!1-ZUGc4A``$^52eJ(E)e^1F`!9yRgaGD3gb;jd9C_> zrWr+0LOCO}Kib$7#5cR*{VC(I9}=kwuWbVqttcCF9z(R(5XcJuZnITkicnKtUVMD~ z*Rin&wLco=H!YZqGY?vl>>Ah`Tx?Yn6BEOPTx#d!Z-4fZp}D#&@8NT5)j%fkSUhn4s}g2AK+#DV=q zvidW&46~d;;WU#6<;hJcy9WVSjh%!+1$<^7ob)`bPfK2KM!-zFR~w(5SHY-~;$yu% zHfq?JD@&Nyg+26|eyBH{2NZNFs=g0vCWY;jgqnW%ke#erMMsFkIx?y@*i2ihluKF& zOXGDh6ghphOCWhh^t5zb&|RuN=+1doHhC_cZXVPdlltO4;c1WhCXt?L&}RR{%%5mJ z|D|(nkps_x;NUM--4Uj%z_CuGHR%-H5haD+F1@d+vp{aU&6#8uu+3_og~DT;HdFDO z?+ha`GC|d&k3fv^rSE=T)4)yW{J#Pbd(UIKh`dewKzAr{UY^ySP#)>#@Z!0)tDS_Y zy5FRzTK_Qf^LKVDTT7~1C-lP1O$Xhu(jpHE1SaR%hH?Aj@L5c|* za+{Y*KlJ#!MTk$TPJ-gRMIX%ZIYA-8=J&m?-^NJaOxbM_V6X1G$Mzv=Jmdt6-|4E> z6TaW~*o#8^T6X3!JyR;q_a9d+GQb9+l_*6yKKzq2|GK%EKTqlDtz>}D0(;0zDTn1P zSxTrvDSO_bflrt9L1?}-Z+CdJ z;jF~_rBnCqokyJM`Mpr{n<;@lnN1m%w{M5Z_#_)bshY)*YimE~0K|djNlk4n`<5{F zd0zFkA&J@D!m;e`FVObv8N1ARlFKpmqt(?B5)_f4mNsPq-rz_tPd0TifwzSOim?R$TtcBCfH;$7vyQ>|Zc-T9a9%p-M zuvE>JDEX$;aTqKGm$)HSi!B27_U(7eiBGwOy0y9wjCSYZO!^aPb%`D@x?2`l&RwM+ zo$}aM>OT6`RA&a@MZVIeJ~qJe{*FYhMTl9e%mg7%1~Y0#QMDgW|5Ao`dn%?RxlNCi zG|#$NYvxa%*!Nt4n4X0n6Z>n=Y&AHierTtV_{jl5jv*EjU{Y6*+Uvp%Zsp-*-J&!1S0b&rmXKg5u zHRngt4}hmXu?@M})ciB!xb>jb=^E{a;pT?-n-+8uCIme~S?33`k1fldK3!A=V;w8dD1cxj{~n`p$B(SY?RzbV7s@VW4F@7H(9=#(b2^8a;^f2eT*I;2`+AQ3b34V(CoE~$Gwc&5b%fj z+BbjQqsHjM)y(VYeN-|JroaQz{;XQL+UQEIxunh-21_jBzEj>@!_;5K|K1Z3%Z*7A zzA-hm^9)w=9OQ4?{bL~?RMpS*TKk3}8q;_A@87R(D&K6(bgV96-Tzra7CUsAM^Jqw znpRweZy+h?6nH7`Ow~37|E+(eXyvf_k2(|Vzctm>QfSrQH?(kk@;;cTjNtFcRZ!~A z%y|R8nxk2K@&mODzA+0u{p>OOy@lybQ4E;)gR%S?%wfaTUW$-oLt2niiJqryZ_|wX z?)$DQQ}Rn6{1M0C^->R~89(>(} zT6FE;!@WGmJ+U8bx|xh%91A@*r2n88o>PhNIW7ZM+qKu>(3QBeV0GAA%21;zh>Wp# z0y)f>ss(moe(Ub8lZpHMY}4b<hvv+bvMUowm^67C&0_U30K$>5B+dl2va z>wV7i`E%fPWD*h=|5gzLY75 zV>2^h(j(?Q8Q$DQ!8vjkRSRok__-s`yIJdW0AgkFkSz_8+9~h*&7t8Rf~Q6L$A|dK z>9(s{fQKl^YbnGOx`j?pwmW_L4Idf$aqGQ1HWQ4hfeyv^9kx57CLd)nz zl{6fdT_epH`uDH0L;6$&BE4e2$MsO>Y=|FeGBnfCV`S(pA8;xK}vUjYDH2Q zoGW#g|Jfkne2%rR@xeVNr^6n9?2zPBgn^`|hpI^!33{bNw~99vn(&w~cwg7Z?Gc%> z>)iHUoiujk^;KJWEywh+q*j`~J>%~~G)4wqk5nGeULR}RN#0?0W7g{jJWUWHj2DpA z092K7ItC5nqil;88by!U+BQQ|tvz6v%ul*JH}+a^9Wqpd@YgD;SmD*zue$HPar|3l zL?)T^=RnEI>uXK;G~Oy`E#upSLXdOa0TnuxFT+fbg390`kFOBa>|NY!=^{Z0=)o{3 zeX0hKw(TnW14UX%eRbUVAC3$c;i9SkR*|fIYjvZp$@O-ts=9cPqAS5)Cgl*PQ3vn+ zdkaq`_iXbzn2#!H>+=(Wnjtx?%PhZ_SjAeo$0E{O1QqJte&%3XLIzi3MWgw$r|>Ch zO+1^)a*?tI$=D6JCFi}9*P3OR13Z?_!XCL(*8Y_)#Ev=y&vd63F#W7>_9S?BzD(~=5<>Ci;2G5ZaAm@?FpuIW=^~NB~w5=?&z-sH+*By+}tyo z%aTYRYg|KFpU z#IFNU$QVTpO+ZcRY!7Q4H}~g#_;Ex1B32=p&eS zf7416HVI2ex9BGWFP=~Cc-vSP^3#4Fo+**6T3-5Ua0_d`iQ7@KB*1V!de9xj_MSWN z0R$2t?0(`QO_6NY>9dK<$RJf`4J&(upf~T_H9A11`d zt8!7*75r?(qd>^$$*vIo!!2}uKN0$ri3}!9mSdpjFq{##rZGT35{aablKhZG3QFBp zO7aC*NAS_n?hlBi(zBvP8yk2bXIF+}}x{=Frk|KQ$J3cH9=1i<4Oo#hGwI#9B!%~QJl z7G@EuYWa)C4r>bJf^+H51>8?963@&Z<1GBb!Hkz;mwzoURfwxoF}&zdz40~wl$S$- zPepREm>B*{eQR?gp}MoFUyH<749jKx{d?jQuaShcvR##GBt#|1vYTVvwpX*-zH|i1 zHU(!&N8Bevir(=5a=c3ZgRtituLYfp^PFFy@{RRwu6K`{oAqu^Bgb#9Usaj7m_e?$ z2_07&e{c9yz^h&{jqfJtck1Kfg==;-tNVXF7ph4ZuWx=@e#E%M=?Pu=4NO$AiR;g0 zp6hiqAEJ|f?KraW8(|K8%PyqSV55rY1DEn?uFT35BL|Pht%!y+M!LG?sb=uUln>Q2 zRBuL`wb*?1D-l}FUMWu<#10ayXCF7dpL|F6lQ|6}JbM(tKu~f@C>I}kh#NCAIgloU zd1AzvW)yG@f#^A2ow47_b0JWwrofqh62YGKvEu76d_q30jN*%`>K&%uxJjoa>`|OR zBnhe7>SZU-oUw~sC7K6O?H%|*8Wps`&~ZA)2_f)p8u$q!jS}J&e)#UT!MAL8=rIHW zU7P?XyqWbl43~ifEL-&aw`cxY5blTma`6Hz$Hh3A+T#DhOBOi$`ADQ;=>;X#m}mv6@}^4 zrcDHdg&zUoDS5BaCiFl?gknnoQhUewX3~70mq!Dz@;@_we)wkp3gPvWu(i%ou3w3w#seh``Ax+rjC>8M2})`yArJ43Xi_T z!40C8LLMaPyoh-EjK}0)vy9sHxbdi{EszaXjx72p;%Hc$zR{^sI1XAJW4e71(GNIi zNIAl(nTLDU^Exu=&cYSiHId(Q3{>mtW#QfSk}h7cn#>4lQZ>HD%fCaJ>Yp;}sMHbH z*L@?1oSc;)^E`;$%kWFU-LIXo?cO=)AV=hVl$CAwwY6n*TbHJw*#)8)2n(s>ydR0z ztGYT-6Cgrf?TO0jD(5sdSO`m`j0|!O3JhF<7ZgW_)9yYWV~n<%P7R)~dAjVh>1tiO z`dGWxuJ#?t*gf!BtCJ)B7R#yUPQtzOB=3m4E%4OHQ+=JZi^XlYd|>z|5qNR)LBCV& z@$#=Sa**i!$+en5W5pz!4S0-M(S0*J`pY>@=Jvl0*+flWpC=`Qcq5uP9}ACm$g8zL zPOj@mR@UnVb7SL?36xX*ePskfZTYOm{p`&B`YM+nc8^az!RVyqeKX2Q{e2kP#X@V1 z@$sg*oSCHt?Ze-*4dc``;-1A5mky zOltr)6u%WJeaVWCNdnyZ-X_nIygjN`9+?mHQV%WXY7a#|S}PMdxx2^ZrL`IypxT2X z;k?x9gL$!X>tShwqZDr_hCW>!QLAf*6Rf>hu+lKl&(AqM=6Y41;^`Tx-V&zxWF$cg z9NX!{0?Iv*?ku-XGbD;zyx^pPnmlVW(W;*T+h|{2Wq18AC9uL`$86sI=g|6lK`$l*Xrr6w=8$PWym`Qih<9A=Y!1lyt4kC z>&u0^-5z;g<#dQ=4=|YqC7&{rA@a8$n zSXIRok7IC-mw{pR=sI+ml-v8fBAk8LMgU`DR^RGv%pk2Kbv&g(oJ<+Or2Z!qzoqIT zrL*n$+MoH?;3bjbhsLYJ^PbBI@ViMO&24;~Xz)sT^yoAw2J)1;>LTd0S#)ZwHAnmE zFx=V$#niXL8{J>1OM&-V;-F&TMLIS442T4d^!+oSDOH&J@yR;etPzzs`V-pw@uN!0 z)p6XrN8fdag=Rpilr{G+1t1A`_5xvuPR)gv=wJgjqon7JzrbGGF?X$Ua!ml z!bkQeF)Cd_R?7rtM*UE&Ij^1BinGAa)(xm)dZ|=FKF^B5CleZuV2m|~EdlozNvDPe zGMxUC*ViPE$1I52hazNacA0uL%k*TU{{fyX0JWg@!2=^WPsvCo8p97&%R1j{v8HwUv_TR(uu!frVUhyW)|_^-1we^?|{kOb#=N?yRB@+#Q znjefJen#U@IfE#C|Hl9XoB!F~PQKMM5fL5#nVaK4eNMB9kRWD0STWWDstIJ*=PF9n zOzzR8zo|(34uN!bpsFAQ?c-nFZ@>GfQ(i`n0&|foJO;IxiLo&bC_P3vx})?&5c2e! zqI)?`uQ+br$Treg(V_xG5q5)OP6BWgeAhvhWw3i#pF#%z60lP_0@$)Z5QSGIXmrrm z&W`12xZIPDg)-4a*CLy|xc#!Bh?!wnhdEN=CD#-ZxvDl0a7WH$U9@Somqm*=mrCiN^Lg$mC!0 z23>w28ivHDq@>C3Rf)`dNxgrD6Od@NM97ty?Lt)%KAGIja_3zy_8iYky;u~=9sr>>**Ic_tm+i2t=wf26MZ?P^cpX4U??&5ma7hOF>L*&?Vb0QN;Yp! z!RJ3}i5pP>w)AVNN+`c~UBJRx6|CkMzFJ&oo^ZRRbh)KQ;WA!*b0gI4_AIqW9+HLl zkHR)DIp%7btx((BN^_K29OaL^ybq6mT~1rS_HMNIt0jESO~IkW4!q0^cj*0i8AfQ{ z$mZtR3ouZtosARy2KB~~{l(1`u+VS7B=l~5S~sE(pAZ+6QW^5Mt#auI7Hs__2@DJz zQBiVViQ?j5^4cyd;vkeWxXxE8`le%D`$}C@MLDsL2|9*;8c5{aduzLnnP=qCB0l0J zsd*NS2k_FIE;kM5BH#-&!-6)bFLgGzsIS9}{yu}J1IiOG%6)hya&VGH+yXD%HUs0?6rXba!05I634%SN;B=$I?kqU3$C%qPpOffPi( zqa3g;Pm`lRS0}ucCV>Ol?s~rM+ceEl5Y0eR)hg_S2Jl{9-;1}UZ{>j}lPYD&J8(BP zD!#c8vsh>MT~jZlRSo`m3Vp zAkb>T{Ssz{NQ!>~0>DHtJmceI&;D#c*%%KPc?nXoc3>IvRjFh8X$3u)k#4`e5Mt_% z_Rd0M%Zi1Jw+JqfV7rPx8i>iv&txE%W-7s$&QF6b%%Ed4%mq+H)z-7ztp=)zNq+1; zjv|0CxNmzV*~P>^q;^tdANPtnS92Qm<1Ks~)Q=3M28i+=*|jilyhTQco`8hJL-e2< zVfdq{PY~a(B(##m=co)3ad9BsHk6(#w;97cNC@(pRhZ#oztH9iPSM1#^JUq^ZNah& zDKcHDRGQ+Hkje-JgpabF8g;Hm3yKp+T?(Bl2k!eU!~3nCXEZ_!u4eGh^Ev^^&M_TZ z%{Psx)A6kvRK^l5V_Njs*aus3ubQ>X%aIE)Dnmh31WPpc$+AIXCyG`?{CTFAQ=sU9 z-iMB})79VGg-1sYA(VvFm%))({5!B)IOQ%&?O!CYY6y5(BbwkHn?T-DVh+bIL^e~r z=>w5oM-urOF7_b|kpSmWELSpLxxt-py#6etm(!Ct-o`~1zNFB+|3XDU+(y(xt|zQ1 z=1id?u!Ll}ps%i+;81d}B`3_2b6;Q`uq{#Y4Ci(CF{b^S60(I=`iOkGnml=E5wG6% z?hhd_7lfBS-kv}pr z%E3?D1OwUHwcA&30W>Pfe-tXI$nxQ9?~$$Jol-y2{qw`f-mZ71f7fd;ZYuPnmKJTt)b%~>5=;bUESCcs>mi-6z^)4`qNG$* ze@}NE`EZ(V4EZc)!zWA7`bwtPK0VF!vNji??a@C)bF$>xwfsYj@y`y446IP-E!y?1 zJKIfq_-;{n>AaK?W?3NFb&*UNlD{>fX$g+_E{UvYWZ*bYev~=YimHlXyTAFBfD*^V z!9jOwS|^*#a2ntNv!6w()&hJ|L<#fh$UPuvCFP=$5r$S16A={*;(O`f$sL?*R5@^wy%UM-^#MP%e?(Ah=Th*{>kEZx{nu2xNU9dpLY|$%zAr4K%uKwu zGcq!wui+3pZd)G*ifXgj45XqEJ$fFoTS^;W_u<5NjmzA=j1*kQb3y!%PyY2x3;&pX zyPM?lRf}e>T3K=KvPheiXA?{`)tO!n<~e3Sb(rL1chd5p&p^V3YoTCj5+}OX14Wp- z)95*>-UmDtL;h-HN61_Ao4u))eW@2>5#%!Fbg~l7wawFS2Y%_6;Id@CslP$F`*RRn zRA#-S^4{G)#Vcawz^-0xJD0Aqn&|Ry>jiAs#BG54-{*cWg9HAusE|wK;<7~Po-Utn zebp!i2M6iFMl}XrjVgWUCAx(94kkT||FATTuOe3X*aOgdN@2ls(yI;7Q1YpXfI?%c zwYP`=*8m3I?c9F z#P`2kVMu(RQU>FDt*isI^cYzzbc`GeTAQ#Roc6Zd_c0JWGSDqS@skOX<#Abanv1wC zBsGY88bO3rD9m=`sg-7J=(EusY4ci1o|SjZN@CvKosx}XvW&+?>3w6^b>{{DX{}`J zdT6-47c-J1!sOyBSh(bmo(-6Rq?=HmVthKUp!vAgMuN@0kpUp%oi+xYEFE+uMV<8$ z6O(@MfxcewAFFkt<$NbMOh`@Duo6fPtO9UZD8oN^{=grNQwQ7hNwg?Y>qB1-3sy5; zDyS7zOJTW(*HC@hd;DPTK|!I)$^Ag#8|O+2SoqU<^HajXF5`rO<|DHWn9 z@O-=Ic_F$G0N3;3nr}lHIgH_2IR1?nd=K1EE7Qv&vli4b>l?zJ5~?OfxE63hF(>gp z`~my`smf;~85d+K%7G?EMlv4&84DfnXC_ovRzTn|!>@M$o!0fAcT8FvX7jnv5?uui zL&W0PR&vk{9X%`1`)3vf8q*Kii9ijSHKvcI(9Vy1IHAKkOdww!*{Z*PjjM_$+XWxV zN6@J+()hlDy0Cm`>vKUnq_?M-S%*on`WM9Pxo@M#A?cW z-svPFagzOckm3?&4VNqQ^fQsXe_NMfMnQaL=sQOS|ECNcOOu$j> z*lgv2=er!VxvcaJ*_9uCS%j36u|&o1~N=PYxBX1{6r1myzM)Ju%p6 z;0=xx^iv8N8Zl1^5NT}&P>w6_IjH)9ER*C{{VG6!vG_j#>Io0k0?f!m=Dm&e$JmGn z!HjK?Kl6{?2X5M&?4)DNc-(SmMSerNkv8+BF7JEqOY;=`s{t_zu|)Io*&; zEUZG&VD9I_)slj6t09Iq!tFk>by^OG3VIa@-XB#oRECvJMY!d%TMw?qw*=`YF{Ug( z6&4jev{RQ_o>-cZ!n2;GoULPJbo>1JJr@z43#s+iPD^c&0o~yeUElG{Ml8<+wjvp_tY(Y`TTv?u56|rdWdAWMbgM+g=BAL2wd0M=!<>jp~(a(@k zZX9hzV$ockmn(CU^pLDC)PBPv0blgC&}_y|C}N2HhBM;IsJjyV_&0qnq%@PaS{e9A zQvE#I91Dlth7jRZOPb&uikz)hKZhPmt0=?bX*wwHd{dVeLrjb4wjgfAm2GAtAroGg zgZMzzc=1;cuA{5g0~gC=S+_LXFx$Tm-l3(BYS@V!gn6c=W;o3o9zmZIqdcLyQ#TVGpLi zHdSz`)}9s~hD&K4DC}I`i{Nps~z=4gl8S zSuJY>Nx@XbJk~!i;|Pz9F$_)M&UD$z$CRY9X&zUg_FC|Zq*PypBS|w(|IYFRBTR`M z%mVP=i>Hl!F@n+rwe$Q)rWMvdBr@Y;9ALd>#^(N6?FdrJ-4kaz z8#odd4)AQ>YV}f%t!GKITb}2Ee`vnB{s?@c;>%LEL?H7!3a>be-?T(#xvoYNlyrDePf~&QXjj37#B8NI@R^^9T z$?k?w+zA9`GHr&^jX~dXav432DSsh&-jRqT{H4vjk!>jPI;ba`K+W;^MLr6>;cGXd z%94GfoBQZzGLWHUWwjxjQ&zVBEfw?JvR->%FL^l;acEo=?^LeKJ$V8{4j@N%@v+)g zO9$gP?88g;Nc8QwH$KfaS&|#MdGgsF_>>{`{eM5xV!Un(Xsz^a zc6;EsT2&m)9zpz_r;m$$p~4|;^``tYV(B{p_d;}eq2CMiRsX&7?a4#C^VLQf$iM<7 z-PPr1&-B^W+bl$JQ)_OH&${CoBPEE1wFl=XqMAgp{r{{}< zPE6cVD_&;>$UX@_p`|Lc>5C20dVe2ze8^3f@Xer9t>H%^%FFe79t;&VYi|XGKk+`> zdkoqeyW}fb*5?5trrvw{>cFW^5A;)1)WsI7Dd!u=qr`rr3+Ss_Nl-~@lJJbxX$;Tv zjM2Om#rt!zGmX$2gt*G=B9W=?YK=}k65HsvV2SI-x}|1Zeoypsw?d4F;F*90YDze_ z-t}f7>T$xIQhz5+tj`0Q+ zOPz++YtZ?nU{vL#lC_sG29f9X0daI-KtK!xcMMJF*iEKEZHIgmohL2~C?sP*4_jx&-}A-Bw?o|~Wlao_aZoE3p?M?3KA;=+xYg(C)q z$_7^76JCV|&#yVPs zqX?IipqKgGKk!P{&l%jFnvf79s-L11*#2i`<`FdI7RJeEk1w0&9Do7H!{g-dL@2j= zIDSj&|DRbQsg+Gz269$5{WPw)IAk%IIn077A6H;k|;_2N6d#HK=Mq9g54RACDv@dF0PwY`i z!-3quwAO>uo!NSQul-OV`?;>_hovLk+~YeA-GakA9+6dmuC#2d=1^kn!l@WmWP@}s zJpTWeD;TF=0Tf`gNgqn8k^7AKlk2(B)nC9GyAsYxjg7k2gx<$cZJw{US!k5RGj!oE ztiHKA5AC8Ye+;O7Dd5}y6+kA~wyMbUKH8SyieXVi%6bTC?5Tf>_uJZ9S-rKK!ms8<79Rz7IJNY=u7k2f^Bp>- z=&Y=)@;WrP>?{^QeDGUtznUIL4_y6wdjjIn9%aB?a4?3d=X|eWma-27JESU@=m*d#@Owg1$OKrg|7h|kSq2iOGXWPC4oVb?8R&rKe<{Ux zH!Jww)qS~*9DrX8*1rM*nI2_+7`GRq9{%&|?(PlFd-MGcxg`v);B5cZb@t$j9r)B#ft_rAn<*kZ1btuioAt z@Nd}!0?-u_TqV(t&!{^{c<(wi`PVo=h5<5I{0{wepu^X<*MO8QoUnfr>mj&9GLdKT zV1W5k3tw#Z28LR2H`}OZ9+*m1VMy>Of9mZ@L(%?)r8-(Na=<@<)@c5_V8jP}zyw}s zx0eO}S5F5=>h2~;@tvw1`0o!JQ2PX6$=eAX{FnFvv=9t|N?s8Gw?c{kTMYmGd!#WW z-k7&f!T#==ZZ}WsEinm<(CXiGi97v+8@@0y-Cm2T|2JG&!csHl22a%9HroMrlNUfz z^!nYdWmMl8PP{pdK3XM6GWBjydjBzzic~E_GJ8irIh(^Yz Date: Tue, 28 Jan 2025 14:40:11 +0800 Subject: [PATCH 19/25] doc: Update README.md (#20339) --- README.md | 27 ++++++++++----------------- 1 file changed, 10 insertions(+), 17 deletions(-) diff --git a/README.md b/README.md index 96bc0f35c7ddf..8717e7725e920 100644 --- a/README.md +++ b/README.md @@ -9,25 +9,18 @@
-### 🌊 Reimagine real-time data engineering. +### 🌊 Ride the Wave of Real-Time Data.
- -

   ðŸ“š  - Documentation   ðŸš€  - - Slack Community - +

+ Docs | Benchmarks | Demos

+ +

+

-RisingWave is a Postgres-compatible SQL database engineered to provide the simplest and most cost-efficient approach for processing, analyzing, and managing real-time event streaming data. +RisingWave is the world's most advanced streaming database engineered to provide the simplest and most cost-efficient approach for processing, analyzing, and managing real-time event streaming data. It provides both a Postgres-compatible [SQL interface](https://docs.risingwave.com/sql/overview) and a DataFrame-style [Python interface](https://docs.risingwave.com/python-sdk/intro). -RisingWave can ingest millions of events per second, continuously join and analyze live data streams with historical tables, serve ad-hoc queries in real-time, and deliver fresh, consistent results wherever needed. +RisingWave can ingest millions of events per second, continuously join and analyze live data streams with historical tables, serve ad-hoc queries at low latency, and deliver fresh, consistent results wherever needed. -![RisingWave](./docs/dev/src/images/architecture_20240908.png) +![RisingWave](./docs/dev/src/images/architecture_20250127.png) ## Try it out in 60 seconds From 08bdf2743469d7aab35c98403f53a3f4738b93f5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 28 Jan 2025 16:21:57 +0800 Subject: [PATCH 20/25] chore(deps): Bump const-str from 0.5.6 to 0.6.0 (#20335) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- src/cmd_all/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 387706a3fbcb6..6f18a4607ad36 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2724,9 +2724,9 @@ dependencies = [ [[package]] name = "const-str" -version = "0.5.6" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aca749d3d3f5b87a0d6100509879f9cf486ab510803a4a4e1001da1ff61c2bd6" +checksum = "671927c085eb5827d30b95df08f6c6a2301eafe2274c368bb2c16f42e03547eb" [[package]] name = "constant_time_eq" diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index 698ee898109bd..8a0080f59f612 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -27,7 +27,7 @@ ignored = ["workspace-hack", "workspace-config", "task_stats_alloc", "tikv-jemal [dependencies] clap = { workspace = true } console = "0.15" -const-str = "0.5" +const-str = "0.6" home = "0.5" risingwave_batch_executors = { workspace = true } risingwave_cmd = { workspace = true } From 92b77f9062f79841bef553d0b5288c432ac01b12 Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Sat, 1 Feb 2025 03:26:53 +0800 Subject: [PATCH 21/25] doc: Update README.md (#20355) --- README.md | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 8717e7725e920..1748fd236f947 100644 --- a/README.md +++ b/README.md @@ -39,7 +39,7 @@ -RisingWave is the world's most advanced streaming database engineered to provide the simplest and most cost-efficient approach for processing, analyzing, and managing real-time event streaming data. It provides both a Postgres-compatible [SQL interface](https://docs.risingwave.com/sql/overview) and a DataFrame-style [Python interface](https://docs.risingwave.com/python-sdk/intro). +RisingWave is the world's most advanced event stream processing platform engineered to provide the simplest and most cost-efficient approach for processing, analyzing, and managing real-time event streaming data. It provides both a Postgres-compatible [SQL interface](https://docs.risingwave.com/sql/overview) and a DataFrame-style [Python interface](https://docs.risingwave.com/python-sdk/intro). RisingWave can ingest millions of events per second, continuously join and analyze live data streams with historical tables, serve ad-hoc queries at low latency, and deliver fresh, consistent results wherever needed. @@ -57,11 +57,10 @@ To learn about other installation options, such as using a Docker image, see [Qu ## When is RisingWave the perfect fit? RisingWave is the ideal solution for: -* Managing real-time data sources like Kafka streams, database CDC, and more. -* Executing complex, on-the-fly queries, including joins, aggregations, and time windowing. -* Interactively and concurrently exploring consistent, up-to-the-moment results. -* Seamlessly delivering results to downstream systems. -* Processing both streaming and batch data with a unified codebase. +* **Ingestion**: Ingest millions of events per second from both streaming and batch data sources. +* **Stream processing**: Perform real-time incremental data processing to join and analyze live data streams with historical tables. +* **Serving**: Persist data and serve ad-hoc queries with single-digit millisecond latency. +* **Delivery**: Deliver fresh, consistent results to data lakes (e.g., Apache Iceberg) or any destination. ## In what use cases does RisingWave excel? From 654b836d24c6dcc98bbbfdc7b67ca76c9ec45575 Mon Sep 17 00:00:00 2001 From: Jaya Kasa Date: Sat, 1 Feb 2025 02:58:25 -0500 Subject: [PATCH 22/25] feat(iceberg): add 'glue.id' to iceberg sink config (#20330) --- src/connector/src/connector_common/connection.rs | 5 +++++ src/connector/src/connector_common/iceberg/mod.rs | 8 ++++++++ src/connector/src/sink/iceberg/mod.rs | 1 + src/connector/with_options_sink.yaml | 6 ++++++ src/connector/with_options_source.yaml | 6 ++++++ 5 files changed, 26 insertions(+) diff --git a/src/connector/src/connector_common/connection.rs b/src/connector/src/connector_common/connection.rs index 3604b1bd2f99c..89e7711929ce5 100644 --- a/src/connector/src/connector_common/connection.rs +++ b/src/connector/src/connector_common/connection.rs @@ -135,6 +135,10 @@ pub struct IcebergConnection { /// Path of iceberg warehouse, only applicable in storage catalog. #[serde(rename = "warehouse.path")] pub warehouse_path: Option, + /// Catalog id, can be omitted for storage catalog or when + /// caller's AWS account ID matches glue id + #[serde(rename = "glue.id")] + pub glue_id: Option, /// Catalog name, can be omitted for storage catalog, but /// must be set for other catalogs. #[serde(rename = "catalog.name")] @@ -255,6 +259,7 @@ impl Connection for IcebergConnection { secret_key: self.secret_key.clone(), gcs_credential: self.gcs_credential.clone(), warehouse_path: self.warehouse_path.clone(), + glue_id: self.glue_id.clone(), catalog_name: self.catalog_name.clone(), catalog_uri: self.catalog_uri.clone(), credential: self.credential.clone(), diff --git a/src/connector/src/connector_common/iceberg/mod.rs b/src/connector/src/connector_common/iceberg/mod.rs index c04e20e53c202..936177668d765 100644 --- a/src/connector/src/connector_common/iceberg/mod.rs +++ b/src/connector/src/connector_common/iceberg/mod.rs @@ -57,6 +57,10 @@ pub struct IcebergCommon { /// Path of iceberg warehouse, only applicable in storage catalog. #[serde(rename = "warehouse.path")] pub warehouse_path: Option, + /// AWS Client id, can be omitted for storage catalog or when + /// caller's AWS account ID matches glue id + #[serde(rename = "glue.id")] + pub glue_id: Option, /// Catalog name, can be omitted for storage catalog, but /// must be set for other catalogs. #[serde(rename = "catalog.name")] @@ -274,6 +278,10 @@ impl IcebergCommon { format!("https://glue.{}.amazonaws.com", region), ); } + + if let Some(glue_id) = self.glue_id.as_deref() { + java_catalog_configs.insert("glue.id".to_owned(), glue_id.to_owned()); + } } _ => {} } diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index f84d0aa2cb339..a18287062bd47 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -1432,6 +1432,7 @@ mod test { secret_key: Some("hummockadmin".to_owned()), gcs_credential: None, catalog_type: Some("jdbc".to_owned()), + glue_id: None, catalog_name: Some("demo".to_owned()), database_name: Some("demo_db".to_owned()), table_name: "demo_table".to_owned(), diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 819e597fec448..15bdba601ba78 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -447,6 +447,12 @@ IcebergConfig: field_type: String comments: Path of iceberg warehouse, only applicable in storage catalog. required: false + - name: glue.id + field_type: String + comments: |- + AWS Client id, can be omitted for storage catalog or when + caller's AWS account ID matches glue id + required: false - name: catalog.name field_type: String comments: |- diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 5b6545e7086f8..cb9329190b903 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -103,6 +103,12 @@ IcebergProperties: field_type: String comments: Path of iceberg warehouse, only applicable in storage catalog. required: false + - name: glue.id + field_type: String + comments: |- + AWS Client id, can be omitted for storage catalog or when + caller's AWS account ID matches glue id + required: false - name: catalog.name field_type: String comments: |- From 552925b2562e0ecc553293f146a7c187c1eb7e88 Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Mon, 3 Feb 2025 11:16:11 +0800 Subject: [PATCH 23/25] docs: Update README.md (#20358) --- README.md | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 1748fd236f947..139feeb6f0381 100644 --- a/README.md +++ b/README.md @@ -54,8 +54,8 @@ curl -L https://risingwave.com/sh | sh To learn about other installation options, such as using a Docker image, see [Quick Start](https://docs.risingwave.com/docs/current/get-started/). -## When is RisingWave the perfect fit? -RisingWave is the ideal solution for: +## What Is RisingWave Optimized For? +RisingWave simplifies the development of real-time data pipelines and applications. It is purpose-built to: * **Ingestion**: Ingest millions of events per second from both streaming and batch data sources. * **Stream processing**: Perform real-time incremental data processing to join and analyze live data streams with historical tables. @@ -63,6 +63,21 @@ RisingWave is the ideal solution for: * **Delivery**: Deliver fresh, consistent results to data lakes (e.g., Apache Iceberg) or any destination. +## Why Is RisingWave Special? +RisingWave stands apart from traditional stream processing systems due to its: + +### PostgreSQL Compatibility +* **Seamless integration**: Works with a wide range of tools in the PostgreSQL ecosystem. +* **Expressive SQL**: Supports structured, semi-structured, and unstructured data using a rich, familiar SQL dialect. +* **No manual state tuning**: Eliminates the need for complex state management configurations. + +### Decoupled Compute and Storage +* **Optimized for complex queries**: Ensures high performance, especially for complex operations like joins and time windowing. +* **Fast failure recovery**: Recovers from system crashes within seconds. +* **Dynamic scaling**: Adjusts resources instantly to handle workload spikes. + + + ## In what use cases does RisingWave excel? RisingWave is particularly effective for the following use cases: From 67ee5a5f1bdfe0ef554291146c85d4cfabaa19c9 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 3 Feb 2025 15:56:19 +0800 Subject: [PATCH 24/25] feat(stream): normalize unmatched updates (#20350) --- ci/workflows/main-cron.yml | 20 ++-- ci/workflows/pull-request.yml | 8 +- e2e_test/streaming/bug_fixes/issue_20342.slt | 9 ++ .../streaming/bug_fixes/issue_20342.slt.part | 99 ++++++++++++++++ src/stream/src/executor/backfill/utils.rs | 110 +++++++++++++----- 5 files changed, 204 insertions(+), 42 deletions(-) create mode 100644 e2e_test/streaming/bug_fixes/issue_20342.slt create mode 100644 e2e_test/streaming/bug_fixes/issue_20342.slt.part diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 8fcc035500398..f977c65a056ad 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -144,7 +144,7 @@ steps: files: "*-junit.xml" format: "junit" - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 13 + timeout_in_minutes: 18 retry: *auto-retry - label: "end-to-end test (parallel, in-memory) (release)" @@ -160,7 +160,7 @@ steps: plugins: - docker-compose#v5.5.0: *docker-compose - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 15 + timeout_in_minutes: 20 retry: *auto-retry - group: "end-to-end connector test (release)" @@ -342,7 +342,7 @@ steps: - label: "end-to-end test (madsim)" key: "e2e-test-deterministic" - command: "TEST_NUM=32 timeout 130m ci/scripts/deterministic-e2e-test.sh" + command: "TEST_NUM=32 timeout 135m ci/scripts/deterministic-e2e-test.sh" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation" @@ -357,12 +357,12 @@ steps: environment: - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 135 + timeout_in_minutes: 140 retry: *auto-retry - label: "end-to-end test (madsim, random vnode count)" key: "e2e-test-deterministic-random-vnode-count" - command: "TEST_NUM=32 RW_SIM_RANDOM_VNODE_COUNT=true timeout 130m ci/scripts/deterministic-e2e-test.sh" + command: "TEST_NUM=32 RW_SIM_RANDOM_VNODE_COUNT=true timeout 135m ci/scripts/deterministic-e2e-test.sh" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation" @@ -377,12 +377,12 @@ steps: environment: - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 135 + timeout_in_minutes: 140 retry: *auto-retry - label: "recovery test (madsim)" key: "recovery-test-deterministic" - command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 timeout 70m ci/scripts/deterministic-recovery-test.sh" + command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 timeout 75m ci/scripts/deterministic-recovery-test.sh" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" @@ -392,13 +392,13 @@ steps: - docker-compose#v5.5.0: *docker-compose # Only upload zipped files, otherwise the logs is too much. - ./ci/plugins/upload-failure-logs-zipped - timeout_in_minutes: 75 + timeout_in_minutes: 80 retry: *auto-retry # Ddl statements will randomly run with background_ddl. - label: "background_ddl, arrangement_backfill recovery test (madsim)" key: "background-ddl-arrangement-backfill-recovery-test-deterministic" - command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.8 USE_ARRANGEMENT_BACKFILL=true timeout 70m ci/scripts/deterministic-recovery-test.sh" + command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.8 USE_ARRANGEMENT_BACKFILL=true timeout 75m ci/scripts/deterministic-recovery-test.sh" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" @@ -408,7 +408,7 @@ steps: - docker-compose#v5.5.0: *docker-compose # Only upload zipped files, otherwise the logs is too much. - ./ci/plugins/upload-failure-logs-zipped - timeout_in_minutes: 70 + timeout_in_minutes: 80 retry: *auto-retry - label: "end-to-end iceberg sink test (release)" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 19fda2f2d99d5..1b2adcbac28b7 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -160,7 +160,7 @@ steps: plugins: - docker-compose#v5.5.0: *docker-compose - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 25 + timeout_in_minutes: 30 retry: *auto-retry - label: "end-to-end test for opendal (parallel)" @@ -562,7 +562,7 @@ steps: environment: - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 30 + timeout_in_minutes: 35 cancel_on_build_failing: true retry: *auto-retry @@ -583,7 +583,7 @@ steps: # - test-collector#v1.0.0: # files: "*-junit.xml" # format: "junit" - timeout_in_minutes: 35 + timeout_in_minutes: 40 cancel_on_build_failing: true retry: *auto-retry @@ -748,7 +748,7 @@ steps: run: ci-standard-env propagate-environment: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 32 + timeout_in_minutes: 37 retry: *auto-retry # FIXME(kwannoel): Let the github PR labeller label it, if sqlsmith source files has changes. diff --git a/e2e_test/streaming/bug_fixes/issue_20342.slt b/e2e_test/streaming/bug_fixes/issue_20342.slt new file mode 100644 index 0000000000000..f36740eb3068a --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_20342.slt @@ -0,0 +1,9 @@ +statement ok +set streaming_use_arrangement_backfill=false; + +include ./issue_20342.slt.part + +statement ok +set streaming_use_arrangement_backfill=true; + +include ./issue_20342.slt.part diff --git a/e2e_test/streaming/bug_fixes/issue_20342.slt.part b/e2e_test/streaming/bug_fixes/issue_20342.slt.part new file mode 100644 index 0000000000000..5c93e0a0434c9 --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_20342.slt.part @@ -0,0 +1,99 @@ +statement ok +set streaming_parallelism=1; + +# pk: [v1] +# stream key: [v1] +statement ok +create table t(v1 int primary key, v2 int); + +# pk: [v2, v1] +# stream key: [v1] +statement ok +create materialized view m1 as select * from t order by v2; + +statement ok +insert into t select x as v1, x as v2 from generate_series(1, 10000) t(x); + +statement ok +flush; + +skipif madsim +statement ok +set backfill_rate_limit=1; + +skipif madsim +statement ok +set background_ddl=true; + +statement ok +create materialized view m2 as select count(*) from m1; + +skipif madsim +sleep 2s + +statement ok +update t set v2 = 100000 where v1 = 1; + +statement ok +flush; + +statement ok +update t set v2 = 100001 where v1 = 2; + +statement ok +flush; + +statement ok +update t set v2 = 100002 where v1 = 3; + +statement ok +flush; + +statement ok +update t set v2 = 100003 where v1 = 4; + +statement ok +flush; + +statement ok +update t set v2 = 100004 where v1 = 5; + +statement ok +flush; + +statement ok +update t set v2 = 100005 where v1 = 6; + +statement ok +flush; + +statement ok +update t set v2 = 100006 where v1 = 7; + +statement ok +flush; + +statement ok +update t set v2 = 100007 where v1 = 8; + +statement ok +flush; + +statement ok +set backfill_rate_limit=default; + +statement ok +set background_ddl=false; + +statement ok +set streaming_use_arrangement_backfill=true; + +statement ok +alter materialized view m2 set backfill_rate_limit=default; + +skipif madsim +statement ok +wait; + +statement ok +drop table t cascade; \ No newline at end of file diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 06f0a13ed97fb..7128c55986a47 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cmp::Ordering; +use std::borrow::Cow; use std::collections::HashMap; use std::ops::Bound; @@ -341,31 +341,37 @@ pub(crate) fn mark_chunk_ref_by_vnode( let chunk = chunk.clone(); let (data, ops) = chunk.into_parts(); let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); - // Use project to avoid allocation. - for row in data.rows() { + + let mut new_ops: Cow<'_, [Op]> = Cow::Borrowed(ops.as_ref()); + let mut unmatched_update_delete = false; + let mut visible_update_delete = false; + for (i, (op, row)) in ops.iter().zip_eq_debug(data.rows()).enumerate() { let pk = row.project(pk_in_output_indices); let vnode = upstream_table.compute_vnode_by_pk(pk); - let v = match backfill_state.get_progress(&vnode)? { + let visible = match backfill_state.get_progress(&vnode)? { // We want to just forward the row, if the vnode has finished backfill. BackfillProgressPerVnode::Completed { .. } => true, // If not started, no need to forward. BackfillProgressPerVnode::NotStarted => false, // If in progress, we need to check row <= current_pos. BackfillProgressPerVnode::InProgress { current_pos, .. } => { - match cmp_datum_iter(pk.iter(), current_pos.iter(), pk_order.iter().copied()) { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } + cmp_datum_iter(pk.iter(), current_pos.iter(), pk_order.iter().copied()).is_le() } }; - new_visibility.append(v); + new_visibility.append(visible); + + normalize_unmatched_updates( + &mut new_ops, + &mut unmatched_update_delete, + &mut visible_update_delete, + visible, + i, + op, + ); } let (columns, _) = data.into_parts(); - Ok(StreamChunk::with_visibility( - ops, - columns, - new_visibility.finish(), - )) + let chunk = StreamChunk::with_visibility(new_ops, columns, new_visibility.finish()); + Ok(chunk) } /// Mark chunk: @@ -379,22 +385,74 @@ fn mark_chunk_inner( ) -> StreamChunk { let (data, ops) = chunk.into_parts(); let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); - // Use project to avoid allocation. - for v in data.rows().map(|row| { + let mut new_ops: Cow<'_, [Op]> = Cow::Borrowed(ops.as_ref()); + let mut unmatched_update_delete = false; + let mut visible_update_delete = false; + for (i, (op, row)) in ops.iter().zip_eq_debug(data.rows()).enumerate() { let lhs = row.project(pk_in_output_indices); let rhs = current_pos; - let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); - match order { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } - }) { - new_visibility.append(v); + let visible = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()).is_le(); + new_visibility.append(visible); + + normalize_unmatched_updates( + &mut new_ops, + &mut unmatched_update_delete, + &mut visible_update_delete, + visible, + i, + op, + ); } let (columns, _) = data.into_parts(); StreamChunk::with_visibility(ops, columns, new_visibility.finish()) } +/// We will rewrite unmatched U-/U+ into +/- ops. +/// They can be unmatched because while they will always have the same stream key, +/// their storage pk might be different. Here we use storage pk (`current_pos`) to filter them, +/// as such, a U+ might be filtered out, but their corresponding U- could be kept, and vice versa. +/// +/// This hanging U-/U+ can lead to issues downstream, since we work with an assumption in the +/// system that there's never hanging U-/U+. +fn normalize_unmatched_updates( + normalized_ops: &mut Cow<'_, [Op]>, + unmatched_update_delete: &mut bool, + visible_update_delete: &mut bool, + current_visibility: bool, + current_op_index: usize, + current_op: &Op, +) { + if *unmatched_update_delete { + assert_eq!(*current_op, Op::UpdateInsert); + let visible_update_insert = current_visibility; + match (visible_update_delete, visible_update_insert) { + (true, false) => { + // Lazily clone the ops here. + let ops = normalized_ops.to_mut(); + ops[current_op_index - 1] = Op::Delete; + } + (false, true) => { + // Lazily clone the ops here. + let ops = normalized_ops.to_mut(); + ops[current_op_index] = Op::Insert; + } + (true, true) | (false, false) => {} + } + *unmatched_update_delete = false; + } else { + match current_op { + Op::UpdateDelete => { + *unmatched_update_delete = true; + *visible_update_delete = current_visibility; + } + Op::UpdateInsert => { + unreachable!("UpdateInsert should not be present without UpdateDelete") + } + _ => {} + } + } +} + fn mark_cdc_chunk_inner( offset_parse_func: &CdcOffsetParseFunc, chunk: StreamChunk, @@ -422,11 +480,7 @@ fn mark_cdc_chunk_inner( if in_binlog_range { let lhs = row.project(pk_in_output_indices); let rhs = current_pos; - let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); - match order { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } + cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()).is_le() } else { false } From 3ee72eb15819074a1344d82e8205039731745cdf Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Mon, 3 Feb 2025 17:47:11 +0800 Subject: [PATCH 25/25] refactor(proto): add `LATEST` const value for proto message version enums (#20359) Signed-off-by: Richard Chien --- proto/plan_common.proto | 3 -- proto/stream_plan.proto | 3 -- src/common/src/catalog/column.rs | 14 +++---- src/common/src/catalog/test_utils.rs | 4 +- .../codec/src/decoder/avro/schema.rs | 4 +- .../codec/src/decoder/protobuf/parser.rs | 4 +- src/connector/src/source/manager.rs | 2 +- src/frontend/src/binder/expr/mod.rs | 2 +- src/frontend/src/catalog/table_catalog.rs | 2 +- src/frontend/src/handler/create_table.rs | 2 +- .../optimizer/plan_node/stream_hash_agg.rs | 2 +- .../optimizer/plan_node/stream_simple_agg.rs | 2 +- .../compaction/compaction_group_schedule.rs | 2 +- src/meta/src/hummock/manager/transaction.rs | 2 +- src/prost/build.rs | 14 ++++++- src/prost/helpers/src/lib.rs | 41 ++++++++++++++++++- src/stream/src/executor/aggregation/minput.rs | 24 +++++------ src/stream/src/executor/test_utils.rs | 4 +- 18 files changed, 87 insertions(+), 44 deletions(-) diff --git a/proto/plan_common.proto b/proto/plan_common.proto index b8b4989e78b8a..5fb77a2aa0659 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -21,9 +21,6 @@ enum ColumnDescVersion { // Introduced in https://github.com/risingwavelabs/risingwave/pull/13707#discussion_r1429947537, // in case DEFAULT_KEY_COLUMN_NAME changes COLUMN_DESC_VERSION_PR_13707 = 1; - - // for test only - COLUMN_DESC_VERSION_MAX = 2147483647; } message ColumnDesc { diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 3963e3d03e4c3..9b886e3cef1b3 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -356,9 +356,6 @@ enum AggNodeVersion { // https://github.com/risingwavelabs/risingwave/issues/13465#issuecomment-1821016508 AGG_NODE_VERSION_ISSUE_13465 = 2; - - // Used for test only. - AGG_NODE_VERSION_MAX = 2147483647; } message SimpleAggNode { diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 146ef76d2ec93..8a1a4a5072d6b 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -137,7 +137,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -152,7 +152,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -193,7 +193,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: additional_column_type, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -213,7 +213,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: Some(system_column), } } @@ -263,7 +263,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -285,7 +285,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -304,7 +304,7 @@ impl ColumnDesc { description: None, generated_or_default_column: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index ab4afeddea566..15383bd3c3e7c 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -36,7 +36,7 @@ impl ColumnDescTestExt for ColumnDesc { column_id, name: name.to_owned(), additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, ..Default::default() } } @@ -62,7 +62,7 @@ impl ColumnDescTestExt for ColumnDesc { description: None, additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, } } } diff --git a/src/connector/codec/src/decoder/avro/schema.rs b/src/connector/codec/src/decoder/avro/schema.rs index 20218fccbb026..8a4cfaa56999f 100644 --- a/src/connector/codec/src/decoder/avro/schema.rs +++ b/src/connector/codec/src/decoder/avro/schema.rs @@ -152,7 +152,7 @@ fn avro_field_to_column_desc( description: None, additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, }) } _ => { @@ -162,7 +162,7 @@ fn avro_field_to_column_desc( column_id: *index, name: name.to_owned(), additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, ..Default::default() }) } diff --git a/src/connector/codec/src/decoder/protobuf/parser.rs b/src/connector/codec/src/decoder/protobuf/parser.rs index 6701d4aaa7a31..b1916a8086f96 100644 --- a/src/connector/codec/src/decoder/protobuf/parser.rs +++ b/src/connector/codec/src/decoder/protobuf/parser.rs @@ -79,7 +79,7 @@ fn pb_field_to_col_desc( description: None, additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, }) } else { *index += 1; @@ -88,7 +88,7 @@ fn pb_field_to_col_desc( name: field_descriptor.name().to_owned(), column_type: Some(field_type.to_protobuf()), additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, ..Default::default() }) } diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index e6618d92c185b..5bd45579f500f 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -185,7 +185,7 @@ impl From<&SourceColumnDesc> for ColumnDesc { type_name: "".to_owned(), generated_or_default_column: None, description: None, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 2002555c857be..6373023090803 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -1000,7 +1000,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, }) } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 53e72d5fb77c9..fcae2de055e71 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -920,7 +920,7 @@ mod tests { description: None, generated_or_default_column: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, }, is_hidden: false diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index a8590d3480947..e68d34e85928d 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -253,7 +253,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, }, is_hidden: false, diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index e52ef3a2965a7..7146fdf889cc2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -216,7 +216,7 @@ impl StreamNode for StreamHashAgg { .collect(), row_count_index: self.row_count_idx as u32, emit_on_window_close: self.base.emit_on_window_close(), - version: PbAggNodeVersion::Issue13465 as _, + version: PbAggNodeVersion::LATEST as _, })) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs index 9a67fab72d682..3c9102a9434d5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -148,7 +148,7 @@ impl StreamNode for StreamSimpleAgg { }) .collect(), row_count_index: self.row_count_idx as u32, - version: PbAggNodeVersion::Issue13465 as _, + version: PbAggNodeVersion::LATEST as _, must_output_per_barrier: self.must_output_per_barrier, })) } diff --git a/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs b/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs index f346902bc242d..15600367fffb3 100644 --- a/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs +++ b/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs @@ -458,7 +458,7 @@ impl HummockManager { parent_group_id, new_sst_start_id, table_ids: vec![], - version: CompatibilityVersion::SplitGroupByTableId as i32, // for compatibility + version: CompatibilityVersion::LATEST as _, // for compatibility split_key: Some(split_key.into()), })], }, diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 335c8e8d157e5..29d06552e2ee7 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -135,7 +135,7 @@ impl<'a> HummockVersionTransaction<'a> { parent_group_id: StaticCompactionGroupId::NewCompactionGroup as CompactionGroupId, new_sst_start_id: 0, // No need to set it when `NewCompactionGroup` table_ids: vec![], - version: CompatibilityVersion::SplitGroupByTableId as i32, + version: CompatibilityVersion::LATEST as _, split_key: None, })); } diff --git a/src/prost/build.rs b/src/prost/build.rs index faf284dff7c8f..867d11bc918a0 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -246,7 +246,19 @@ fn main() -> Result<(), Box> { .type_attribute( "hummock.TableWatermarks.EpochNewWatermarks", "#[derive(Eq)]", - ); + ) + // proto version enums + .type_attribute("stream_plan.AggNodeVersion", "#[derive(prost_helpers::Version)]") + .type_attribute( + "plan_common.ColumnDescVersion", + "#[derive(prost_helpers::Version)]", + ) + .type_attribute( + "hummock.CompatibilityVersion", + "#[derive(prost_helpers::Version)]", + ) + // end + ; // If any configuration for `prost_build` is not exposed by `tonic_build`, specify it here. let mut prost_config = prost_build::Config::new(); diff --git a/src/prost/helpers/src/lib.rs b/src/prost/helpers/src/lib.rs index 0b6fd3f13b94f..34c0b765df503 100644 --- a/src/prost/helpers/src/lib.rs +++ b/src/prost/helpers/src/lib.rs @@ -16,9 +16,9 @@ #![feature(iterator_try_collect)] use proc_macro::TokenStream; -use proc_macro2::TokenStream as TokenStream2; +use proc_macro2::{Span, TokenStream as TokenStream2}; use quote::{format_ident, quote}; -use syn::{parse_macro_input, DataStruct, DeriveInput, Result}; +use syn::{parse_macro_input, Data, DataStruct, DeriveInput, Result}; mod generate; @@ -69,3 +69,40 @@ fn produce(ast: &DeriveInput) -> Result { #struct_get }) } + +#[cfg_attr(coverage, coverage(off))] +#[proc_macro_derive(Version)] +pub fn version(input: TokenStream) -> TokenStream { + fn version_inner(ast: &DeriveInput) -> syn::Result { + let last_variant = match &ast.data { + Data::Enum(v) => v.variants.iter().last().ok_or_else(|| { + syn::Error::new( + Span::call_site(), + "This macro requires at least one variant in the enum.", + ) + })?, + _ => { + return Err(syn::Error::new( + Span::call_site(), + "This macro only supports enums.", + )); + } + }; + + let enum_name = &ast.ident; + let last_variant_name = &last_variant.ident; + + Ok(quote! { + impl #enum_name { + pub const LATEST: Self = Self::#last_variant_name; + } + }) + } + + let ast = parse_macro_input!(input as DeriveInput); + + match version_inner(&ast) { + Ok(tokens) => tokens.into(), + Err(e) => e.to_compile_error().into(), + } +} diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index 4967696c01cfb..eee4f2d23a596 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -399,7 +399,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -457,7 +457,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -505,7 +505,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -563,7 +563,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -627,7 +627,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state_1 = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call_1, &PkIndices::new(), // unused &order_columns_1, @@ -642,7 +642,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state_2 = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call_2, &PkIndices::new(), // unused &order_columns_2, @@ -730,7 +730,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -787,7 +787,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -837,7 +837,7 @@ mod tests { ColumnOrder::new(1, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -945,7 +945,7 @@ mod tests { ColumnOrder::new(1, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -1064,7 +1064,7 @@ mod tests { ColumnOrder::new(4, OrderType::ascending()), // _row_id ASC ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -1153,7 +1153,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ASC ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index bdc057341a65d..c15f648660436 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -484,7 +484,7 @@ pub mod agg_executor { }; let exec = HashAggExecutor::::new(AggExecutorArgs { - version: PbAggNodeVersion::Max, + version: PbAggNodeVersion::LATEST, input, actor_ctx: ActorContext::for_test(123), @@ -552,7 +552,7 @@ pub mod agg_executor { }; let exec = SimpleAggExecutor::new(AggExecutorArgs { - version: PbAggNodeVersion::Max, + version: PbAggNodeVersion::LATEST, input, actor_ctx,