From 8e4b4cdf0cc8cbef63ec0bebeeb7c45993b482e1 Mon Sep 17 00:00:00 2001 From: zhangli20 Date: Wed, 12 Jun 2024 16:33:48 +0800 Subject: [PATCH] release version 3.0.0 refactor join implementations to support existence joins and BHJ building hash map on driver side. supports spark333 batch shuffle reading. update rust-toolchain to latest nightly version. other minor improvements. update docs. --- .github/workflows/build-ce7-releases.yml | 2 +- .github/workflows/tpcds.yml | 15 +- Cargo.lock | 51 +- Cargo.toml | 36 +- README.md | 22 +- RELEASES.md | 15 +- benchmark-results/20240202.md | 152 -- benchmark-results/20240701-blaze300.md | 201 ++ ...ter-resources-cost-comparison-20240202.png | Bin 105216 -> 0 bytes .../blaze-query-time-comparison-20240202.png | Bin 98182 -> 0 bytes benchmark-results/kwai1-hardware-conf.md | 306 +-- ...3-vs-blaze300-query-time-20240701-tpch.png | Bin 0 -> 73734 bytes ...ark333-vs-blaze300-query-time-20240701.png | Bin 0 -> 92487 bytes native-engine/blaze-jni-bridge/src/conf.rs | 2 + native-engine/blaze-serde/proto/blaze.proto | 66 +- native-engine/blaze-serde/src/from_proto.rs | 161 +- native-engine/blaze-serde/src/lib.rs | 7 +- .../datafusion-ext-commons/src/lib.rs | 5 +- .../datafusion-ext-commons/src/spark_hash.rs | 6 +- .../src/spark_get_json_object.rs | 38 +- .../src/spark_null_if.rs | 8 +- .../src/spark_strings.rs | 4 +- native-engine/datafusion-ext-plans/Cargo.toml | 1 + .../src/broadcast_join_build_hash_map_exec.rs | 150 ++ .../src/broadcast_join_exec.rs | 602 +++--- .../src/broadcast_nested_loop_join_exec.rs | 252 --- .../src/common/batch_selection.rs | 37 +- .../datafusion-ext-plans/src/common/output.rs | 32 + .../src/ipc_reader_exec.rs | 4 +- .../src/joins/bhj/full_join.rs | 324 +++ .../datafusion-ext-plans/src/joins/bhj/mod.rs | 146 ++ .../src/joins/bhj/semi_join.rs | 283 +++ .../src/joins/join_hash_map.rs | 340 ++++ .../src/joins/join_utils.rs | 64 + .../datafusion-ext-plans/src/joins/mod.rs | 113 ++ .../src/joins/smj/existence_join.rs | 175 ++ .../src/joins/smj/full_join.rs | 248 +++ .../datafusion-ext-plans/src/joins/smj/mod.rs | 17 + .../src/joins/smj/semi_join.rs | 252 +++ .../src/joins/stream_cursor.rs | 235 +++ .../datafusion-ext-plans/src/joins/test.rs | 947 +++++++++ native-engine/datafusion-ext-plans/src/lib.rs | 22 +- .../datafusion-ext-plans/src/parquet_exec.rs | 72 +- .../src/rename_columns_exec.rs | 16 +- .../datafusion-ext-plans/src/sort_exec.rs | 63 +- .../src/sort_merge_join_exec.rs | 1788 ++--------------- pom.xml | 9 +- rust-toolchain.toml | 4 +- .../apache/spark/sql/blaze/ShimsImpl.scala | 14 +- .../blaze/plan/NativeBroadcastJoinExec.scala | 14 +- .../NativeBroadcastNestedLoopJoinExec.scala | 31 - .../apache/spark/sql/blaze/ShimsImpl.scala | 12 +- .../BlazeBlockStoreShuffleReader.scala | 14 +- .../blaze/shuffle/BlazeShuffleManager.scala | 20 +- .../blaze/plan/NativeBroadcastJoinExec.scala | 15 +- .../NativeBroadcastNestedLoopJoinExec.scala | 34 - .../org/apache/spark/sql/blaze/BlazeConf.java | 16 +- .../sql/blaze/BlazeCallNativeWrapper.scala | 1 - .../sql/blaze/BlazeConvertStrategy.scala | 3 +- .../spark/sql/blaze/BlazeConverters.scala | 243 +-- .../spark/sql/blaze/NativeConverters.scala | 4 +- .../org/apache/spark/sql/blaze/Shims.scala | 8 +- .../sql/blaze/SparkUDFWrapperContext.scala | 1 - .../apache/spark/sql/blaze/util/Using.scala | 251 ++- .../blaze/arrowio/util/ArrowUtils.scala | 6 +- .../blaze/plan/ConvertToNativeBase.scala | 1 - .../plan/NativeBroadcastExchangeBase.scala | 71 +- .../blaze/plan/NativeBroadcastJoinBase.scala | 105 +- .../NativeBroadcastNestedLoopJoinBase.scala | 144 -- .../blaze/plan/NativeGenerateBase.scala | 1 - .../blaze/plan/NativeParquetScanBase.scala | 2 +- .../blaze/plan/NativeParquetSinkBase.scala | 2 +- .../blaze/plan/NativeSortMergeJoinBase.scala | 36 +- 73 files changed, 4750 insertions(+), 3592 deletions(-) delete mode 100644 benchmark-results/20240202.md create mode 100644 benchmark-results/20240701-blaze300.md delete mode 100644 benchmark-results/blaze-cluster-resources-cost-comparison-20240202.png delete mode 100644 benchmark-results/blaze-query-time-comparison-20240202.png create mode 100644 benchmark-results/spark333-vs-blaze300-query-time-20240701-tpch.png create mode 100644 benchmark-results/spark333-vs-blaze300-query-time-20240701.png create mode 100644 native-engine/datafusion-ext-plans/src/broadcast_join_build_hash_map_exec.rs delete mode 100644 native-engine/datafusion-ext-plans/src/broadcast_nested_loop_join_exec.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/bhj/full_join.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/bhj/mod.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/bhj/semi_join.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/join_hash_map.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/join_utils.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/mod.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/smj/existence_join.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/smj/full_join.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/smj/mod.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/smj/semi_join.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/stream_cursor.rs create mode 100644 native-engine/datafusion-ext-plans/src/joins/test.rs delete mode 100644 spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastNestedLoopJoinExec.scala delete mode 100644 spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/joins/blaze/plan/NativeBroadcastNestedLoopJoinExec.scala delete mode 100644 spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastNestedLoopJoinBase.scala diff --git a/.github/workflows/build-ce7-releases.yml b/.github/workflows/build-ce7-releases.yml index cf8fda8f8..fb51a3e1e 100644 --- a/.github/workflows/build-ce7-releases.yml +++ b/.github/workflows/build-ce7-releases.yml @@ -12,7 +12,7 @@ jobs: strategy: matrix: sparkver: [spark303, spark333] - blazever: [2.0.9.1] + blazever: [3.0.0] steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/tpcds.yml b/.github/workflows/tpcds.yml index 98722c4ff..2b280d9d7 100644 --- a/.github/workflows/tpcds.yml +++ b/.github/workflows/tpcds.yml @@ -34,19 +34,18 @@ jobs: with: {version: "21.7"} - uses: actions-rust-lang/setup-rust-toolchain@v1 - with: {rustflags: --allow warnings -C target-cpu=native} + with: + toolchain: nightly + rustflags: --allow warnings -C target-feature=+aes + components: + cargo + rustfmt - name: Rustfmt Check uses: actions-rust-lang/rustfmt@v1 - ## - name: Rust Clippy Check - ## uses: actions-rs/clippy-check@v1 - ## with: - ## token: ${{ secrets.GITHUB_TOKEN }} - ## args: --all-features - - name: Cargo test - run: cargo test --workspace --all-features + run: cargo +nightly test --workspace --all-features - name: Build Spark303 run: mvn package -Ppre -Pspark303 diff --git a/Cargo.lock b/Cargo.lock index c5f0f8587..89905762b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -97,7 +97,7 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-arith", "arrow-array", @@ -117,7 +117,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-array", "arrow-buffer", @@ -131,7 +131,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "ahash", "arrow-buffer", @@ -147,7 +147,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "bytes", "half", @@ -157,7 +157,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-array", "arrow-buffer", @@ -175,7 +175,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-array", "arrow-buffer", @@ -193,7 +193,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-buffer", "arrow-schema", @@ -204,7 +204,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-array", "arrow-buffer", @@ -218,7 +218,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-array", "arrow-buffer", @@ -237,7 +237,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-array", "arrow-buffer", @@ -251,7 +251,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "ahash", "arrow-array", @@ -265,7 +265,7 @@ dependencies = [ [[package]] name = "arrow-schema" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "bitflags 2.5.0", "serde", @@ -274,7 +274,7 @@ dependencies = [ [[package]] name = "arrow-select" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "ahash", "arrow-array", @@ -287,7 +287,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "arrow-array", "arrow-buffer", @@ -751,7 +751,7 @@ dependencies = [ [[package]] name = "datafusion" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "ahash", "arrow", @@ -800,7 +800,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "ahash", "arrow", @@ -819,7 +819,7 @@ dependencies = [ [[package]] name = "datafusion-execution" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "arrow", "chrono", @@ -839,7 +839,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "ahash", "arrow", @@ -923,6 +923,7 @@ dependencies = [ "arrow", "async-trait", "base64 0.22.1", + "bitvec", "blaze-jni-bridge", "byteorder", "bytes", @@ -957,7 +958,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "arrow", "base64 0.21.7", @@ -971,7 +972,7 @@ dependencies = [ [[package]] name = "datafusion-functions-array" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "arrow", "datafusion-common", @@ -984,7 +985,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "arrow", "async-trait", @@ -1001,7 +1002,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "ahash", "arrow", @@ -1036,7 +1037,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "ahash", "arrow", @@ -1066,7 +1067,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "36.0.0" -source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=71433f743b2c399ea1728531b0e56fd7c6ef5282#71433f743b2c399ea1728531b0e56fd7c6ef5282" +source = "git+https://github.com/blaze-init/arrow-datafusion.git?rev=17b1ad3c7432391b94dd54e48a60db6d5712a7ef#17b1ad3c7432391b94dd54e48a60db6d5712a7ef" dependencies = [ "arrow", "arrow-schema", @@ -1866,7 +1867,7 @@ dependencies = [ [[package]] name = "parquet" version = "50.0.0" -source = "git+https://github.com/blaze-init/arrow-rs.git?rev=2c39d9a251f7e3f8f15312bdd0c38759e465e8bc#2c39d9a251f7e3f8f15312bdd0c38759e465e8bc" +source = "git+https://github.com/blaze-init/arrow-rs.git?rev=7471d70f7ae6edd5d4da82b7d966a8ede720e499#7471d70f7ae6edd5d4da82b7d966a8ede720e499" dependencies = [ "ahash", "arrow-array", diff --git a/Cargo.toml b/Cargo.toml index 5052eab71..ad86c08ef 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -64,26 +64,26 @@ serde_json = { version = "1.0.96" } [patch.crates-io] # datafusion: branch=v36-blaze -datafusion = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "71433f743b2c399ea1728531b0e56fd7c6ef5282"} -datafusion-common = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "71433f743b2c399ea1728531b0e56fd7c6ef5282"} -datafusion-expr = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "71433f743b2c399ea1728531b0e56fd7c6ef5282"} -datafusion-execution = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "71433f743b2c399ea1728531b0e56fd7c6ef5282"} -datafusion-optimizer = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "71433f743b2c399ea1728531b0e56fd7c6ef5282"} -datafusion-physical-expr = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "71433f743b2c399ea1728531b0e56fd7c6ef5282"} +datafusion = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "17b1ad3c7432391b94dd54e48a60db6d5712a7ef"} +datafusion-common = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "17b1ad3c7432391b94dd54e48a60db6d5712a7ef"} +datafusion-expr = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "17b1ad3c7432391b94dd54e48a60db6d5712a7ef"} +datafusion-execution = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "17b1ad3c7432391b94dd54e48a60db6d5712a7ef"} +datafusion-optimizer = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "17b1ad3c7432391b94dd54e48a60db6d5712a7ef"} +datafusion-physical-expr = { git = "https://github.com/blaze-init/arrow-datafusion.git", rev = "17b1ad3c7432391b94dd54e48a60db6d5712a7ef"} # arrow: branch=v50-blaze -arrow = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-arith = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-array = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-buffer = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-cast = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-data = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-ord = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-row = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-schema = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-select = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -arrow-string = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} -parquet = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "2c39d9a251f7e3f8f15312bdd0c38759e465e8bc"} +arrow = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-arith = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-array = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-buffer = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-cast = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-data = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-ord = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-row = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-schema = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-select = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +arrow-string = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} +parquet = { git = "https://github.com/blaze-init/arrow-rs.git", rev = "7471d70f7ae6edd5d4da82b7d966a8ede720e499"} # serde_json: branch=v1.0.96-blaze serde_json = { git = "https://github.com/blaze-init/json", branch = "v1.0.96-blaze" } diff --git a/README.md b/README.md index 672528fa2..eeb171e13 100644 --- a/README.md +++ b/README.md @@ -73,7 +73,7 @@ Blaze._ ```shell SHIM=spark333 # or spark303 -MODE=release # or dev +MODE=release # or pre mvn package -P"${SHIM}" -P"${MODE}" ``` @@ -94,11 +94,16 @@ This section describes how to submit and configure a Spark Job with Blaze suppor 1. move blaze jar package to spark client classpath (normally `spark-xx.xx.xx/jars/`). 2. add the follow confs to spark configuration in `spark-xx.xx.xx/conf/spark-default.conf`: + ```properties +spark.blaze.enable true spark.sql.extensions org.apache.spark.sql.blaze.BlazeSparkSessionExtension spark.shuffle.manager org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager +spark.memory.offHeap.enabled false -# other blaze confs defined in spark-extension/src/main/java/org/apache/spark/sql/blaze/BlazeConf.java +# suggested executor memory configuration +spark.executor.memory 4g +spark.executor.memoryOverhead 4096 ``` 3. submit a query with spark-sql, or other tools like spark-thriftserver: @@ -108,16 +113,15 @@ spark-sql -f tpcds/q01.sql ## Performance -Check [Benchmark Results](./benchmark-results/20240202.md) with the latest date for the performance -comparison with vanilla Spark on TPC-DS 1TB dataset. The benchmark result shows that Blaze saved -~55% query time and ~60% cluster resources in average. ~6x performance achieved for the best case (q06). +Check [Benchmark Results](./benchmark-results/20240701-blaze300.md) with the latest date for the performance +comparison with vanilla Spark 3.3.3. The benchmark result shows that Blaze save about 50% time on TPC-DS/TPC-H 1TB datasets. Stay tuned and join us for more upcoming thrilling numbers. -Query time: -![20240202-query-time](./benchmark-results/blaze-query-time-comparison-20240202.png) +TPC-DS Query time: +![20240701-query-time-tpcds](./benchmark-results/spark333-vs-blaze300-query-time-20240701.png) -Cluster resources: -![20240202-resources](./benchmark-results/blaze-cluster-resources-cost-comparison-20240202.png) +TPC-H Query time: +![20240701-query-time-tpch](./benchmark-results/spark333-vs-blaze300-query-time-20240701-tpch.png) We also encourage you to benchmark Blaze and share the results with us. 🤗 diff --git a/RELEASES.md b/RELEASES.md index 795eeb7d9..551e0e674 100644 --- a/RELEASES.md +++ b/RELEASES.md @@ -1,12 +1,15 @@ -# blaze-v2.0.9.1 +# blaze-v3.0.0 ## Features -* Supports failing-back nondeterministic expressions. -* Supports "$[].xxx" jsonpath syntax in get_json_object(). +* Supports using spark.io.compression.codec for shuffle/broadcast compression +* Supports date type casting +* Refactor join implementations to support existence joins and BHJ building hash map on driver side ## Performance -* Supports adaptive batch size in ParquetScan, improving vectorized reading performance. -* Supports directly spill to disk file when on-heap memory is full. +* Fixed performance issues when running on spark3 with default configurations +* Use cached parquet metadata +* Refactor native broadcast to avoid duplicated broadcast jobs +* Supports spark333 batch shuffle reading ## Bugfix -* Fix incorrect parquet rowgroup pruning with files containing deprecated min/max values. +* Fix in_list conversion in from_proto.rs diff --git a/benchmark-results/20240202.md b/benchmark-results/20240202.md deleted file mode 100644 index 98771aea1..000000000 --- a/benchmark-results/20240202.md +++ /dev/null @@ -1,152 +0,0 @@ - -# Report 2024-02-02 - -### Versions -- Blaze version: [2.0.8](https://github.com/blaze-init/blaze/tree/v2.0.8) -- Vanilla spark version: spark-3.3.3 - -### Environment -Hadoop 2.6.0 cluster mode running on 4 nodes, See [Kwai server conf](./kwai1-hardware-conf.md). - -### Configuration - -- Blaze -```properties -spark.executor.memory 5g -spark.executor.memoryOverhead 3072 -spark.blaze.memoryFraction 0.7 -spark.blaze.enable.caseconvert.functions true -spark.blaze.enable.smjInequalityJoin false -spark.blaze.enable.bhjFallbacksToSmj false -``` - -- Vanilla Spark -```properties -spark.executor.memory 6g -spark.executor.memoryOverhead 2048 -``` - -- Common configurations -```properties -spark.speculation false -spark.sql.adaptive.coalescePartitions.initialPartitionNum 1000 -spark.sql.adaptive.coalescePartitions.minPartitionNum 20 -spark.sql.adaptive.coalescePartitions.minPartitionSize 65536 -spark.sql.files.maxPartitionBytes 268435456 -spark.sql.autoBroadcastJoinThreshold 20971520 -``` - -### Results -Query time comparison (seconds): -![blaze-query-time-comparison-20240202.png](blaze-query-time-comparison-20240202.png) - -Executor time comparison (Memory Bytes * Seconds): -![blaze-cluster-resources-cost-comparison-20240202.png](blaze-cluster-resources-cost-comparison-20240202.png) - -| | Memcost Spark | Memcost Blaze | Blaze / non-Blaze | | Query time Spark | Query time Blaze | Blaze / non-Blaze | -| ---- | ------------- | ------------- | ----------------- | ---- | ---------------- | ---------------- | ----------------- | -| q01 | 1064427008 | 449834240 | 0.422606939 | q01 | 10.6 | 6.6 | 0.622641509 | -| q02 | 2368745984 | 2721906176 | 1.149091627 | q02 | 13 | 49.4 | 3.8 | -| q03 | 2393231360 | 1122979200 | 0.469231358 | q03 | 12 | 7.4 | 0.616666667 | -| q04 | 70389727232 | 24298207232 | 0.34519536 | q04 | 292.8 | 105.3 | 0.359631148 | -| q05 | 14231480320 | 4769662976 | 0.33514876 | q05 | 58.5 | 22.4 | 0.382905983 | -| q06 | 14682045440 | 2202324480 | 0.150001203 | q06 | 72 | 11.9 | 0.165277778 | -| q07 | 5505946624 | 3115785216 | 0.565894555 | q07 | 27.3 | 17.6 | 0.644688645 | -| q08 | 2245357056 | 1041340864 | 0.463775176 | q08 | 12.3 | 7.5 | 0.609756098 | -| q09 | 12896677888 | 6773364224 | 0.525202248 | q09 | 53.6 | 28.8 | 0.537313433 | -| q10 | 3314154496 | 975653056 | 0.294389733 | q10 | 22.8 | 15.5 | 0.679824561 | -| q11 | 11654519808 | 12878838784 | 1.105051001 | q11 | 53.4 | 58.8 | 1.101123596 | -| q12 | 686692864 | 502959360 | 0.732437144 | q12 | 5 | 4.8 | 0.96 | -| q13 | 7461695488 | 3379008256 | 0.452847247 | q13 | 37.2 | 17.2 | 0.462365591 | -| q14a | 51038400512 | 18640779264 | 0.365230475 | q14a | 323 | 182 | 0.563467492 | -| q14b | 37148839936 | 16267428864 | 0.437898704 | q14b | 259.7 | 177.7 | 0.684251059 | -| q15 | 1749106432 | 1486633472 | 0.849938829 | q15 | 9.5 | 8.9 | 0.936842105 | -| q16 | 27584167936 | 13769887744 | 0.499195327 | q16 | 123.3 | 65.1 | 0.527980535 | -| q17 | 58074173440 | 14808196096 | 0.254987634 | q17 | 257.3 | 67.6 | 0.262728333 | -| q18 | 3536184832 | 2220624640 | 0.627971881 | q18 | 21.8 | 14.2 | 0.651376147 | -| q19 | 3174092032 | 1489778944 | 0.469355938 | q19 | 24.1 | 9.4 | 0.390041494 | -| q20 | 1256781568 | 897696896 | 0.714282353 | q20 | 8 | 5.6 | 0.7 | -| q21 | 8047820 | 2254438 | 0.280130271 | q21 | 5.4 | 2.1 | 0.388888889 | -| q22 | 12014388 | 5016781 | 0.417564424 | q22 | 7.4 | 3.2 | 0.432432432 | -| q23a | 81502380032 | 31732547584 | 0.389345042 | q23a | 357.9 | 149.3 | 0.41715563 | -| q23b | 80083304448 | 30878326784 | 0.385577581 | q23b | 351.1 | 136.8 | 0.389632583 | -| q24a | 49402916864 | 16241479680 | 0.328755481 | q24a | 215.1 | 79.8 | 0.370990237 | -| q24b | 49900470272 | 16226333696 | 0.325173963 | q24b | 215.9 | 79.4 | 0.367762853 | -| q25 | 78905024512 | 18997395456 | 0.24076281 | q25 | 352.4 | 84.6 | 0.240068104 | -| q26 | 2931277056 | 1529305728 | 0.521719953 | q26 | 16.5 | 10.9 | 0.660606061 | -| q27 | 5451323904 | 2849344768 | 0.522688583 | q27 | 28.1 | 16.4 | 0.583629893 | -| q28 | 21703208960 | 11386787840 | 0.524659181 | q28 | 88.8 | 49 | 0.551801802 | -| q29 | 54227959808 | 16019457024 | 0.295409547 | q29 | 242 | 74.6 | 0.308264463 | -| q30 | 302891104 | 135692304 | 0.447990391 | q30 | 8.6 | 5.4 | 0.627906977 | -| q31 | 9615978496 | 4488949248 | 0.466821889 | q31 | 41.2 | 23.3 | 0.565533981 | -| q32 | 2419902464 | 958353984 | 0.396030005 | q32 | 14.6 | 6.6 | 0.452054795 | -| q33 | 4866960896 | 2139079296 | 0.43951027 | q33 | 22.6 | 13.7 | 0.60619469 | -| q34 | 2799148544 | 1131107328 | 0.404089783 | q34 | 15.4 | 11.5 | 0.746753247 | -| q35 | 5119056896 | 1135299840 | 0.221779102 | q35 | 42.5 | 16.1 | 0.378823529 | -| q36 | 4177485312 | 2434962944 | 0.582877679 | q36 | 21.4 | 13.8 | 0.644859813 | -| q37 | 7496896512 | 932590400 | 0.124396862 | q37 | 49.1 | 21 | 0.427698574 | -| q38 | 4062855168 | 2151781376 | 0.529622959 | q38 | 30.4 | 17 | 0.559210526 | -| q39a | 17670146 | 6114509 | 0.346036133 | q39a | 5.5 | 2.5 | 0.454545455 | -| q39b | 15856440 | 6545408 | 0.412791774 | q39b | 5.3 | 2.5 | 0.471698113 | -| q40 | 15339921408 | 5915641344 | 0.385637005 | q40 | 75.2 | 31.3 | 0.416223404 | -| q41 | 429260 | 339148 | 0.790075945 | q41 | 0.7 | 0.8 | 1.142857143 | -| q42 | 2318457088 | 973158720 | 0.419744116 | q42 | 11.8 | 5.7 | 0.483050847 | -| q43 | 2643484416 | 1623958784 | 0.614325083 | q43 | 32.7 | 8.4 | 0.256880734 | -| q44 | 4751142912 | 2148898048 | 0.452290762 | q44 | 23.4 | 45.1 | 1.927350427 | -| q45 | 859250816 | 774846720 | 0.90177013 | q45 | 5.7 | 5.4 | 0.947368421 | -| q46 | 5104092160 | 2574011392 | 0.504303471 | q46 | 25.9 | 16.4 | 0.633204633 | -| q47 | 4227521024 | 3525534464 | 0.833948417 | q47 | 22.9 | 21.8 | 0.951965066 | -| q48 | 5522035200 | 2340343040 | 0.423818928 | q48 | 27.8 | 12.3 | 0.442446043 | -| q49 | 18826491904 | 7428071936 | 0.394554226 | q49 | 85.9 | 34.6 | 0.402793946 | -| q50 | 40762097664 | 7524097536 | 0.184585631 | q50 | 177.7 | 35.4 | 0.199212155 | -| q51 | 6946065920 | 2681179904 | 0.385999778 | q51 | 42.2 | 16.5 | 0.390995261 | -| q52 | 2423949056 | 989473792 | 0.408207338 | q52 | 13.3 | 6.3 | 0.473684211 | -| q53 | 2978816512 | 1240832768 | 0.416552266 | q53 | 14.8 | 8.2 | 0.554054054 | -| q54 | 18391373824 | 4565063680 | 0.248217655 | q54 | 111.6 | 22.2 | 0.198924731 | -| q55 | 3162129152 | 972509952 | 0.307549093 | q55 | 105.5 | 6.9 | 0.065402844 | -| q56 | 5093037568 | 2148585216 | 0.421867145 | q56 | 23.8 | 13.4 | 0.56302521 | -| q57 | 1928995200 | 1695657856 | 0.879036846 | q57 | 11.3 | 12.6 | 1.115044248 | -| q58 | 4623944704 | 3115723520 | 0.673823698 | q58 | 22.1 | 17.3 | 0.78280543 | -| q59 | 3398420992 | 3844507648 | 1.131262918 | q59 | 16.7 | 20.4 | 1.221556886 | -| q60 | 5130304512 | 2142691072 | 0.41765378 | q60 | 23.9 | 13.9 | 0.581589958 | -| q61 | 7130669568 | 3692626944 | 0.517851361 | q61 | 31.7 | 17.4 | 0.548895899 | -| q62 | 861110208 | 734616064 | 0.853103421 | q62 | 5.2 | 5.7 | 1.096153846 | -| q63 | 2913974784 | 1220212608 | 0.418745081 | q63 | 14.3 | 8.3 | 0.58041958 | -| q64 | 90821681152 | 45009911808 | 0.495585539 | q64 | 393 | 217.5 | 0.553435115 | -| q65 | 6929706496 | 4274123520 | 0.616782763 | q65 | 31 | 21 | 0.677419355 | -| q66 | 4168856064 | 2715433216 | 0.65136171 | q66 | 43 | 14.1 | 0.327906977 | -| q67 | 16813858816 | 11755253760 | 0.699140744 | q67 | 80.9 | 55.6 | 0.687268232 | -| q68 | 6089261056 | 3126788096 | 0.513492207 | q68 | 30.5 | 17.9 | 0.586885246 | -| q69 | 3271612928 | 955816128 | 0.292154405 | q69 | 20.3 | 10.7 | 0.527093596 | -| q70 | 5136711680 | 2733509120 | 0.532151557 | q70 | 24.8 | 14.6 | 0.588709677 | -| q71 | 5308280832 | 2298735360 | 0.433047051 | q71 | 24.9 | 17.5 | 0.702811245 | -| q72 | 34922856448 | 25395292160 | 0.72718256 | q72 | 174.4 | 131 | 0.751146789 | -| q73 | 2774287872 | 1024954944 | 0.369447942 | q73 | 14.1 | 9.1 | 0.645390071 | -| q74 | 8379421696 | 6975468544 | 0.832452262 | q74 | 38.8 | 33.8 | 0.871134021 | -| q75 | 21823078400 | 10669447168 | 0.488906605 | q75 | 98.5 | 57.9 | 0.587817259 | -| q76 | 5128589824 | 1943261056 | 0.378907482 | q76 | 36.8 | 11.7 | 0.317934783 | -| q77 | 6024270336 | 2879578880 | 0.477996292 | q77 | 33.3 | 15 | 0.45045045 | -| q78 | 90786725888 | 36193472512 | 0.398664807 | q78 | 377 | 154.4 | 0.409549072 | -| q79 | 5194788864 | 2371766016 | 0.456566393 | q79 | 25.4 | 14.4 | 0.566929134 | -| q80 | 86372556800 | 32238481408 | 0.373249127 | q80 | 351.9 | 133.5 | 0.379369139 | -| q81 | 537159104 | 260399136 | 0.484770963 | q81 | 9.4 | 6.2 | 0.659574468 | -| q82 | 14299611136 | 1773495680 | 0.12402405 | q82 | 83.6 | 22.9 | 0.273923445 | -| q83 | 370437344 | 190986624 | 0.515570655 | q83 | 13.5 | 8.7 | 0.644444444 | -| q84 | 719424896 | 87703536 | 0.121907841 | q84 | 12.9 | 2.7 | 0.209302326 | -| q85 | 2387102720 | 839722560 | 0.351774791 | q85 | 28.9 | 11.6 | 0.401384083 | -| q86 | 810701824 | 355365728 | 0.438343319 | q86 | 35.8 | 4.7 | 0.131284916 | -| q87 | 4160053504 | 2180360192 | 0.524118305 | q87 | 30.6 | 17 | 0.555555556 | -| q88 | 14849074176 | 5351185920 | 0.360371688 | q88 | 61.4 | 24.8 | 0.403908795 | -| q89 | 3119070464 | 1416152448 | 0.454030284 | q89 | 15.7 | 10.2 | 0.649681529 | -| q90 | 947142592 | 274404096 | 0.28971783 | q90 | 7.5 | 2.7 | 0.36 | -| q91 | 140899136 | 61533388 | 0.436719413 | q91 | 5.3 | 2.9 | 0.547169811 | -| q92 | 1332381184 | 475557024 | 0.356922651 | q92 | 9.2 | 4.1 | 0.445652174 | -| q93 | 45288034304 | 12807971840 | 0.282811388 | q93 | 200.3 | 61.8 | 0.308537194 | -| q94 | 16076772352 | 6874429440 | 0.427600098 | q94 | 82.9 | 36.3 | 0.43787696 | -| q95 | 23590234112 | 17770180608 | 0.753285471 | q95 | 120.8 | 103.8 | 0.859271523 | -| q96 | 1928228480 | 637357376 | 0.330540381 | q96 | 9.8 | 4.1 | 0.418367347 | -| q97 | 10208055296 | 3761395456 | 0.368473264 | q97 | 56.9 | 26.3 | 0.462214411 | -| q98 | 2648099584 | 1812735104 | 0.684541894 | q98 | 15.4 | 11.8 | 0.766233766 | -| q99 | 1629526400 | 1538929536 | 0.944402948 | q99 | 8.7 | 9 | 1.034482759 | -| | | | | | | | | -| sum | 1.50781E+12 | 6.1011E+11 | | | 7367.7 | 3390.8 | | diff --git a/benchmark-results/20240701-blaze300.md b/benchmark-results/20240701-blaze300.md new file mode 100644 index 000000000..1106d5451 --- /dev/null +++ b/benchmark-results/20240701-blaze300.md @@ -0,0 +1,201 @@ + +# Report 2024-07-01 + +### Versions +- Blaze version: [3.0.0](https://github.com/blaze-init/blaze/tree/v3.0.0) +- Vanilla spark version: spark-3.3.3 opensource version + +### Environment +Hadoop 2.10.2 cluster mode running on 7 nodes, See [Kwai server conf](./kwai1-hardware-conf.md). + +### Configuration + +- Blaze +```properties +spark.blaze.enable true +spark.sql.extensions org.apache.spark.sql.blaze.BlazeSparkSessionExtension +spark.shuffle.manager org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager +spark.memory.offHeap.enabled false + +# suggested executor memory configuration +spark.executor.memory 4g +spark.executor.memoryOverhead 4096 +``` + +- Vanilla Spark +```properties +spark.executor.memory 6g +spark.executor.memoryOverhead 2048 +``` + +- Common configurations +```properties +spark.master yarn +spark.yarn.stagingDir.list hdfs://blaze-test/home/spark/user/ + +spark.eventLog.enabled true +spark.eventLog.dir hdfs://blaze-test/home/yarn/spark-eventlog +spark.history.fs.logDirectory hdfs://blaze-test/home/yarn/spark-eventlog + +spark.externalBlockStore.url.list hdfs://blaze-test/home/platform +spark.driver.extraJavaOptions -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/media/disk1/spark/ -Djava.io.tmpdir=/media/disk1/tmp -Dlog4j2.formatMsgNoLookups=true +spark.local.dir /media/disk1/spark/localdir + +spark.shuffle.service.enabled true +spark.shuffle.service.port 7337 + +spark.driver.memory 20g +spark.driver.memoryOverhead 4096 + +spark.executor.instances 10000 +spark.dynamicallocation.maxExecutors 10000 +spark.executor.cores 5 + +spark.io.compression.codec zstd + +# spark3.3+ disable char/varchar padding +spark.sql.readSideCharPadding false +``` + +### TPC-DS Results +Blaze saved 46% total query time comparing to spark, benchmarks using the above configuration. +Query time comparison (seconds): +![spark333-vs-blaze300-query-time-20240701.png](spark333-vs-blaze300-query-time-20240701.png) + +| | Blaze | Spark | Speedup(x) | +| ------ | -------- | -------- | ---------- | +| q1 | 8.946 | 15.073 | 1.68 | +| q2 | 14.558 | 10.482 | 0.72 | +| q3 | 7.892 | 8.239 | 1.04 | +| q4 | 192.499 | 357.818 | 1.86 | +| q5 | 15.943 | 37.471 | 2.35 | +| q6 | 20.139 | 41.034 | 2.04 | +| q7 | 11.274 | 16.532 | 1.47 | +| q8 | 5.922 | 8.885 | 1.50 | +| q9 | 16.797 | 18.52 | 1.10 | +| q10 | 8.908 | 15.634 | 1.76 | +| q11 | 114.716 | 108.502 | 0.95 | +| q12 | 5.923 | 5.642 | 0.95 | +| q13 | 13.641 | 22.386 | 1.64 | +| q14a | 74.169 | 162.795 | 2.19 | +| q14b | 71.721 | 187.395 | 2.61 | +| q15 | 27.634 | 57.961 | 2.10 | +| q16 | 51.884 | 61.577 | 1.19 | +| q17 | 51.08 | 146.804 | 2.87 | +| q18 | 10.858 | 33.96 | 3.13 | +| q19 | 7.726 | 13.939 | 1.80 | +| q20 | 7.926 | 8.858 | 1.12 | +| q21 | 2.456 | 4.68 | 1.91 | +| q22 | 13.092 | 21.736 | 1.66 | +| q23a | 170.957 | 418.987 | 2.45 | +| q23b | 236.542 | 528.576 | 2.23 | +| q24a | 68.542 | 164.819 | 2.40 | +| q24b | 69.916 | 156.236 | 2.23 | +| q25 | 72.066 | 166.081 | 2.30 | +| q26 | 6.899 | 10.715 | 1.55 | +| q27 | 10.116 | 15.436 | 1.53 | +| q28 | 22.418 | 33.152 | 1.48 | +| q29 | 60.618 | 146.133 | 2.41 | +| q30 | | | #DIV/0! | +| q31 | 17.947 | 34.14 | 1.90 | +| q32 | 1.13 | 1.207 | 1.07 | +| q33 | 13.376 | 14.912 | 1.11 | +| q34 | 8.123 | 13.009 | 1.60 | +| q35 | 9.667 | 23.604 | 2.44 | +| q36 | 11.766 | 14.016 | 1.19 | +| q37 | 6.912 | 15.854 | 2.29 | +| q38 | 14.037 | 22.247 | 1.58 | +| q39a | 23.506 | 14.385 | 0.61 | +| q39b | 15.658 | 14.812 | 0.95 | +| q40 | 21.323 | 53.26 | 2.50 | +| q41 | 1.636 | 4.159 | 2.54 | +| q42 | 4.386 | 9.169 | 2.09 | +| q43 | 6.184 | 7.436 | 1.20 | +| q44 | 7.177 | 14.616 | 2.04 | +| q45 | 13.453 | 42.177 | 3.14 | +| q46 | 11.486 | 19.182 | 1.67 | +| q47 | 30.546 | 22.316 | 0.73 | +| q48 | 14.617 | 21.361 | 1.46 | +| q49 | 23.28 | 50.818 | 2.18 | +| q50 | 31.91 | 78.861 | 2.47 | +| q51 | 14.767 | 21.594 | 1.46 | +| q52 | 4.402 | 7.476 | 1.70 | +| q53 | 5.893 | 8.728 | 1.48 | +| q54 | 25.024 | 50.243 | 2.01 | +| q55 | 4.607 | 6.456 | 1.40 | +| q56 | 12.508 | 15.227 | 1.22 | +| q57 | 13.066 | 11.239 | 0.86 | +| q58 | 19.706 | 13.923 | 0.71 | +| q59 | 54.67 | 22.584 | 0.41 | +| q60 | 12.192 | 14.538 | 1.19 | +| q61 | 17.277 | 24.019 | 1.39 | +| q62 | 6.786 | 4.75 | 0.70 | +| q63 | 5.849 | 8.618 | 1.47 | +| q64 | 121.374 | 292.479 | 2.41 | +| q65 | 22.164 | 35.938 | 1.62 | +| q66 | 11.608 | 11.713 | 1.01 | +| q67 | 250.606 | 617.991 | 2.47 | +| q68 | 11.785 | 20.809 | 1.77 | +| q69 | 8.581 | 15.859 | 1.85 | +| q70 | 13.819 | 13.642 | 0.99 | +| q71 | 11.371 | 15.644 | 1.38 | +| q72 | 260.602 | 331.28 | 1.27 | +| q73 | 6.464 | 10.715 | 1.66 | +| q74 | 75.604 | 99.419 | 1.31 | +| q75 | 32.963 | 47.765 | 1.45 | +| q76 | 8.876 | 14.479 | 1.63 | +| q77 | 10.093 | 11.927 | 1.18 | +| q78 | 115.239 | 208.522 | 1.81 | +| q79 | 10.867 | 17.617 | 1.62 | +| q80 | 98.788 | 179.521 | 1.82 | +| q81 | 5.629 | 9.121 | 1.62 | +| q82 | 10.638 | 21.232 | 2.00 | +| q83 | 4.092 | 4.624 | 1.13 | +| q84 | 5.152 | 4.547 | 0.88 | +| q85 | 21.524 | 12.985 | 0.60 | +| q86 | 5.256 | 6.288 | 1.20 | +| q87 | 13.981 | 31.153 | 2.23 | +| q88 | 24.139 | 29.653 | 1.23 | +| q89 | 8.124 | 10.193 | 1.25 | +| q90 | 3.507 | 3.555 | 1.01 | +| q91 | 3.121 | 3.695 | 1.18 | +| q92 | 3.525 | 5.06 | 1.44 | +| q93 | 37.673 | 106.785 | 2.83 | +| q94 | 27.961 | 51.743 | 1.85 | +| q95 | 60.55 | 87.68 | 1.45 | +| q96 | 4.003 | 11.587 | 2.89 | +| q97 | 16.22 | 36.13 | 2.23 | +| q98 | 16.228 | 17.334 | 1.07 | +| q99 | 8.363 | 6.842 | 0.82 | +| total: | 3309.135 | 6112.521 | 1.85 | + +### TPC-H Results +Blaze saved 55% total query time comparing to spark, benchmarks using the above configuration. +Query time comparison (seconds): +![spark333-vs-blaze300-query-time-20240701-tpch.png](spark333-vs-blaze300-query-time-20240701-tpch.png) + +| | Blaze | Spark | Speedup(x) | +| ------ | ------- | -------- | ---------- | +| q01 | 18.436 | 38.834 | 2.11 | +| q02 | 19.276 | 34.415 | 1.79 | +| q03 | 38.373 | 85.78 | 2.24 | +| q04 | 22.427 | 69.202 | 3.09 | +| q05 | 68.087 | 126.88 | 1.86 | +| q06 | 8.945 | 25.513 | 2.85 | +| q07 | 87.404 | 206.581 | 2.36 | +| q08 | 79.142 | 164.408 | 2.08 | +| q09 | 107.604 | 237.855 | 2.21 | +| q10 | 26.112 | 98.856 | 3.79 | +| q11 | 15.511 | 59.842 | 3.86 | +| q12 | 20.874 | 52.742 | 2.53 | +| q13 | 21.404 | 72.36 | 3.38 | +| q14 | 11.752 | 33.146 | 2.82 | +| q15 | 20.925 | 62.572 | 2.99 | +| q16 | 9.378 | 24.72 | 2.64 | +| q17 | 70.05 | 137.646 | 1.96 | +| q18 | 153.348 | 215.998 | 1.41 | +| q19 | 11.616 | 24.857 | 2.14 | +| q20 | 22.572 | 80.441 | 3.56 | +| q21 | 132.751 | 291.799 | 2.20 | +| q22 | 11.561 | 32.92 | 2.85 | +| total: | 977.548 | 2177.367 | 2.23 | diff --git a/benchmark-results/blaze-cluster-resources-cost-comparison-20240202.png b/benchmark-results/blaze-cluster-resources-cost-comparison-20240202.png deleted file mode 100644 index d88fb04cb2ad309c7e6c099801100b87b37676ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 105216 zcmeFZcUV(d`!1}A3W|UzB27d=lrB|@fS`iX97rgkgMjqjn<#=K0;1BJD8(dn2pvKh zMCpnFq(!6z2oPE*p`Eqoedj$#X6F0N`R`oU_s(@O#IUoo_gZ^B&;8u@{j6W_X{nw# z#&+z$fdeP*sNK{#aNyX<0|yR?(i{Qr@C^34f?o$cbX1iO6m)USfPeT}8{Dzc)I7is zUeg?)KFD_9Fmwp`Q8>u{pVzk!UOsT>_xGs|9Eh+xK>hDyw7_rZzj*Kiedb@k54}0~ z@1wzcZ>at_`q;@ghyHkdNEG@Qtl`96@Qc<}&Cuh(0Tw3c=b&xI7jW512kzXw0rNdL zH$q$MR-xwsCDX}ee~gh&|i6rJ~|{dw#S=ze>>5}IObF_ zvNrqMEOwOXpPxc#u}z>*P-+BAvr8HDj6KX^T$C70nc*vcHu}#mMBz*&v7m;n4y_zE z`G}h42@SiH%|9K2he~)_9=?qqks|Rs$vGtdT$O)3dJmOUsg%PwuYP;_4~F_*)3KWI zkkMjqMEClIe>6YPhd60zPI~`#|Kk5RnSV@pPYAfG9UpD-Kff$bp7S>cPQHrz=YzHf zu4)Z`9ru56S+U^ORW+pk`JM0ZfvXBxhsFH!%lhwx{C7hB=f(Kn74rXH$m%7VC9Lj` zSPz&t2#R~(svrE8VVmmq!97FY)Gyn2sf8tCIAqW+S12dNd3A4YapINs86SL*&(c?2 zZRT!;Eyt^bRF5+&oziBT(!CnuC;S#n^EnDKX3QwxG`Ct6cYoQQcL@?yP-ab}DW;6XhrV__+MiZ>J_nx}zf=os#{DwS_BnRrBRT z&ur8Eatz!*@kTy5gLt1y`m|lHu-cMY_j9%n4L);r2RpT1 z`mFxsPl-5Gfk{nmqTvw3T(xQ|hlj?}2kH9aYYUa*&mx7Z9}JB1rp7P!+!e@&PKMq0snh5H03KdSEN!r{Wg-mT2n-iuD_dFi0S#=@1%T}Acx|` zrsFIQ`4~Ob#wO|`-PhNLe}J@lPNI?pt10y@ORdN@T4rkcRzlk%K|6T))sHXgB0{D? zWSm$Jd7NvnQhx2Vzt)e$@An1qn_WI|wmy^0`?n^kir&>w$OIEGVwA5k0KRW)b)&pI>IhDfV;@ixQipde8 z3fUg#isagvR&5!NQ5-GFxondWw6^SHw?pnHwTZ6t;4`UD6P>1#Y=uMA*9WX(F08zB zqFqmSZK7=<_nYC@7-?^T;fijNCFb>d*FD*X6qNRkfuu;P!-@Z#tM46yF%$e-Vo|r! z%3v5YKj#jmA1i#wvW8S2Fm(4mvfOiHEP|zC)Sp-svD=%m3!4zJD5d7F&OdT(fHs0? zYmDAC@fq=&v|!pD@aVoCdFzbf{i5bG`es31OZdGVyfv=zhP1!at?ghk+Kd@_Z4FZx zl%MxQ>g!r_zB;Cv6iAS30VlwD_g*2VPSnzNOpw6B;osU$>JH4IV=k||cf2)Qi9aQ1 zQo!sA_S9n&^Yj^NI+2>Wk{{q%KIeAL8w_lbLq*_^W3{Va)LVY#OE;MCaV(0r6~D11 zP*_{Cu}x4^MB%f=yRN<4Xb}niymq=W`bIB^JV1;ZsSqX@du;& z7*$av*4I9Lvq^Dk2xjft#O@NYY|Mlsj-nlpQxKRZcihAWx_m(7Kow)+l~WpudYc~vIkG!DXhvl>6DI3?70 z`4Jl1M2qkeDkq0qor-&#-H(qPM2}-q%8xT%I*<3?4r@5Y&m}xD>K{!PHOzLtgQUry zSWwc&-}30V@IYC6r!?L;)xGVCW_#dIs~Dhqijfj;({4T1U-b{n_piZwmK~2kZA?jH zzv=yGIxZp@@ikv>p_*9G0#=$ykYjR}Si7KR2ATm`4P(Ia&aHr$#)1q?;P)rD`ShCx zI9;{-l9l7ZBfr#--Nncn%b0*@6iIiyYQQow4%fzXSzSrQA{xIvB^~7?zG_we&HV8R zw)PM9IiuYyroNeTW+=k-#fC#IV4V)#_R3w|OTcbh7xyS1k85V{FC^G1ZW-=a$#kq7 zRorNFSDUcEr5wj9EX#~U6N{?^P(>*4W_M41<2#CjtG@}T81WY9GYyy*dzganpp(lWsUw0t zgt6Ppmc5$N`QEJF$^2C~(jBQ$*X9$EY2Jg#E8n*w6t^=sM%U~6X)aH#d#5;+v%+!B z79t;y{QPV`Y^XfNc!9&G;Xu)h-d^d(m&xN6j|`RDH4jES0Yk5J zEf0})z9m-moZ*qDC~I0Ky&Em9=K}K7UKEcqq10V7N5!&!yi^x9oU2kGB?Ze?m8BGZ7l56Gh!jO-pzzB%~Qv4>^<5yUj?Ldo{iDV&E3O$4s7<>!)Y77KIS1GdQSFu`NrBKM{bi6f`FQAku1)u3y=fpK`2bQ~1zn8otw&G%%H>X+H1(g!)+G3n0`NxR zjObT_AZoRM6u z@t{@>e9t$(bqAMkfY5K6)r3G$y+o>EM!bjOGIZSv zN2k(!Mh|jFQz$-lM3a^z+jPVGzWB1RS)J@6FW+{Gf=Z`I@o!8m=>ZFd9AGqC%yyR( zGNsZk5J#7yeI%aHj7ty3AB#mAB=K8nUnj#1xdokR-xgH+Vuah~S9srofH978bKwy5dVpvYisWbbfwFw)kU_Gp+c&-F5frQR%uF&B?3oCuD+a z^svz~gw#?xgPdn(@rvui9xZ_@o%(t=9Qq@?TifdjjB>bG>&sp({!%lJ5sowrSI8f{ z5GnZO^!dDaTQ;n}I_AsoQ|iU(r-&1Hm-BX->bT&&eCOmG{H9jUq~BAAN7ZolP7i~5 zxnw)}Q&L?uS1rcES}fBU5er)nh90C(p6$bvo73ciX^SyirjtAA_JMv!b7D%n6#`-j zf+2o|xXuWoiGL7>_Qu+gqHR3v79RN=Mj+={@p;?e$LP{p`lL z(s_$2fAu%~LSw%Q6jZ(~=06xER6g)_$?tsD1Dv$KI``sf3_p)XU zl(mr$;<0@PU^}vUMoyRuDu;4s4eGM}&G@*R(1rEYc{WKT1Q=J?8KU)5(Vh2$_u>wt zNx>K=zMzlgRfSeT(X+{7+mz3$TqZ$+?hI=qKKKs=`%9z@y3ZyjEK+Q*tPA6-oYR|U zH!h}Ky6^w>7gfmuzAe7MmCVKC4uA*z z>+Qq!edB@^!|p8#TXW(MB!~OvRVh_$%Gb6grES1qIR!i=WQ5e)`4QKv?X2gU0o*g? zaYSI4^$aq%)7~re=WdSbaW@He%jLXcv`Fuy7G4Q>75~N##akY5bbCfSguep0%h%|z zZ{0H{o&^En)VKnX_vc8FXi1fi`@oc=uI^P9=7cJrN_IGdtnIs_j;r2|s)$!IS3ZyH ziABDNJD`tJ;Dx;u)u!UGL(-9Pg~1p#a?1&+fj1m|6&S^K!BUM_?C0_@AtR%vi{tr| z$z7m|$Z3Bzu4WYey5Okp(0qo+4gh^-RrU!c-gXtA9&(o(N#E_cWbwH>WOqf=qCPhx zxO(GDLOtvVtiIP+EIdDdJB8Fo2I z;rZv1t+Jk3KC*_ZKSU}_xM6BXJl#{@$-1D%cxeotG&xs}g^LxXI=rudA@iuh4AW5_ z-$e5DyDj{R=y&-nO5|UW15EHj@a_7=CnB%A9m9ViWsxa{3lphir!NT?W8_QnrVPzS zWi#3eXNH2e=JoYV9Zm3^gxYtyab@-@*9&n0b%W)Yw`Q1HuVhAxx=CrvqIe~=tV71kXcQ3sZb5s4UvlgF#iHPv35MLMX{Da5=$3rXS&uFE;RtlVbXo%ohjs z2icfs`j4!U-to`ua7CRm&ip%35;uK``4v(pjwhpHK=tAr+vbZD^#w~lzA`{3Y^b4@CkfIEk&w;y=5qq1 z_(+lZ%3_mxBJ17iSAA^M8k3(J5e0mt*RugIuAIQ4Ak-v)BEhk?)u?hx8YwWBBUr9K z>7iqAjBZsKB4cN{{;cD`_-tj)J_Mz-M+0iG?z*q1ey+hpZ&5Q9pcr(MAF3(S)AI6o z;$(Ro!!tD054<0pJ@}HcjC87(Jh0KD!XWv`zFoM+F_uv~{D#wsfnm2+&Re@$vNKt? z--vQonjY=Qd2Q?P-l>SFHjv+Ikxy)W&#;v`Ipl&3Vr9y6dF7ziP+*TdS3VZbuE&*< zL)$-L+9@^U7&XayY$FA~ZFBKJ%DwMU`WE9!YZvt_>y{;rTO?=F-2LDo6VE4l2*LE9wj z_)IAgv#jIHu!t!(KgLf(Vb=v>bT*Zbv$-v1Xmr^{ihlIYpPk) z@G#TDbeU82Oit98pYCqF2^?d$vGDXHQ^?qVf?L`DW6XK}aM4cX4!PRel98(Va z<}H{$$sx@>dR(~je5rMcbM?%#p!Jc8#DIk=^eyX_p1=rH{5`iCl6^f3?SkGAz>!rK zgI!>Rvz-!|E!5UT+}84fKPag2OJgV{bBqdeR-QtImTpv$Jv{Ey?22nItUE7#CbL}g z>1tW8mH_%tH^77{PWGHXr@`|*5CI?LDem=TA7IuGg{i2Y73MBL0c7cdvT3(-NPKH& z9l-_R_X}|i5D!GZ?i*A}9p-;E%RT%=eua0lu8mq7>`R-O@_kyL3DQChsepulF*h<* zQ<_-n@6qcHmNmjwxh?9c!)Jxy0=g|?HZu+gs@D;m-Uh`W0H2`+*8>%j+0QAy@b4K}J8A5ZuAgXgRPG6>*KU2}Wtom8 zSv%U}Q=88Hh>ZTDT!BR+Hk$1pei(Mq?gD~^1*8V;lrC~{Oi-SIJu*8v9v30i{VZ&C zwy~M=0q!f_uEQ+tl$SBjvG-b6>xvG!bX076(^g?6=>on^%gA9-93PvB2nwpZJ9}^I z{s@E?E;Q%%4>73={zA{d*&H_F0+!P4Yf^B9m7Na8&w$wU2$EjezZx@+N$j;dWq9jU z>{DezKZ(3}VIF>2>pa?o7YU=FD*7xqiA32n?FYn%hv=t0S90ccg607I=!lR?X2_3A z2e=qzy#s=y?(X&Fj~81o{ibqnlr5AcX+7_sop4sxS21ZdC~RL$%!d!6JZv_;z%HX1 zERr|F$`LA3IEDOP$Aq5Yld2q_X_#zM1|~(|ZA=SydR*FtDmWO8QCuRKd5s9FFl1L; zKwO`!pila^N0&@42Yto44c0kd6(MahCYJi;Z0p?vZn3w{AVutxxrn5@^>|jDYI|Ds zV1gX}NqaNB*Z1CWRPE&L;E(@@_tj86#hO+w`|^a;8Vz%CkiO}R+50M+W*u+%4B(0CyaD~g;Y>&FPL`l3Mtng= zev@(SZHBaTm!2E?>2G~lUdQeQZB~4`l=2!!ughDCIic?-&~Jo+>Et0$KD^V0T%%0~oh>P`f2r7=W!kb*S?F2BOPU7>JRel$&3h0-bh(O_cGw;&GcCE z8L;f{8k_9$@k8GI(gP*1Iu3OF`5A_gjb^d;25A?$JHxC`y)M+hmDk50ogd7Fnh!tL zOGq;asVG}8;(X`j-@>df@kCm2dyIjBhS4U2L!{XtgwmTKj0%XuXErYA68!=%?PEA| zoWTSkw|1-i_d%7G!%X*Bg*|Z#7j&~l4;I)@kKuRMDp~-E38Q0q`l*;n0maZ+32_Ix z9~Yw`fnhvxzDsVPMC%ai+TJ=+sfP^)e=#e;U4J&JogZF z>_kFW;Ajf%RXw3*OCP59ouoVUF*7cOAxD?#&V_6*X<~({x?o%yl5L;Ms|e{ecKzGo zJZhr_Em@Sh4oY9p(EKaw{AQKH@%;XF4A`J4&a2p!bx_@SCA6=E)XN%9>W^GR@6+7 zE=+b@q4i1i{qd_NA)voz6t|ITe*5~(#1|iKvpan?z&l@35$re0%|ufabbA9G;>!HZ zZ7PF#vS9G1DQ+KF+waDx3l`su#nQ2)S?z&5AribpG?S)}vn?E1iMQgF=yRT)rnaBW z%(I^!A0Fc8NI3O--3)ZK&4bJ`Xp0x@ET7*cg{_Ej_)uA!Z4LIb#kCex&!Q~Hp0Egm zkwE8&lV@v60F5iD94T`HWleu3uYD&^Uxz3WC@<6XIc%kn!6C2swytCk%(Di_E$k^a z*9Q7r;UWNN{u-WC1z=J;qeIPXq2IaX4n3#$nK5W5|4l)Hoc`f2?%QF%tpn`L{g`lE zut>@4%A^E-3{VpoJ3vb^o9bqY2;uv~23Pvq@uAeiM2m$~f&LY)y(O*|z|SVfymQ_h zol3u-I8hds)|=r^{y6*Ht5r~a8k7_WC$GwNd*T`{0Q1V@Tll`d*qMogJB?Y{rw|jT*;^FbpTyFD-4_zxW$i!3&BxR z_`sD}D;Wv<@n#2c0K^e8fi6LY#6T)@a(u%bY&o^9aP{=toCPn0oq${#D5fTR>R9Wg zLvW%!xP#)0)S6S=>INtL>w`btC%i8x?zKx#!s(mU75gb0la+8lEf_M;?_6D$$%~d5 zmf&uLsGfNnNbu}kNhaK{>_gkikP1t?WYKQbuyflJe2Ox2Th(6g*N=j6rQPTIxZFtJnj^AP;;(G%B==WXx)oXu!7f~fdUY6Y!# zF#kN2q4g&}7R3|uO0|A@!X|c_2q}&in!9Z_0XeA?Ue7dp_a3Q3yj!pzegNXgWT@TA zeuBBIeV#z0ysRDj5P_na61hda`aUdw8wA}@=+~7V*To- zsJls`5DP^rw0i`&J2|}vGVMNor8|n?qDV3~9AVcXvtT!F#XfrgI6e1qHsQL}FF6)v z9wE}-?!R&m+A+0lJRv=N55_FEBC{@eu$1o3^*$#cZwY5)hAn?tb_`yx!3znTo|5Zt zkDqk`tfXbLf<+HRVu%hxAdW{_&yyK}F|L`mde30%+6y5Y0`4hQ|U#`DLcfmY`3yE+_+G64d*rix~+ zvMsq*Z>TI~4PaKexkRgIJK4Z+wDx(4&sc&#);lAnKSB*Q$A z|67i`)hKJe%9Z}d5i%d{5!8w*N3~=oUw^5RoKAi7BL+yl%O-HA4EvNgKTOB#ed~W( z8|2>I+Cu+*;3$VJl+$Opm`)ytHsh7oKRm62L`uHV_Y!WtW0Uy()8}aU<&V)?uNXVL zhn)0}93sl`B)+nc$L^8~rJFos6etg5G;G4#G!##z2Fzq9%+^RwovO4y@mNAn>#(>P zOY5y;+xEWA7j7i3PtN;bma(M#!3U#s4t60%dPWiX^mb!R-ZLGYup z75#RZG-j>023i&L=tqnX-{%TkZ2a-&F+G38ea!*kX8DTXh*J&f_bTn---{`!2aK67 zbd;C^s&7p#hjwyuxBv0|Z;Jh+&KuuN6(iD29n$+%=w-6#@VpK2wuNNB5kO?gQnbiF z3>AQg#!jk2G4Q3jD#v0Q2f-t4HV1NRHW8F-rs1Y&r15)Hs@g; z^thL-=ELIA<={K`a0kB-UbPgd;nK8c3AAdLhV?Pj(JL~UEyF|p?Q%i~oBi2mIsA_D zzE2#h>_rs?YmCQx+)pG&a0OQee|%eAkn=?0XZyoJsvrSTP5hch>;3`-seGP0k4`b= z7^DzPmCEmB?i- zvEfeggj_AqW8*BcC~i7&qRf|#u+6m1mkt=aMDC*PjWBs$10#RT)|>Jp@OWK8v^pZ4-K!HPl>G=>4ol9dM zdMmrsInoYUf1Ge0a_#nx#^UshPWTcggMom(vU~0O!$Z$tad{PEU9n154m8)-AAHHW zW1ZdHcVG;}Pvbct#M~Nv!nThH5WPwbyO4Tz-B5(~r|D5L3hlEH7=riN*I31{T%vxN z8FZ?=?OrskRN0Xj_cmh^@C+=JD1QT|9UVuX{Wiw*Xo4tRl#__`2=sJYvMZl4KP$L<#g7_M1w|9RSK?ySW1{f-WkV8=<%u z$^c6t-%oxPq{`uTGuGmbNRZP2VK$KbxLdN{*m_UA!^7c)Sloc+%V$}e3Qnb+Qe>0% zP3P{b-yd=cs~r%D6GF$6w#_pMj=g9?k*ryb-A-`Gv>&6es><Pfl7)y{jL{918` zKF*h8p6?H(BB9HQ0rkMHv=E$c6e(CfU^%JyVO^hv{3p;gNSzJ_O6+FrtP|wYVii?) z<;#2zJ3>rY11{j#jYzt<6a_wmxHmZ9tuon-x+R)pHWyA*O;3ZJ)L^{at)U!9%YW(o z0HQ^pwavB4ukVYP=j!ej$*hY|KkVKvWSw|<&pUo6!a>G|zlryB!io(xmabENc&T-Z zLT#n{O)%tYlwmzK?d)%Y@beK4BdJ!ER_pFNY8lPWU1<$NfPysB8__V3!p$rT16DD? zRt7G#l6Kd+VO6MGHy!;sjJalM+6J$cF`qgPVw?bE;<|vN(!EjiYF%rMxXR&I$Xk>` zng-mq!EX6fKms8-;epI|9JYj9u63{^O1R@w*yc05(-+ z+2GFKPtPA?^6xLD=qo)f3L)jE?|#UB_Q&+Wf0)M$E{%XYk&UW}$ z$NqbgMYA3(QJNj9?!Z`Lk}vQnpm zy8Q=mPl$9&0Qc-^b&@+QW1es+;_H~2&!=Lh2|8L z`d>Yao+67t%47zn>D%Ky;BL03%T5e-0mZSd(YlGqL8?;*-yyQt zG;pa!#~i#Z0HARGr6|bEU92DghB(=hGL!2vnDdH(*0&BgT%t7bi{a9z#XCe*#yzl| zWz{G8um&H>xQs^3f$f{K)O8kEQs5lGz|A`*R57drW{SB5P`&*zS(KamR+c9>2)l>V zO@3ZaN{Yvgy>Rdrz;o(DWandVkL)--e(wQ`2uo0NI@NSSpl{+Ddb)+SXrn6j|+Q3qNpBmOMtr>SgGF^cN%+i!jF_Q9b>;E z2MpVhkaQDea>m%T@y8H;*ZV*BMEx)O50E5Xmb!fEOAUa{w{@Fy1Wm9lQh>K>D{M~u z08mIs(YgGaO$(?yjPqw_JjpnH5tTh8D#;r-UE-qiyWO?5XU%_}riTqMzMhOdRszh2 z-B|SJXXmUURk!?##u8)`hFpo#_aBPtr}nh70N=(B?A{hO1@#~*W0Wq07gP`qZmXYF zSpw#YRbx?+0OjlEDivbe;SB<80I?bs1Ag$tLrBT|TsF7V$|X`zHI1qX-M#8Zj}0INiFO^NXj zdXz}!Z}1!4-Dp*erucgA&#@F;kmVFfXw#pwT<9;fJW9SjH;yvX>XtbG{c7dmUE^=U z%@-l=cix+|zgfWs0^NVC3`-@qvB zIn4Ec>ewV?;6K@v`)U4bZGsx@LbKv}37N7vGch>Z*hmTF+lux~p)-@;nk$v|W}gus z9QaK4^MIf4sel&ikP+JI=UFnJpn{g@?|xgmR!|m{8ieeSBLS>Yc~1*}LOBO6IF5cN=}2Rt{Es9awF0&v@ev zEC);=srLtPnQ(d61mn-ENE6nxM|JWSfrtCDfZU}Ao8V6gXz`9!phN5D6@OqOch)46 z$Sb}6R~ZzyUEOApxn}_HXt7jYQ3sqQ{f{5xwgDxG(95o!c&Yvj@}!7Nfr!?p5;Y6? ziOupbeTbIAkR77?9^`qx74Fm(a`w&769q=+!ugxiGW!$l-B8H2S_wq&6bS|3!ag(P zSgP4uLf2RczUvGquY6e9PgX&nj82H!|4mRs*$z&JH-dHmYr(r*O z?p+e3vshcV!zxl;>NlT(vsNYoV-%Zhdr>FkPj&;5K{DLI)F#RL88CI^6>C+F7w?fX*EKB_+8at$A z;xlzfeZ5S>RnpS9PNh1-93Nl#!EqM<`U|=QiBiPGWlhb^h*l;8QXlb+=38#NCeaTUFn>MzhJI_Z~IF;jjh#Vy=AUr;SjAlkVftpurElPA?t zd_>{QE;aMz7jSzzw62*?&hSHy+1EqS6Uj9rQiQw+np6JT0n?cg$tc1U<&1%OXnAe` zaBQ^g>IvxXkiOYw2q{LBU2ddDpeB>GL7NVs`FB8nK+mcSSBbX6Q&%2*Oy414;6&7n zbl-bIlcKiYt$zw(et(Z5nThmF{~2Ph7AiFfLJbS=9@BO>LY+X}DxExPi57h5cvlnP zvJAU~I?xL8#o7&(uvV^+z$KuazmCv0AVGF2EqFL1ROSDG`e&K8?VJpE!=-C8eY*oj zbps)qKrG8D+G`?KwcHsIybCh@SA%-U`f~Uhpb=BuueI5$b$$rnOTMFI1$tmu884&P zJuo;u0pMILIM|p;Jw1D)waC z0k_H*0_P@DI|#+$z78c3Hym7F(qxnrvbS|1bfs3;2cUzK5dtanQjZvLF1Il1%C&a1 z7yI!9f&4O^C177G7|gTt|7M-Q3~3Hl-)7N;rdf*fh{BNL<-iOn#+Y#|Ax3f6w?wq~J7y&vJ@}xW3s;YmXrzP*l7N5S{g+3d_p@C}ZLhw3{h6CU zVE6cV4B>@>J#E9kegL@AribS>h+aI#iJ7;0fg3ZS57!*rwM&ZxxIt3IIc3)0Y@>Iwd{9|NK+-!Wt&YW$l#2blJ z{Q?3Mlp`~M7MR3Ymi^N5k;&4|Vua%Gfb-n|Y2mEqBxg#pu4h0N3>YrjXQ36{%Hd}FQmmOd#vF9k3*kzrLw*#|3!2*}`F*qIKqns~w{kS+zz zn0S8v0c3y&6?duxRr>i}uK`=Ud5@Dn=0cDnuuLV>>mQcWy+hJ;@z-4fO$YJBQ(q!O z7E~P6-`9@n0mnUfu*vhNP3MJ1z@M5-hzP+MLZBWd*0$hLpTyvARJ>TgRZWG@EZiO+AZ94 zG5pOgeP|8dqE`=ZhZkwY<(AP2Byqn$mxJ)hQraP=zFCa&jGKj%4_NSfIe`X*sFnLB z%pdPu9*2|-RJ;Laz+LT{nbrsSXp3J?>0Ae$VQpjg1S076YTNkHuh%bHWW(gupYjVe zJ6@6~1EeZEMSIOo)~SQ%kG%lGg%#$@dXu|S+RTxj@dSYzt@2iIImnqV>O^+byhEBz zmQC{>>USoQ9%K~81}Dgo-|KUCC_vu#+?sf1iWaa!yKm9u2LnjR0{oJ@4|jBL6tW#r z5Gd6Qzt1IjbD(oY;`_uNq6mQMgl@-l0!%XxwZl7YbeKk5Vi7V&EX0nDqJ2!xq=I>F zIl}IE4@PPW;q->RszrM!5`~)U2x-Y)rEg#?18#hUQV52R46y}d~NHOx)34($xxS0kT=bTn-Pan z_H#p*qCwU4TUEdRK7H>9voC0lOX;(|*0MCp)P(%q#&5SVmbV2u8H~fNQ1d$3_RS}x z1qF`S7>2*qy!9S64>HT7Ycq}zW`I(<7ti}INC=#9=9oKvU9fKTr-DK^;vlQWE&lr$ z2U=_i|aBimZHwVx04O3+e8dFsSm*RRLIXnzk3_l#l7-7f`4N z0A=zmT~alGtE)V3U}^8XWE1tv+plXjq!5=ucOLF0Sc4B;% z1q)f$1{A0X!LB!qM)yuBP}=xy9L^Sc8BA@{Ph20JS?NUwyfB~Pj^NONbq|0B)?czH zR#eHXQ`d;}@{0{6ZC1EX&LHqzMcM5bY8gm0eulmN`ZIt5E3Wtaeh5cXM(<$2!1i1} z&Hq-m@i^qG90pEA&4d~NNzZTr*ufrxl(_aXe%63C2qrWZ=l`$>QQu52Cjl2gQ$4{^ zC=z4wP}dZlK+dd10rJ?)*9!K|ZS&TJa;^~rJ`H#wO;V<+;#zTx( z^|JE>wQ{JgOOfLGwC)kkYIFH<8}1+zf7d+%`3z8Ut6Aw4x(CejLF6M>R(+^_j5=V^ zXPY<_c$DY_b#W~^-yJ9(1xi;vrS)ot_GCZx!mLo{It7zEYYxA2b%@v4AS5AckM?p* z7Wv0H;Md-}Tn}PWT;o!z9JUmEEMQY1?@?gY8fb1Pd*Af+TR{SV6Sh@k!|u)f+jy8; z9(#ZD+Er4G>CXT&^irrlBfkKOAEC(}RmuQ_5aX*Op1qR%wj%&BTf>pC?zW;*NCJ*^ z8YLI_L{Zv^Di5a4BL_Vj53`hAAC>5>-NoAjSI3=)?@T)j=0k5+xxW9`3NY$ULs?Y0 z%p<7oA}{M<`hw7tb&2$oWT;;&?E-vgcD(|TWEHqIgm*@jBzoodU;jeLVJFQF%BHPV z7rv{)zADA0Rl-6G?1zQ{75@z?m02JpZRNWFyQsm1_6EJ&2{;0(o8;FWfcjS6>zt9> z_jro)+hD8XtMe9hnByAVpaFuc*@jBdJbrtVyZLxUJ&RbaAOG5QPzXe2_MD=ZpDSv9 z2Krku?~k%cL)eu2k#3ZzN=~k3vINo>ARsnxCX5* zB>hYhVge^F{@&Z!MW_%QYT?NTvmoD2?voAk3`+CcPmEa7mhG_&{c8QDVqY=WbiC|{ zicXqiZ3WB?I75L6-|e@PQRgiRWGrH~v)RnUxwPU60bNl7**9+wXZwG@D93Q-yyJCO zyD_WnqD5a=3M8Dke>S)10M3sb6+g@oBBB{i zUYYVYI%KH(r6$-Zt;1)i-*gxXE?SgpU!2^pll2O3Bg_;A0;wWE#pZ7{fCv8Pa=K!u zW!?I0!S1;`hnwQ`5Ozf>aqTrh=?J_^{tj9z{a6rL3Y19wit#J?Lv#b?uaB>R-q(Z7 z>nIiL%VUts!|uT*op=U!54xV`TKcmvNcOHW3J(Vj;%V*XmUYK0^e#=ye(8%0C<0|$ ze6tEy@WxmRXg%$fmX*Hm2E2GNoQ)A)bWqGjcJ3`WmBj`LcF^9jj_d zCQg|+)*#>CZ{lOZD!K>wlA!_T(8FggLp$tzAZXe8doeJX?t-$v!;;>hE>ap$LX$r%!F*IQ$>(do$AOi{ZlNPbr(;m=+U@eg)o42FMbsNwKF0Hb|$)ERIMiSkD zT%g-En)LGsHdLNe>$%W;#fIi%8L(7mSQ}VXHjDt*jt)J$G3lcm*vo*@dV@f0rqJt6 z&HxXq`c(t?Z**QML=DKk7YXx4SpLe}hIK0cmh+J@5&2?+Xtr1Fc47wa8f5RdJ*a`w z!x=?m0k_b{0^B6&cfIn;=r(1H&u0~n-%p%@vSLO07I+#2qz23%m)CXe1Oww98*x^J z%V(=rf_|`3BRny`0?IW#Mg3sA3nM-G_BE)qm#Ou3$WWyWdybo67qq4VM*2Sw<3D}F zYiTeQ+^t*&-NaX<3-xP>_xJ2mV5}W0vGMDlwSlSC66pW9+2tgA>HDr$k3%{=x8yPB zT)0E-=%e&2RI|b^__Y1yq)@({5UzBEOKCi4wwm)w$(hU1D!Sl)%-a zj=3;gU%kh=_Wmfzof9gZiOx~od>Mi6tW2LCh0(;wOeMZOC(+KBShG;=P}K?1KJM5p z5*DkrcvZ#B`^zkMxnl?z#_|&<6`+oG;KgdW{r>6^NXh5RO%xpRgz(|1`cFZhdcN2} zdt?~K=3@qp`qI4m43i`V-$n`#CX&|t>^R;(P67-fmX5>v{X!Y=^}K_~f{S!_zuJ}&IiPapnLvSoF(Az+ZG1o*(!=#$*Dc#1 zDqL==``vd;rs?ne(F}gZprH;wzw*_;7P;h6@4#Z>`ActhML>jO?b3%RMvGB0`M?9J z@OPX|K6ghSgIa3#OlZf}-9JfpFsihjNIwi_f8YSjB`d!hxGlR?k%OoV;P-3ssb5JG zdmWp3@zh*<&kM+cv=rqM_|EP@7j=smwR^5mvx?q4xM>kB&{2N7M{q30Z2JexE&kpU zeY5b_I2F(^N zz?e4bnNG*x2dHRiD$wmxbT~~4fUM$NVPpsV|0FJw2@8s(=jkd_49@H zRr+L<#fX9k&P^`ql6;D&4sJQ{97sk{q6YD~{DV+e?Cpo5?R^GTQ5w&#%$S%q(Cp?8 z0kiS0_evn#gcBfir~VlU?k3b^(Ce|!s)Lr|1B+D=t584Afqd@DdRl|r z2x#J5JA7ZyO^2rhNUT{-cz9RY3HiJd?46L1Suv|5V_P7Z*$$I(tn1?N01bXfDK3^? zI{gt%#cIr`Jv#px)O~!CqoXJpM1C_FYBIz?{wSzfu&0(5C<3V@|D%^a7_)3yt=wMN zR4rg8+ltKB?l2buQl-WK5MXV@Hc38L1O_&N4qs0H0urs4?0in4jB3St#SlTDzvq*W z%#hb0(}j+8EZf^{K;8&9XQnNfIVJeq7}bP7>9_aGh0~M{^GJBYdbk@ZT;9+AL}ia^!&e_HA7lq;LDyuFdhf{%eZxt8S8dkcX>L6g*f z()4BL&$0eMsKV)czZNmLpoD)Wv_n51>j11;3vjSbT7Ke9a140bcV0v4ZqL+3u1&RK zu+}R^Y>S@I-Y!e$Yi`m4xb$R)`n^Jh-`sW#SjBunCofWjzd7RfynoyA1f$JX7!T3u z4?2$W)?FcP36<#xj{8>9Wcla`T-{X^ch=}o*pm9S0gD*M(r1`({h!06!k_=FGpFgNtS%q)o^rv<)zj0ET|>1C0Bur%z2~3ZweA;ehb8} zo<83pp6mn~V5dG@$;t8=zaX1BQ(g#_tQm!#rLRct5gd~?oAmtx{k&>v{COZ17h0bk z@mCMQ15))X%LzF=;2Y)|b4yP2nT70F0^aW2DykoN>JA>Pn$s;*9;)`>>+XJCeH0|x ztYj>~>Dsc4&m07WFE01SF8zqd=bY0V?TMMgb) zHQUFci7MFgA@3sVjICk9voJY?5mf+fl=Z-nH#gRPD=d)7KQE#ePno!eoPtEb6Yrm%$wl!x;@YLYSo!qEIOppk zNgz}E8)6F1m80dE7C#lGL6Fn%P9A9R_1>y|{#8Kb-8uJ<+}5}P|EFcuIf2s4+rdz& z#I{zA08}^HVHzq$YwEBg3T9~#x%1K6&jr?43jEr|c^Wb=chF8Ass8N&9!2Z+K{b7w z&gr?7QnL(~>#eRCEA;H07zUr=#y$c<{Iy?jBxFHf=7jvCQVZ%eJIm>2AaSsx-Rqi9 zT*ZTOzw`P7&=UV5D;$fX20p@Wr4$E*C;3y5v1oesS$Mdaf61#x@O;9;xOY?*JwoayKW^`o11Wi(8LGz&{e;iRs1_c2q+)uLLC?E4 z#?-)8Z5$TaA&!Cekm0V?BKe7=C&V(<1Mz0^dbbaEX4dcup!*9GLDrR%{rY-sVfw-z z=9h|~PK)X)Y@KiY?JxMu@*PO#Hun5{Yx0ucIwvT}E(S}V@Y?=dK%DYpo0rLr-+==7 zk5d^NHc}z$maU$c55jxbhQCP zSNeWn^Dox2?4&PHM%uXBpmu8a`3SH+WWgnsC}=OvsP0pC zQMU}=XA(2aznBx!TA zb>2xS95#hy8c-2A8xI4Qr76AYsDWss4J6#YJ;rF0NC51M-6Ps*;r$vWvcWUW`GMR% z%=WKa(~t-Sx*ysAC}VBTw(nPwPSeZ3dSVM|m)m}T-AwE}{p!el0y*H+9|*rnm4fWf z0vhbs!LgMaS`lU$540%i`>5U1PZ%?E_r3S;n^d8JqOD3E#=e#&HDq^FGhunb2U|0< za&t=*XbsGpy?UK7`)=yr9+~G3q0qJo9~WDoHax#HsHZAcasQqI=YiM? z=;pB~#~XirUE>DkK&KagHu#7h7x1+E(_}3Ph<0w&N$CHn&ixaO_Mho|o(+gmgIQUr zzrOPS9QVIpoP{>68Q++y+|MBW^+iJ$DH;zxVud6Q?7!l_{Zey|<1{Cu$5Y0?{AsBF z?i>vUzy%haU10SyfAzOt0UmBQL68NaU82?hgjfIf?d}^oE6Ac1UmdOa_jkL`81(@d z4act4%LxB@zW?qV|DBZoPRc)h?*F?|{@)B=8q!q-^MHsHhBySs^ChGJ>Q8B{tydY_ zS&)H-E#qj)w~)P^NT`eLasD~?O>;t*mP$VR@|*t)XED;)yh9}ns!rV?k*<}~$9M_2 zcS<2+N%BXC2o<|o2{0#fuSRdZsAW7udBKdi5LdYD`aL3oe*EJ9M|k}CPt)q5S5KTA z1-oFiK&lSplWBpr+5&3Mp~n|~=RUzMFrw3x*n+dO!{9b%{2c!mdv6(532 zYBIO%O#D?K{`U({M!fQI#)jQ|j#I-1ScR^LHQ~_lmHup^E<5qlkOkDyzG5Bex7@U{-2Z1SzWm;)U6@cvO*Q<2d}VAAQ#;ZuAa_zd6~!bE~dH zr_$k?`AGqF+Qgp1nwGt^a9Sq~>t;(g$MgkzPdXUyJFqeo`!#)3FL#0&%X2;u%I}QM zgEw=fFw9O!umxQ@OKQ9Qd~weh2Um&lIBp?f>d9S*n(oRp7Gi&r>zz`!W+ zDd_ki7y9+LzA7a2>7>hN;FlW;fXtR$xv#iQz z^bcZ_z_D*)SXc?T$As=X4AQ-asp26m%^K4k}!VcqZypg z>siY%?mdTH3~*q4W{_5axmu(iNd3xb96z|PZZcC3J%%HuV`eM=k4esk1-Bti{x=~1 z4ao0T*8g`a+5WLs5vAp7OGoC-E6Tr!1>A`h&fNkBRn_WljFSh?&1U zIF@>WiY?YzK@0O;{|Zcz5R$sh25dt-N5yV)#5Un>`GctM0T2mwN<8NH&`sFAe4;w) zGES^W6#FP%kuN{>pXkfboQ6s^yB_KcUna@MQ0JLj7h-7dym>cU5uPp zb(pxPZfwS(+{u^h#Ox2ISrs#r(_5?GW?M~k0uO8W#>}pFl6S^{40i9LA5RhR!AOxB z2!Tm<^V6~!H#M@sA|?U7yyU~Fex|Mhl*qA`Zp0Wj=>ATJNJs4D4w@ypT(AWdH8C+k zq&=K^4ex@fd) z!hyOtgzsxTn&@}_pVb?AB*FV(M5aE)txgqvNW9-Z0?8Y;# z7Bd1;*4J)YJB~x0CvPEntEv9lBHiJ(908l`vBAb9crMvwviH3)-1-zhWDrv;_vJPQ zKU9*+gcHA>UR44Pr^Rz9Vs`SMzoc|?c~IM+eW^h zwz&n#riuW8q_#_bJTcMrH>O|Hopdfl_|8v;1}?sa6y>4QR!IH&$oNgQ<%stP&#I8% zurwRLpmn0!LnBOS(o@0OETmQ8KS)>xGk4zsmg+0N&NukX9Q3oAI+N^%7!I48Q?oUu zuHe2ev4jQ0g!I)gaD6R zYcQxF9T23SmXcBg#3QAaC)8XEux zV5-%)-m0vy-hXa!HC)v0^AWVVd5nigIxZiFd+PY9S--q5en=S+4Nu%95W4NFS4P`S zYkx%e1YKEWQo3ci&HPowIkHTTHe>!*Se{8V7x+kDOkRbvozwv2`m2CCc1ng zT0KMdfY`ixD&qsc?6=?@+dKrLBlY@vwME8x_tE7aCEE_ontgez@=_Ys>SLp)u_uq8 z^2xVAFO;Au#h$Ran+?=yynbEt0AfQi&Ek8SqV|RI_QG`Iu6uMj$s@qVxC0^NMQs-w zl4DY!H*kdQ6FEl#2+R+uPZ z7=fycjLcw52A7|EJnz8*mpzsL*g2UfC@3V=5={?K9}s-{QJL;kx_XHRA9%6hN@50Xx;WX z$^(Ev>;WgDwmZj7x(gxXlQUkHm4(1c!#K-pG2C+EJ!4KgaK;`eIemufEssfS;CG8@ z@~{kww9QA$B_+1@5xzqhHE*7QY0clsFsZ#{l;k7~Zf^{zo5k9sS5>*@Y~>{B)>Hk6 zz>#!lXsDsalgh7RaN(M z!m5<4r;=ZY9+1Hy+Vqot|72fvNJ0W9@sv(unxXPwLjt|uU8C_&Z*n5lgD*AZ z*jNmOv_toBev>OtFoAUIvb6<#fsuQP6KYV5cP|FxMI}OQ4sH~|l zey9JHp(q%~4_^LdzpS?y)xMoXXv!d&y7x$g_M63|kc@N3o^31Iu{vF^W93ThN!;?D z@^|BN*%hnVJf%o$zDdlttPKkx36lQ3~7gK{~765FaU{I23wDR8;h0 za#BB)4BcAX{YdsT>pcIYMMS{YZ^!bCDTDiJg11T34m)_+D-S2^<40d^-Oiw;zMxK| zXv6cI93ddIxgjNA^_*o0H^6xy~e%e<>j2lorTog6ciLTEf&wI1k4BT z5Hm|(ufJGA&2JiR`8HNDs}uOdp5ETxKF3Wo-DpIFdGs0x5LXcrTYnL>{g^ECCNH(| zPr+?S+Hm1@HWBTG*RK&6qZ*%X*q^?C{J^e;ELp$5IcdcU{9K2>k~^8W8A4YhHM1wremaZ}}1R6aS_J%dyUM zAlRK}OS7379vmya5n5<^LG>KhK|~MLsA+vL33HU_)9K(!@C26()09S_qPJzK`pw>6 z0niN^^VX6`sq8{91ffz?R3sKhkG|g71-GbgG*1go-P~rpuC+rki>Jmv5#oa^2 zf2_OQPlh<}_Rc-UON;Y1(ahyc4b>8s$zzWhCEwy72TgM)Sc2Sz*^y|6^Xs$w53hpC z!9ntI6<}K!76yTDc?nG0B$+MD<-;?*A)dtujz@AYai%s(rUb;DauRiRU7%-i1#~0H z=1Nfg%wXfAT27ss@Ox(5yu6~m#}ks0xX8c48#vNm^Y7bI!2_i;s1i0B!&dSTzd1qU zsf605AN|>H9{4zecYMEg>p@g-Bv#5&qujtyT)#@>s$JOCb1IOqyl-fbec8k$Q{6+L zYMnf)yga2`0!iLJMB|7kH#pS_?=)C{@~r41%tsIg(3prwXyJ(Ya*iVQyL#R^yxiPg zUQ;QwPFt%VYMzNsCVD#$b|P4gkg|HQMXg(taU9u(Ps3&ik!=3Kh$<6$->J(z8WKMdY>`V3Re9kD7;xcT|*O5M669v~wxn=mq zXRwiSyNXt26nr=fTA{miCq0&4e{tAJ&DT-p1PDWUWM7|g>HPGN>jQexGuFw*DNlyC zyw$0X5nemZqPVw_lgoQtl_a0^JF21Mw%=^hCaX&dm zqDOZ!Y{|dM2QXMVLfwFz+hgla{OctFZ_&lW)K{cG3 zfY>W3bGUfZfRn7B_E-L~o6P|yZ{O(rt=HVcTXdY(y#P+hpS<;omMDCxvO?Z)nKql@ z*xl8SJ0id_YYFQ&ZE5O$cM!3V`{;y4(1A>zP@WcLD)Qw^ z1_Z~UKjU_XeeOYSM1W-Ig}W)b^~DelT=j=~d+)%s=K=i;NSi)O|G~P)owPcX-@e$| zUjC02f^$6QqA`V2dEES?2S;IWhqA3wxbQNFWB5I{nh5`0_Cbvl(ZT2+e~Ya5zm$7_o1k z{O}!S0 z8=lc?xDI}6Jn0)0ye1v`!#rM}DC-TXB?&)&C%!8Dn4}8M=i2N{a{U0Cs zej=7ey{gJKzm|t-+J6hx2Dge6_B(rjbjJJT{{BfCE=SW&#&v!FyFZ4@_2wX-xo-hC ze%b9?<}l{TWx$1P*-+I0AExd81e_R-5sO<(-+F$tH$vS)89=j51Sa`^j4QwFTq~gV zl+9CNfJUrEg8K^4>=yhS4~*aGzoEcvy#E^t%$DANtKyfdc|!;O7odI_>^F4a|0{b1 zZ9QGShlfYDXCCJJg~xVhsgRNOxw^DYkTr=INX@jP(a3SZn@!QWrRzh5)kJp@6q~or z@3Q6(j*4PND)KeJA*%(Hc}PY0&S%YOhMr!`GR2Tvlg`hKc<$%YECk~0K18oNMuvH& zPrC<8Y8PQ@O3G<#DS7k3h6ltJ=H{Bua#$5oPXTM=fT*y2tw}t{=iGOQ?hv^b!5z-O zbJ)7G6S*Vv2S*0vTD~w)$R@Gi!wT9EZhAiGtv9OzNnFmf{0jm#9BkkhgfUFLlp~OA z1Rl6wxv-BpoWhF1!-uG^1q>)>(>sb+@1K5iWLjIx4&Kgk1e^SZ6;^CEzoc8yLjiYhPza)OoG2BLtf)-#%v4 zaPbFvL}Qn?g1r160vH_8dHmj*bQj*i1+=XB-Q*FN*euNKM1oL<#~3NZ$rCl6t5q2o z9e5?=j8ruHcB(Y}W|Dw_0D#6WkC8cTJsjT~VG%Ng5^dbRb|xQ6Wz7o2w52sksJs^m zFmaHwYx^a`F1f>+%PV6qC$j`N$jgfq2Y3xQQBpSI9j}siQ^{M67oD&|gGCo#LP&@p zflNw8t?FqSbRxTa^IIB)#bx_hV>g3HL1(kl;8mYzl~>y`?Z4zm%;4H*o9RmB4ogQ0 zD2B;TSs0qI#!f^OU8&aGFerTc9nOnP%-OwqQ+nL_F9$)~nmH}4m^6{fnZR7+c=zff z@fufST6gO_!>QR2n#CgsMCnXn5Z*95d8Z2~4_902b4HPL-<^V+T$pgp)Msg>TR zD;nPk!V*U=QV%Kt9)aAj61tJ*GLnu2qt_6`zxn(^Ok$X6Ld;W>^W(3qmm{ zI>�<&=*a+0JA)6jN86O|BN1@XtooLYB<&0dvrh^V9i<%wE?Td1^JDAk>h!ps z7BgM2#ShKOU=t2EB!uXh7ZOD@S1mexmTnlr(u|3+CDY>?kR8%$}V=@-%-BZMb@L2;8>%wn8@;;SaBtaaQ}h z-ENg;w&M~&76%xD%_Nm>?QROnzpNp zmXmuAYh9AWU;(_hp2x(nBGOuQ@u;`csNL74$oYMk`gp3Z&uDVvQ`C)Ao0#VXnyt)( z`xFj1$~IBcW@#r{-=yeF)z8t6MNARa*l(C6z2}1SE3@rss`h!`gg-1u1lNfO=;kaM zoLc=7Nry-g(rEw#%a0g>ugKr0`2iQ8Pip-x{TJgp45iqU8mFO(rvgjbCX}xqw(Nez z;7Be)gXUCL+}?5@PI|cZZwhG*z38U~(O+m-oQtdtYK038j|_P@;gS~b6Bi_OC1v*@ zBBKh{m=sKv#e~UW0}Co~kumv+k`SwUv0DV$ z0X-d^xBsAp0xX`7gGB}h39CHOyC>|PQc=B79+`7^Wb8*Yoz>VfaxTeK$-${cQs{2FEET?Vw&YSgpOUt$0B+@4g z^z@>y+Dd4waSICPrUsI_;BNV~iwbd1u}hwZl(H?2uYow3p$9n%ewA66qO1qU!4m%|l^>SpA02r?JK#gTGuyB| z_iPdYi8r7YWSnZJ5Y%H#3q7sn)G3^o2S~u!ZJGtzz!e!qvf%P8WE`5$UY#dB@5w7{ zJ8M)HAdsdv2CbPav$E#5FwKww3vfXcC&R!}SAWWHrV}ln!i3PTY0wMC zyJ>_4?T6~^S^Ykb0INL~r*#v6U|m1aD2!11H#-i03m(Br2-k$gapD8LJ5-3JDS@1Z z3Ry9TUH${j<@sXXTtX=GaLON85T^^)sRk{==AYN71``dP!ulhPNrwf{HQ`r?2W|j? zSid>-w3e3Ev17-s3Z7Z#82ujm2|tWny#qU+%adETt}qR(p?!dl{=Nq(aNPyO>pLLl z5Y9zxzTZ{5Ie@N4$#}lOD>O7z-E}TIAm8Ce#3b3L;m?bN2QG9J0xmc>w`6h?OEHf! zPTh0Ka-fc^_3F}*0d>!$H8UK4U)0OEoZ7ieg^1yrOr2V^jt~{Nh3~gje|~yH7ha}X zMGa79dbKBRp%&LN*gX3YmX)e+4)P>lQ3F!wP{GB?>ZBcQmR;`14g0J>-h{wdCDiCc zg2KZ~NOh(JmDm3~OgvAquAE4Z;uz?l_Y~ZF(s?(dw+MJt9QMyJ0w#Fln2->@ReP=- z=gm*0XFYjhrys76A!=h&V+RCm({Il3^EY|Q;t)#LU0L;tw`RZGr)!w90)?a`%TLYZ z*5B^BAbeY$GWnkk=gdg>pq5hJp77ix@?8Ys>}ynr`uf%vyerNB_z9UENM324qWr~6 znK4VpcZyx8cor%{dRB@oXX`EB#aW{Yln`qQT~N&%rpte{Yl--TBNE3Dtmm@EbX#pL z@pnuC`j>yRh8?P^s=E2fF*fm0=!K``>Sw0Dq_RfIY+1QBD$V*fET0r(=5*}_XKUuL zbv9mXP99u+``&Y6$KkOhnYA2!_~c4j$U(M!&W0qvrbjp|0cT5wVwSnrernxaG>*4LrriEa<;8MGG~Sm`rbUM%9=Y3ZQ>1E!F*aK7tbR!tmYqG@;Av7tB7`}!Fp#&TOGM3kE#4Mn4@rrL z>Gb7EryOxVo}p4bUP-l~rW6KGnMkUo(= z>xq=QT}QqX6weSZ>@0Gn-@ZdqpdE}9XA%7iltD&N@~d4(`!NelNVMRls{$HPKO&0> z6pEd-X`fd@uK9~Rg5Vn87@SeAtL9v70IkN5*t49;{u}2!MJ~NARDVKr_QLDDHntmQ z3Nhjx{3cYQF5+BeIOSt)4+X4?p_~vgD=TXP2pj7RucA`5#<1Br@f`7`)1Z zctD?xdn7juh2zi0sXswPKZy8mX1EpE%zPlgbGBFxCZuA1uP=mqmGzV?ik#Qr+KYM> zLYs0Nf@f^R4$7;Kl4l7N%e6|3K7mbeXhI5D~%m&LljRa>a)%7=Rl zGnF3Qhm?oKE+Ybz-)sV>w?zDNz-d+Ldb#;4hoArb_Z^*g`gn}V0NZXJ z1bs!w>pCfn!K5mnD8y$uui}ZDz07)J@6!W+fD$h?_SbT99Q@@# ziD>Dy^yw@lr=quGqA>e7j5dr0fvN@GYG7RNjbDC0 zJU}Z7?LJPfr--?hz7^;nS-P?t5a&%>bR5yrzgHPx+VoT3x3L!_I1P#g{1`t_`T*MP zu|q)y^F@v@Ylg||Fn_=U(b6Al$o!!`%nuXk`&Z5c#g+O0bxXrR^>}~OhLsU~YMlr6 zFmW|{h%k8tH*69-!WFil_eV89tL6M|DqlJnuP>uu3xx?j#aC1mFf{XQ(l^*ONf0>d zCwluSzVEWlo*>{uO7s{T4Px#pGzIpJvXvoGhTqSwP1;3^`p5?hqU3dgE5@r78bY8d z- zB?W4OiR#4T_N<=>FX|FNcyV^f4hp-efl41v%5LdUD<*N8cuykcS6DY`5s?M_R#%9p zFw?F9voz+i*(WD#6jJr>c6WFC@ENPkoIGC#atg6jaSh-`QzgLVmTaEAoq5xqd zMw(KXks_tqH*Va0%B592JJyvn+6Bb+eLw@}1omiW!XLmgIk<3uONU!-Bu7Jv^9VD> z9b@kJXFKkt{szh8+W(SH+_c6>I`Med>M`f?QuypjpbCw&eZ!a)ib?A4(1yEvXfZ;x z5>cxoyaG|&+~AbS^iPx8aCjyy(RiXL{o!6~mW7rlFWiWlbE16k$%L#;k=neC34qnS zXAunyV~_@oA%+n=@7k)5gZsOtoRu;T&JZPH6``|c74g2Y zaX%g&9ANo9_%d6{@&u7lIB33%^dh9aynV~EdDN{27bd1J z^DTp0wJR{F2AJKZt#rMu7o6(@iRHDE7OPD{2vY_Gy%Tv4l7_0U6&~ zcD|?OY}TrJo6pMO&7;Gj>gru@f~s8U=K*#w?Y;bB|6gI^LDYyu9l7i;PltNe>X5A^ z=rtxHl_-OTn3LJ5(7?VRhUWSC`o4WTCx0KXll}b3H~)&c+(xr%DPcIGOM266>A_uA zW;QCSAyF1)Pt~G*M0;sDg0Yr{@Q+wD79p)+QJ1K?jY7AHc=cZ@H$pFssEqb?gi;1~ zo?jX+-;!^!|GQ14U+GolH8YVy{wXRqS4*=p?#VWW1D~^LpuhwJ%%u2pZ)-#f5O< zN0kTmIh|ImGSfrNr1YX5Zj;(wI_!*iI>)&koSi4N`~D3))Dn&wrty2^#Jqf%S6SYs z(>;>Cl#OPp4RF-`ytL4jXUi_X=W!*|V$V$Zz3$Llw6es2aLQ8L%ZKQg7QYmOxbGAE zn!H;<8U1Ltsr!F9^siA&^5*CxSBr{QSvV{OI&q}LkD#XnFNsK2u?u-y0u-EM?QiH? z%K^FzF;Huz`o8_1ovBUleM5`qnPw>jnfMF%R!Q8 z)NqFAb)>E3c65-R6i;kV|J=k<#AS=;(O&Zt%guHi!p$y86x-=61)bi$ zET6%v+orVhyNx;=y+y>NcA0DdpuAerRYeARO7yDx%8_Dq+%9n8#mXfef|`#Wv+nJ) z?hh}88MPs@IPmB3R29uv4$FkX28hcAgCg_4M^AciG#E>Nd@T#`1t}BG0#1}1p zAW_qDkQAS$3VA?X>-Wz;eeVmPa{Y}ryB*@P$Im0~39-QIQm-JNEeGhI%IbRCW7#Zo zecL`U<$JKfXu9{VsWlYxK&`PeS2gyl1yt2xL>s9vW?UdP9Hi(~f;SUkd}(t+z$;o( z@R=9|mw8u&EU^Ue4fE0`@uh=)?xSv86?gC`cnXoa;-c1Wqp%B|C*#p?B4U<~KEgPV zizuYh0ffmSmD2bDPbu-Fp)t&vlC}kh5aXZuU+nfrQC6Yz%BBuGe_XQYk1nM?R`&S{~eZ}EdLOS;>kM8M; zQeM=7Y5<3k@Y=^&!Wdj{m+|o*+)mi3MwCp?5Ae2{O1`Ln!ve;$GB4^A-XXC2n}-Yu zh}Q7RJA`0%;s@czZ}uby>lK~GUyn6${DtkTRZTAEiTiNSeRzlYac7)~<5(P3z2SGE z?z`3Vli&aE`VzCv`~TAG%l{-Tn=R?}`Tjpp7dfa}?Zns+hLU-Wnp%=jl<;Y(@qNod zbEzF6!J6A!%DDwh$-g^DZE`+Dp?B#u*OCMc=`JrfW>Jv6dpl>eP&rtJm9#sF{RF{_ zcVeVCH4K-lBo5xXc;mpuU47z3&!t*AmCs&l;^quLcjFxW-|BCdcE88|Tu!o^C|deP z)2%C4cBinSRT`g56lxGFnlG~m%UY}6*gSP|`=D`r#G~gC( ze(baN)|-{v%%jB^X|&L1W(kP-NnHiu9?QhKA`8BB(ce;Zu}Ffei!zE z!A!O*beOlDdduIAL&nlL7^7jm{P2WgJpVhT>s(Vj%OB z)5ZpF9E1(C2xYTv8)0gR93$ovWdNQqS9}sqfN|TkJz>gE^D%E+eVe_bX#gtXHg-vI zac1nvKbc=*DztI`iZD@XhsxgjH+h$!41N4Zg67GX!EY zQUSZ;&EcSE>Iulb#c8qtbTR^uJm&xb3um}u<3>0Tf>Sk;9bg`U2f1|zlxGTTtH;HB z{tjS0-AP!C%3-(dzxFXmKs&L_wFfP!K81)-L=ZAZx*c@If4!DbvU+(a z%aXLv#~g}M$czY+@fOpTfbl9@p?W14nS{P}=YA@O`8?fDD*I4l5r#LvM|dVf#~Z2U zoLFY9WK~qcP$IeyDRbC?@HviaLb6(0LJ6A~kkEXI=5QzXzbC8137qTJhw~BiW-h(c({4~*wOidW>7f;rn8=rb zdWWM$?Dm{2auQI597=*ElN0$cu?x3Zs;M0Sw~FIteqdix#=*fs(5ddz4u!I7^`HFg z*oMm3o@&reV?MM$=I`*Uhji*^b)!=^5WZbP|b= z*hB$xw~Wj}XMv(D93J^lYW?g_YM&I>y#TDhKJW5^vs_7OeL2@9yHGWhPFK7X~fv0+J>@-w~1?aT>J+= znw)g`NGx^>^=!Luc|gIV_N|lUI*JQ6dY$oG+k48It|OOvFA24IEJ(II>z!i%CxL%2 zZIzNkO*V<{$1KO`>649O0+4y`Mn*<+X433xMk?1NJpT@S8YoJjx|^1gD#otnT*}d<8%H?4eCqpjl!u4^ z2IC!$yYvrGq}Y5G0?oS^(G!|l^DoZ6>^W{PVI}GpT^zRHdhA1HTS*&eunX2@L-ITYM;UA#%TVT`3Zl7V)pki;jc`2Tc<#LbrQL%LmA! zK*};ie70OSVMnf*wxP%oRfzl=NF{I?iEA&;_*5yUzf zDFxoQ7)pH(AxF66=H5ojBlF3mEyD;D6ScPFz@Gt)m@2@$biz%10}N+w%ggWU9r5(^ zrj8TPp6c%^7%w0d(i<4}0>&f{Rz<5GC8@92+xF~?%jM%#PgVxdX=MQ0Ez1*&RtGRK zaD456!_N)_*fWp9!zMTqh)T*ZseHJ^F>j9*<*7_H=UqOM720_toD+Jb`!Mu_zcPM6 zYgF&lNOa(bT?D;nkWn1y%W^arjA@IJY_p(qyXktbxG7a%VHV10Na0a0AjBa8bf`B{ zgRl0ACIl1siIv| zNF{M+sMg8dH-W^hO1oi`4xj?6qyr)mIWGwEo;P;Q4=nR{9lp?Z-qZ|e9VPiA%S$6J zu!p6{>+LpQfRd|M?d*`s$stT{zdMco=8U9BhxCP0stUl<9Z(a=z;agAg^GkNZ(-cI znKZKlx&}Qv874wyko%>ZVCxQ<7ln{$!&-nC^0NtHbSaPH!sPP>LS@-jR z6AwsZ3D2B4lMfbKG@<`cXpQLpIxqThUzkip8T{?o>R}ENJ?B|<%=d5&Vp)kgXI;#B zGe@%E6qxAw*GBc*zxU;@|Ksoh2m1hve!VWn?ii01AzNq^y@j#SEb0a%lHREU+!*_f zzGZ_W_8>iK9#bkpa5AJxV$D++CfpD)G)T#g2&}dNWxS22C6RHkjaOQx!+oIM`Ptuw zH_SLT%UIV>rY6{T(tWm;uE;xksU2dPXIvC*&I~YJjkr+r@*E~>%}3!?ms<}{;?A!l z^1FZZo`WPus4<58^lujMw``-Egy0lk+5@5W*Z%gGU;GbB9B23B^{8X_etw$y_SEG4 zSEWM(`GvU;d`a+>%|v@CC}Ynht*4&aQZQAb)TVz+jL|0vA8gorXnHuB)ht^g%V52@ zM*Lq}Jh_zq@;&+O7k_fF`@Wx1?2wrbIw{rc`01(jXHUbA{OyQ~GQs{`&wu@$-tF*2 zPL*rZj|=^0U%&bp5?b0jMq9M;F<<*Rv>a8q?p42Ohw1HC&I&wEt@FkGk%`8OO&rI1 z2O9awc6}XUD$-HLlrF{K&AVgr-mAKy`GQ!X5VP`#tB3kO9Q@Xl-W@Oq!_G>Ve_lcN z+Xuh)4{g;J=pZVT%t6e%bO$`qT7}z>d6)7-m&R}Hy>}4fM_Q35D6Idc!hAaIL=y48 z6d!!;&)@wcm;!m?BzyQ)%-enof;N6IGv*Q$)7$WbLGO06Z(aVoPM{(Hv}%j#Y0UY5 z2FMe5Il=h9Yx37P6ucF}u)wX@vzt%0?`BLD9)fQ@Bdu9G#bMAMQ#-@b4u35+iLIxP$ubB^mgBV%Y5Yk`&of+ZOklV$*eciFJ$ByCcMZD>9>Sv5YB9gL z9rOMiCNMgjKis=BWxbK6FK>xnFx5y4C)#|f9NkySaL35h^cXmo9!omk?S(iTl981< z(Z(^od$JaytE9Jh`e-w!tIQqB4)oeXBIC%8{_>7)sVxtUmd&-bvYha1r)>LD zVc`r}e`ZrW9IH4HDU%7$|2md)bk_m5*Vepkmg+t}KH)nC13>rD0q3w?oI|eR|Lz@>u=WM*vtP9+_q1s$IweazvB` zjT;4E4^#+ON=NZ0aF1;A*U4<_9G6KyiHBeRQfz%DeYc{%j{^>R+r3$MvS#meT59FZ z%6^RaE8nLFnBbu!gGj-hA`-5+E_KDukh>sF>ox+~em!LpFroUp+(GUoK!qZ+;Oo3Y zpO4gfXhj?VW0t1ZycHoywb{lbkm~q$I;dvRaqPSeQ}Az4?Z?+t-dh|Hgi;g*qQc`3 z;sH?Me@m85`h+a}NzvXB?b+Hmdv4n?SLNs(j1?6V;CZ24kl5Mf-qqz@36;egU7Ub> z#kG6)ZbzgTL-v5;NMth`=3!&VADjiF6*v!;R4 z=s>-a4J%yKr*({#|3e%`t?!RxF8JyG$$ZWdO39oMwSqGJ-}Lx*3t8w;d@pT&BX!VR zO;|oufF%VyL|JNt#Il^bzD7tM?$2rPk$UTzH2gXC1$ewQ+Yenby!O#`1^Bt1ZK9^L zpPU>AG=z9w=NTqvX2Q(njO5FBcSy^3vvUYH?upQN_lZ8rlkD20WV)mrRO6=I>X2R8 znmti@D>y{%g>9K1)zHHe#BJIO**C_8GT7fg7WCcGq52M zte#1W`jjQqDs=*_(NP3qrQ0!Fr1H92Ttq0l1JBUM;OrVqlp{P}^zq}z!563FNi92LVW{*fQ}n-l~khseYrOr)T=%XsgSctp@>>l3wixLuP4wr2=pKUkAV5hHdJE(ou3H_Ve}2dj62l8+++`*jay)vr};yCcY(WV7Q7iU@u%8*>DnjC5Rv_Hhb$SMB;#ozqA zkc%C6+>)AksJGPWYKUxB8TQ@)xv`g9w`Q~lnrK=n?W3)Fj%w>MX#0}Z&=T7$)S=!s zn5op?hQm!J(zSEB{mjMIC-^LQb`v=o<_AW6HhRth-jMsT$S>Y8zIwKSgI<)&j+l|mi?{*#b z=dfaaU)uqKP292JyOQtV1T%{OQ;IpG`M6;Z|G?cuh&{fyG_wZvDYUAeI z!QTd!upJcbvuDg^$l?*Aq01iIk6TTs3@|TB*30N{;UDZha9=TssEBJ{R}?!wJc0gpj)U$1ZlF{rj0TA>#;PX z0XMRg3NZbR;6X@O-U(wRV_eYZJTu4TChs+TO*`ppx_-Vj-Cm!JeA4^cPWs;;V@efK zg(rp!aaPYCJGZT0cXh#a*M@BUmj{$4x{t=iLzks7r!)4RfSg>hfidzAuG8rt!=`%s zCouC7cmh}8wAW@w?hPvtC4gL~pOoAGlWgthUlNQ9^TJ#;EPR!;{L3!-?rCio$gs_i zFk@cf%=8&N!8S-F^#`02B*Vtn(Adzt%^)`iHX?Y!Dd72USe4(kFqjH!>(G%?zv-+a zS?lQqxpPO=31b1@;belys(s+W)k#t$ps0j1x z00V@O!z%QQ8uMW;hqc+fJC+Y~_< zT(-4T&kT*y50zugRXGNH~W7-2b^>}~vg=vbc&lc6v{ z+XiJ~FrT0%IF|ln{-?~>zv4F&6_xM}N$*=l(pA;XI2bN*ke_e);)@|0&JahhHCJS? ziA%md{k+C=wu>h+whLCIw=Bj@VLBBqt3l*q7`DCmTjY6t=e~&x*B}mH^p>5A4!7|zB!Xuhv_{S>MpWjrcZEV ze3$i>mE-bo_iwu?SxlvX)h~b8PMy0*|5_f;ZoIkv%pFDR%OltwE5mj#1r-$@6yVTT z=OU4=S%b7xREckX4~KDMu`$&f$^HvACsI-wm|j$^4}9-yewAq-_rSMc)x>;XZd)se zE($~z3Pc=%SyG~U{rW+qmN4QV($P9%Z-F3QDNY@>|L==L>&OVvvwb05Umoxod@7P)j)1&e-FsHWQi!h3A6{dqqrj9f6UL!nhv z@ZJ`pBH`PAK6{XlqDFA8?>&_c1x`nK(G$EYXEPw=;JZ+n%OYOCD|hdUcGVYoodB$n zx;>3l4&XR;jL1WHuGcxzHN&+7O3~ya#b*M6uCB8bm$x)$dYrQg81#m_!R%spSfI5I zRECza?2ib`k4bAI?`+)%ZSL%|%lY=IU*Gu`|Hxsxe~zF7tZn4*Xs>}hZZ@ZVJ||Kx zn?faf;G;$EM8GI=^uGo)>JQ>e?0%XbHF-Sr=M zqu=et|9V%VYbtNl2J&?4V@Hwv&>!w^tG76ERzmp%M`X;SHJ1@-gDb115>O7-*-o_r z2Py=$H;|nSM84u%Q$OHan{C4@V|wKFN(#0#5hN2}Kf#Mms?h!X*30aM>p4q=+~guW z`HW!XrjNWZ6m)XlgPtv*lQOW)9xE(7#RmeietaQF>I+`ILRkWBtv?l`%fi);$YF}_ zg_1zL4!C0b*!Y^n7H2boO8O3P`@{t;f_}PIc9cqpM*sWbD4Dp}=Fj&} z;_oV?enMG7Aup`j)!?~nA0ZN_@cjeeQ`8e3byk7);6!cml z6*gPAH&#Hf~Q1cFTKb9SZ=oo*`vzo*?hfr~ zmyUd5u@qCIK{wy6xO{cV*uE^^o^HtOLc%t3_n?}213fwBk(`L^u<=rXWtL*}3VNjG zUO3kjx{&1GJYJEPdwPu;>+l>vw97StlAz4sX9QjfrFl%=BgK$G|LuT^knE4U@gmkG zX{bMJrv*%|Ypl==#*o_r`Nc6H(g)JX=Zq9cSIr&;Z;=qF?-K7M{* zvsCE9>nu7q_ft^d8CSn<6Nztbcrjur(sj`H6qY`bw)rrCa>7tMV__onZPrr6LRa=v zC%ZPD@sVs)$k-$6cJ~zh&pEta7TlgYMh4`%gNkMi5%7^l$Tj=LTF;eE_phB3YLK2A zP(|*7ClKOUD4>lub5w;tH@l$15KO7AXy)2Z>w}*keVz)uG-V_GnP-AAzojvJDI!yp z%LFY=FDkqZ`xC=`-+DE<(^ezE_?hUJ`BH~_o#fPPb2@U*F58c`|_%R~2EuPz?&O|zaFdxk!T=Du&#U0hod zhi~_8yf5$RXG<7L69>(nVc>i%dq1*RDpyFGjj^g1@k0CaCdDh;4t%Z}!U909-;Ou+ zrPoZ`)wWte^6WURr)ZL{Jh?^J`{+rQ@rUD;Hx1qiEFWl&TEhh{GXo1O#)TWs3uU6g zjV&1^DNA1nn!agC`WtLcOae6DH5ET#3 zi|!Oz+beBZK!)QnHaajM$CX6d(n;HLz^J3f*az zD+P%ObW65i;bk~0+Zkb}wJZ!}$0E;-vE2o08vF8Y5ZJN}w&#f;wrbpcaxuvv^huGr z^`E%9E-dB(w$s}=*Wa-DcoH=N!=ZxTWca9_pj9B93vBnm}zZr_HMJ;XGK#=znB}SvGmk!c?E|fOfD767pcx*9a(VI ziyK}Ib*LD?D^sZJV2|OT0cbl+cX5I+S^UcOz3L)7Fsbr}s~p8c>uv7w8-(_^mLaI_ z?K!QRF;;I&5rI{*BrRuLQ77(1o7Ci7T2h>tQr+_)*5jO3?eCfmvSP>`XB_s}(kA_t zA3gusao;5$H!LM8YMa3XWV15jpPvoyV7KcNtDOR>lo5aH9}$XObN6+C?|r@}7^Ov4 z!xvn}JL2z7LbGoHD!7~QgSFQRS1V8{Xz_sXjxXcy*nGnIh zo9r5LC|DRPIT=0E+`cr85U)=^q1%e-Ig|D>5yFaG8q^Z;GZ;Y50Dbjb$_vGY3ujgt zT)vcl%HXk)>G<;uby|esaeK9y$rpg0o-UR3s&*v&?hnm)8wKv1zwbbYh zYLnpluO#LhnxMf+`YBh?!P?=&4#GOgv&fOyL2&y+i|TvZGee{U2gf%TgsvQDjuAuJ z`M6|N1N^T8UvKiQ5-E$!osWM^Q6>{1u}~UT)TjT~Gsr=954PnsvpuHI4}BZK)uW0c zbNmZk-B7ikbj-hoZ*6JDVhx}2VU8=qFs-Um#G+4&!`(N3Mu&;}YZ`w3I6y?+Awem> zX9=vnsFcH74^~yRkqUWtWT;avDb3rE+Rq@yxtze0&Wf8Wc@@n-W&@w%1T z7uEhf6^95$`MUlT925A4z$LSB+?>zkNHBcS{px3E(~r~W+dJX|TTxKEgBUr16UhcA zs*CwFWF@6;@A7IggRXykiI!M}k}whH=%vp=Qf)5>Snz(mC>~!}&r!`{+E%)Xs7`7p zJst0s5LRCK{aR&tXqU^1(FGM|AmV`X3Uw&>#gM`UPYcw zpN4+R$k!9swe_8R`c-XTDhIRUk7PQJR?i@o$6o zizUDQ6T$DcwcL$=$A8Lf<%1~duPZx|8nZpXq3$dGWKvZ;rCU%H9Pdv~1aa9k=2l~* zZq}Kidn_Nfo_Js*-nwft6s1w(&5>ZCrQcP|oHBr%27VUZR$otBZoS>UT})ngea(Zp`)>;JmrDeC!5a2YMF<}I81z$y0er+qc*pHp|uNNpIIV+^kHr(|fIqBt!6B~*h zR={A9Qf=t*?Y4jUKPvqVJL`x$s{e*b!Wi{A1z)=0XYkTTQXo)CSfH&aMz*S0%`3gb zYaXN)^}K&NesbHT6G78{XK^T zzKdr+{h@Co8B7bkjr-wBSa3sq@dzD~jF z=PMBCYc=t6>swS4u?)uK9SCvv7-`Q0`z(gm-EdG79emMz(CaS;HqwSnJ9I;eR&uDf z*V|L&)`T)^p!-0Suu_EhqjmD_zvj|<>#&KWQEn*JtV>1v0F?JfJ;3~W=Pg2@+?^A> z>bmi*{p(Bk-~U4k7eX8k2UhE*etm;Y{_*&4z*YBK%TZu9EuodD6DqV14sH4i4*1w^ zExBwL1}P$#t!5(yPPBhCQq0`;{Ulhyr6DwpWxcL~^LlsvOcJBM=i`s#ear*rtNG0%=XFZq6F7b6Zj^@tf z7-mKy4ic6iQVgr7$Vmsg&U_FnhLm5drtqTLV~wClU7jM;N?%c$>StG+%(I_u|1a zYcw@2hjjr_wjtg{1>(Rh9F;*&$*5X3uzr+`{qU`)OhZuP6}yMuR=F#_ytK)-pvZFn z20;fD?IJ^*Oh@~p;+(-6i3c%F$MajN2f|DTKv_4^b87KSycWA?mea{mv!T2>jJExf zLB4GBNYyaJSv?;#TKf?ER@yZM;~QauBhjci>07vZmv&RTIXD0#Oh0{TZh+M1T|T~F z)Sw}HzPZ*-Wb)abi#rL59g+Im_<*kAgFEyxAsDwSx0Z-((zK0#3f_(NwC(O_a`h!# zq`wg~kZk|{H!UMbq6oNJ8zEcPU}9!=57YeJ!-3(#v9pgQFGU>Z5$5><>d0LPc`YA= zS<8+|mBLw?S>GS>zV-UIK`&KS`7~{&s-`?NQ&u3;N;tQ}WrF?MIu7Wt7xy~(sdUr6 z57!tGDk4HxV|b@6Jq^e_clcPjp}om?gXQ7;bSQGf{AOtHZ`LQR^7@gw>BPqE^(blS zu{-fzO^ctEmoy4mkt%r3h~BhgTAFvTDJ9CfbVc=Os;wRFn`_b&z30W790ts3wrvEx zvIHHVGQ4UxXnb!B%z(rg*!f~4j*sR^c80i0zPma%J8P*k@w__KpcN2t0@evqvSqWN za$&W1uI$dd7YzpdQs_d(DX06RYl@@F$3@gQgf5ZyNarzlp`+DFtm>agXFq;Ee=HKM zqgUK^!_1Z>PIONlp9Yu4eJAm#FBX1w08o4u;hx#8#bH4`cq7jGO7Q#HzQPOvZo{&f z93iQh2tocU%hEl8(gW=}OQ&h#7=u`PQnM%6GxO&5(cbq2{odiAy`RqqQ({>KDOmL= zdnn0UGJj!jsd#{3h=VO=IZTdcgypVP_Fm6p7HCa-&!AKdjI05noqC-!+TG5RvjFnQ zw)JQ~=t;fW8djm$KG7|7^Ii}dczOX@MTUwQ3itb)k9_F0v(r%^X*bc}oJp2nHkiLQ z{&c5>V@_Osv5Ddg+GF=Sd_`M5@dszj-CpWWKW<}b`j9}IuN#-cLwU254zHPD1ZUO) zJ!4_koVPe+=yK7I=VBPBh_Nh{VuCcS}f0h)B0Il9GdfGz<;WDGlEl{lyb+ zeD8Bz{4v)sbKm!zv(G;JtiATy5_bfR5#_?-A>WfKDjttL!J|m7{cg>f+*q)LwEl@g z&Mt>0V<^(5mefVw>B&!{h<`fOZHYoy_&Sv$UPa}+_h(+i#4S{a= z62CnM@rJ|8^=GHW_b=lLY}LiJ^tRZ>ofCDDzs+L0D3z4cO<7>#_S(UdsXT{dQ73mq zQibEzVmx38+G}rBRO)a(IyB`lwRML#bor%R9s5kFY4@sQ2t0s)kWwtec?quGakd-Zn4BnjenQ%KBzths z^_D$nu+%N;AceRLv;m@ zevi0V*_5oi?wd5Z1Ld&ZQTc*!M8;3ce5Wqv4?@CDc@xj?dV*1$Sl=u3ynUHlZLW-A zZ@$j0u!Su^A)(b{om-b_lC$Lha!gm?uzYpU+FpCwT(*m)8wJ1Z{fq)mxb5Z(!NZ$n zvM(Q}i$He8s{mFmaJv>f}($fLhc4s%>` zETx(q=QLohS0ZlZp2Y7=cAawh+9`62Kfk>c-LmU1R7dYNN0j!l^BBP7P(b6JPIjWn zzqc~Nj>@5-QZ&t|TpP#{>6Uc^qAK;t$w{;A`LONxEb|Ib$J+C$P&Wo);+Yi(_=AR` zB^E!L2KuYsSgc0Ac^-4d|D#9-*mL4Sij8WCTFB;L`bBODxyXmb9j*N7`~vIdC9nG@ zlVZgMetT7IXSIFeTPG~LX{oNI2_lxR6VHpV0K}QtV;xk7pM05;NO?VWO4eREL|j*> zYSEpG$&D`sq#Pswo4LvgyY?{bd+hH*pwSA?)$c>!u|bKSK$7dDkc?Rf9|53#c8Of# zLS2M$F2ttlDMnh*d;4@p@|bCBpG5SpxJKT6^vLOP`qvz}3|stt!xIk11Fou5q*mK92eCEi5e`8JzRC$%`cY#4s4!~U{}-!cyXA&DE{Wjc5+|W zlzfjX9`*ghW*PhY_|;wMen;Z%BeyH?|B0p8bOsT2bPWbc4;BuKUAI+K7kj*&O}G{( zO33~w1$msZIt8@_u?4}1K4`4E4Kl()t?~?)KhiENG-Nw}XMlzP%{4N=|c5C9y{Fk&C@8r2C{mVE;^ZEC6kRuA&pS|qh``(l>Pw9 zV?K2f17a90pmQt#asKZ2u%7jkp4b90fbSH2o%Z|C%re~nFTFP zYpR1+|JRb4zlQELkf3nI`h&Cf53~8#nEl&}8nyITCx-Q|R28=Wa*+S}!W+%M;EamV z-#FLz*MH+&lK5ChStHUZxorRQZ8byxtb4no^)EC67X?t6>6#}|@w=PlUw8f=Ui4^7 z63o`+uK%Um_m3m_D{1_%hqC|fCOvG$vvH3N4;xBq&VNJ9y0<_2M1kwU??L|L9j*Tj zc>WhC4BY1CcCVz+&tPFN*>o&u`XtK`cI`h#Rr1SU48lBz-xw7*Pt?;zIz{ULfr@Z` z|8I`ua4r;xoK2ni9E|N(>8AKL*vGDze>wuz=t@TFR~<&45V96Zu1ajw)Y{vfs`vh5 z$f%+LuoKe_*v5Ya(4|{KAPf$VpM8YF;~;9679&$B+6?9VXz14fcl|oz+PsSHpFaeF zs`pGR93nukr=N*HOs~in{Oce;Cn5+OdKX_0ZclIxVnB%r8Rwq4j@S@-#(RAF#o+1{3=0`6Hanjq%|23s7RzdQLWA%x)<)1&yl;}abvJ5DP z8DlN4!b4XWSMg`~Yc>3@Fa9%jAK=yfFBHNTnSWuzk0$@Xgbn^)E2x?CZoGl}Z<4{1 z@qe9~b9=X+I{7wynSCdK0>1tj;iVgYo(8pUKQH)IfzCmUF>bLW7#8$bN!*|R1WyaL1PEatqEio2^Qh_a)#|5BmEWqcDlg$r zc)TRN)bO#wev`%UuBnai&-KydjYjtTQG83s0v|dE)r_^IxCif!1LJjzl&soaH(^vr zEtBW3qyuUV3bO%#B<7^+Z=+Uu_4@|iPM=o)f|LnX=+WR(Vnd1V)NZ4SRlwDu>+j@j zam$8|2NXXq8^xqBna{?9Y(h9TICx7hX3kAbwt-oeJfr24$P*UA^YW&8eS_TQ^% zCLN%2J|udCAF)5bEr$O+en7ZP_JU8$;fvU(W3>!1V|g|psA90mW{I$L;)Z7R>8DY; zyf?4-7cSzyKe!xfL4H+fD)8lRjr|4N|1uspGkSxft+z1|8hSO>&3j(=gl?h$_w*pi zm(!uOwJ}CS0!e?F*{>JCV*xq3{FW#GH1&IguL3rXS;qpd!1=JD23*EPmi9$BN0TwO z_&H0xni(Q&OKV(CfB1?&TvWdj@I1ejLjE@8ai*E;6V~1uxL^`p9yCq0yL{wV7`~b2 z`e=Ceoq_>qHXXW*Wh(TKS5gE>tN`x6JY_uAO|eLpr=Ag_>!tTU2t#h5Zwki!0kQcJ ze`RPIT^?_CYz+Qo3UFbxtu+TGl+u6-BQPBQmVfc%D|M3JuENv~4!^eaMX=xbz3gO( zUa?eX9B6KRA6XusUQ_2p@j2nU*v|Ugwn$6RhiDH7)7_O|as6*Md2sNUQ+jQkZ=Tg0)%$?Jurl@a?`0{*J|nm#Hc>(G_ug-yghpz*%v6dhb6$l6 z(?GOF=>CKfz+2#8#elld1i3e~>>!pB4PBRxuHV8iCGtdybxGj&qH#7%6#kS0nOv&R zsZlrndnC)5W-_c4bR3x_)33pYD(ak%0GO}+wXN{)`TF(YCL>@ITylsKNR0{|F;t_H zjt8!ULz&RY?Q5hl|JKiRg%v2tWHkr`>XQKm;^zT|?5RIp^{-2%%c6P4R6>8QO{tGC z%zw}*l>bn;*d_`9{v)8hDtR4UE4elo#;@`HppHsdG49uQ4Akfjrq|p@`oUN!pdq`= zbJ@7TkBh{G5J85oiDhB~^2M#llYu5u4Ex+MB?B8YQj(>A5O`>syB2&=J&4T0p}g8@ z8O6&!!bpm-%8G7%`%7v8$~*p z`Bf*TADW)A1v5|Rqy@Gc?YjkUlvf3!HxTG*7FAD|@l<6{ydp~5L?vaOc zSb^;<&eYJun`PS)H#5Ee4)-N(p;_XGr#;ByhR5!P8Xvg3g^^cZMpk)o%aAfBw;iPV zyYGmu` z?;d}U?uK3{OS5i&-xrh9W%cPQo~R&?$83CCv~ZA7!$rW{WR2`mGxr@2H!V>of9wix}z2Kyo zbFY`>sYNY*K$dsoVzVBJZ0?n=^ryiA%%L5@1!0n zTiXn7vj9J6S6c1}rvSMpI75&1)OwB$YL)@>C&QsYzNjGkWQ6oa+v=j;c@i%QGa*ac zAWysa@B(Nx)+jek4vvXGJWe_knP7Ru0&4HYYBq{ z@hjIz!ZvfYj{%MPUr#~hx^xZ!qnbfSCQ(n)WhR$wTvfmP%5<3C1Xpd>Lwx8Je`Y$* zN($V!a{3rDOV-x<<>uC6V~zr#E(!{gU|>9Ytg^rN^c*;V$Xv-X{JW;7UXg587>PTG zE2<>GQTG)V$M?J5Xa@cnH{f(jW4<_=?u!NTc+z>Tfix* z`5LJX0awjRhGRItDauQO&8_svOQo3E>8Zo9(^IF9iuhP}tH2g?2W9xT2p%-pwmDWg z&lWeMdfs#XCox(Im0*FvV2joI#`$V>IQ3wD^uyKNq zNezx$GLx&VQJHy?L|E8wSRbEt9l}+ObSCNHytIW#neOJB%$g$ck6jG52xxl)dfow zx58lE-HnuxV7E<2AN|BNgUxlTt(>!ZTZc;JSYV5A256R%rVG7CHuUG}gLG}kBO;m4 zfIL(T7+1xP8{>A*Q`yOU2Ho z?KcH#Fa|W}T=~6MJGsChC(uWpDH2zaGLad`sn;n2ANCB%oW+z4=@R!)Bc04tuUirK z*t#S_)tr{sN(> zUrTflSV>$iw|&`BsYeanAF`hCW}K`BzF1# z%vC$@<-4UCip&P8qCku@PwX`)*Jp7x`z^%iu!~oI z@|V@`*=uF9QFrl@REH1w^*Wv*kRRTbqSAH0ZvG3o`Tc*iYwxag64R@&Iu!6Y`mFic zjOU>%|^ zcP=%oGNkjwwH9n`0(UXvVm&9^_!20EpL(XbK9_Ks542o7dwgKsllg6O?9nuDn$kkr zV)b`G0nMKtQNNJg?X0~B+Z8%2_tSpTfiz3e5q=Sdj`-Vaz# zbw3z6Df`?qEPTqV0Y) zXUJLi-9sv;NxSOSeN?rx;hQ&a`24NUTEU|8IPTWJu77zl0t6r?36#JZysKv!rwp)Y zLm*l*ixTp)94OOFL`N`i!%#-`?`HJx??~Q*?AGwkJWoldmj1L9Ttx?Mx z8LS_J-Wjax8VLnYX%Z2--oK_wDDu*36o=xb!jM4(UeI}&r2xsogn5sS8A7b~FnMZ^ z>rm%n?{uoR)*R87xR;!}b+$}RNPtF202hTkFDcs=U}QMkiSS$>i=dU5E~prO{uD9W zMymft-)+ZF+c@mbr*P7P_n;Nj$MV}WzrF{)`hWU^TLd)i4|WxB)$b43fU@T|z>hnC zPO@$j)}DL4LN={|gcp0Db>@Vlj;)L&O>gR4fwCj>6TP_A;ew>KRqZX5AKwI8 z#f~5Y&Q-N(mg1mM1^S89Zd4(sA4{AI%qM(9)?Ng8K?DEqe~PYN?;*Ip_Vhm*HAzT~09;e%?qh3nDv9ra$cx;`p+jf* zkNyU4qirBX3dCVkRsYEk4QvunmSkwQ5XXW3-$w~GuubKUP~Si3bb_Czf|g&TpK+$V z|M^Lph8WU$+lCm5)`BIMnbT|D+e^>_Uc^g6m9YX@xenQMdm@#wjY))jE)m=X^2B59 z8_>_`hhcqOcbqoQj17_$L%&^nu(DE{0qv2L0d@R`9&h8TmlMrtgj#SfR8)7^Q*#w( zP_qN{47^W)%Jmdr4gC0!WK-kF<^uFErT{B{)A<6{QoGf#2IOk*2mx{KYI5`BZ0j2g*bgdsVZP<~5z+$M%5MRd zJMZC2W<(9BzPhwWu@Wvr5S@D$AfINJ=5dm1j@S}E+m=2A+6cAsG_cls&}2q@^WIVd zqjIdxTx-x~4?p4!=#~>~j#6Hg7}+XI#br$8G^`vm+bg1S1&sA6ApNuex^H_PpS}YL zo^z+LGsdE`y|l|W%3=qxryUX(vwH3aPdoT$#x$p%G5#1rE#Vpxcu~X2r%#*Y5*~G~ zV2w`!b*tGW@Kj1uqzoHFZ$V2ge;~!vuxyrS8s~_=41%}UpyRl&_KWv^!Uy##BaJ{L zY7?}CasgJx)l$Gih-3Y_16QaAp{hwo6$>x^@?kC$tOmL*|*qI@}+ z^xWYp+qhYhwOdkU>kWk=C4(^Ap~8_XF?X$Wh5Dbh(8$5srkmD)l1hF}2{GbqXDy2F`<4MgHD}-Eo)Kx}Q5^3Oa6@a@OvPu{L(@Kw&{=KJ}3eSZ5@f z2Jk18mT~eecR)QNtP04`R;KhNyaDaw#67IaQ8f*lN41Y?3ri>BIMtAv<0nrY)uXDY z%Wv1}whb47cb|fH<8-+++hH}ORkgYJ<1=&)%45qvx8_Z*wTZD^6 zzVgs{Un=4!pNG_nfOU^#K>*i2*sGG)QKI*%- zOl2!MFN&KrVO^D?4U&RWAlJLWuX%JXM`^uKIpsL9%{#~*({D2H9DbVVBgH70o*CV7 zMf62Koe)qEuK_yt#T%%ebyc;1rZh3PfWv&-Y21RP3Mk{3r^GMfSpYqeizT@*ip9$D zm+Vzw8P){QxO~N<79KK;#6oLX05bC%2C;Vx6kA_(B7tfC@HlFrtwE`F*nEHbjMK`9 z*mw!7O`uSkwZ6Y z#mvhueY9z=j2VDNkn95ulEaB&U!)XYC@{NvoYFaqy0G}PiQ7U!-6A_V=3-bU1+(v0 zZD@`Mkfd2Old5}@PN2=QMuZ^&1l&V~zVc}GXUgMCSVSr+3?rbSx$^`Q1PL#U=7N^J zw_W^*HMS3*27^&J;US>qXUFjzc?G%7#Y^825|JyU&dpRDo55Q7wK=whx!%v~9GWrhVDWFYDXgc*E~aS=b>%ejM|f1hcRn zP~uK21;kSi#4^4BwV9^n>?5IlrxoSMfF!J82jy=+xa}K#mwoU13P*P;uf7L$mlNlJ z2)AuonYc{mq5abXYf;`o+40+1b^yXAot{6d&A?t$SbhWYel_j z!r0L3_n)nmDj<_;F5s9n+gJvMYNlwJw(6*uuu^8{e6+-PN>!fvyVB=xj%xlwaSjsa zl|aa55{I_)ncd1*Sq`l^<0gXWiO^7vsg}^TXbggv^G-66)pgj(lb0idFO=>L2iL&; zg5z~w2a)n9SMSC;4ar*;RK$S;gvT=JlHxJ*L@gE(^O0Y{K9=RR_R!jmO6|8Q`NP$- z&do=fqdz*H5cUHdSLDZvwFI01- z5@)mc9IFZt`x;3uWD>87sV(STl2&Fbx~r~YTsI=CM?)1W$?BCTPPyMR7u2P(&C&+l363<-a@ zuEfdLhjYwvt!d|N;wuqQ3+*PJEuP*JrTl^kCKuTIt}{hwX?tNTlUnEyEa(G7i7Ty!PWlw9AOV$Z=nK)r>ibRl@yb zf5)EUi-oEW(DXA6?#aWP1Z^^ z$~Y*W$>bsV4gEDEJ3fJ$QnvUG;gY>FG%@bRc08@f>NPsM;&C5%Y9x z@%hj-C?xl)=3VcATkV{pwyPekBIQ1!+U?5aV&yntkhL0GdL9cIfnDHnha$AeLT2Aa z%HA#4#iV|tQka!z^D(Tu#s*3hSE0T=B?P!bsq+z$Imq_omm}{ZY4X=FNW4n6{o7tA zS(OIqQ*{_&I{J=+o&o1dIQZg6>9)3`v$Oru_`SE7Oumu0Yqb*$8IRVk)DEQG+A_K{ zgp)a7Sd$3eC8;sRGNo02<5${kAKcCDaA#h|>M2y2oKkg5A2`eD@IY%W%ch^iBzl)W zs~Qg^YJuUpX_tb&xhL1+78rL~LU=9sOWI5~4haksCl2`Hbw}gJz;@hClu!L!)v?+| z%Ni`fioNT|{;hAKy6$qO)=s%1oGI>KX*48|vDH(HePrw}!s~zX`$sH|TJDvk8pqdv z5_Ek#HEwnEf!MUF$5?NzRQvMB!l?#Bvs)du2<7Ze2C3F(fjz?5ffKusN+Lb0lD0Z8 zmb2@|q(rX+2`)}X8khK@{qHuj>m3Br8neT#{dS&o$Y5X9^CgGK7=j=v=kRX4yv)}v z-rYIEPk@uUforBXT%2OOeX|XF$_zc_4laCBnc$59L>!1T)>Sow>dW3DI)n?J2GVYe zECdUBo-j&r_fYEMROfyipiUXiYH$;SZPU3J7STaPy z@%%lRL-vC4hrU>}jBkcHV6rsOr84IdC1@Jnp&?=g{Pxh!fTRDs@(huKa-e>iR^06fsd`>v0Y}mb#8PqJBox4`noQJ0?r=fjq4qJ9&_R9i5ixPl614w%#HN^TBytSUCV#W6YC!alrSul+ugePbnhK(;`Mbz?V6y=+zT z?bNIL9!zgykzexyM|RGd1jowJw9t*DcLh8ZS=+27B(%_gl zkSs+}K}rxE5p8`TBfLeFKh%<6hYzy^0sKL#3!$GKhwY5sOY6svwIw~hT^hd`=pJh3 zq@8kskZr$gLnKvnr`S7(74IQ`$T)e z&^sTXPs-{^v>dA$4i-^;Z7M_ff4;w-?v95p)sVH3z3B-WM~(OOp0)!Ef6;D)&CE7{ zJ|`vPTi&51>-&jv;DORTGhLy`5yAvsGda3e^m`C}yUzfWIX@~NwvRw;tkjPmyp$St zxH`R>>+0>rqmv>V3OWYwI}n@`T)eO+TV5a}RP*R0eR!(+S;;xOF{B>Tc2i6Gw=I{x z<(*GoAPZgYl+u*(T0awibj_29@^ln zWb&O;FKBdb_%_{U=A1H4mPC<|1g_f-6Pe!a?NaZNFcSm1ppTydo7}A$&woy94}rwd zx~wl~pybo&IDC{iDSd@#Ijzb6j3U6mq-!-d1c||Gy#7JFwbLem8-5WTLwZoycAoP%nek?GV5kb~q;b9TOZSelP?ptu5f8Ox3KcF9 z-54Dd#8>tv2v`qsGg|f-(=hLInPU5paa~g46|X)BY&Egr|62AX$uaflH1p zdH`i0#>+%3e({ct957)*y(PQ9RKCAvGSJR zWdn9T%Bs}rJ^o%}lYZTJ=Q*G6GG&D7#}Li+I2IL;vQW0b7-62G`kWr);6e?@+{?}) zLpQSJm{%Oi7hg02@osaJFIA1Ikz2Az-;EQxWSXC~*BU=4jn~zT|Beb|2J@&tXpG&G zH7z-aub%R=*Xri7$T@4Dw+MghkXpFD!BzQWN#r`<<_bA)8#G2z^IpS=-+qCZ^XaEu zyytM!1b;0svdc!^7xhBrbgO${Gx~MjJ;$-U@#T=C2hUzQRdbK!&E;5HcAM2H6a$NQ z8aT^Y?@C@O$|k!38Rd%K5F7M$3R1WClwtA}60Usw(5&6#`~>!mfM$UpAscU3xQ1>z zoFEOQlHsw)u{=-hYmGnORM`?}V5Cg6b4Xpk>}2Kim7V4$VdG8Wm10a)S_^Z&kkg>6 z-YlXMiwEni2D4G}G9`R`f#jh5hMm3WQ?+U@3rLFwXo`9&7BD30CpmlilskrjAKlr-R zXV&9XzMHgyRsqF=vZK#Cr{ zKf{2VWG)KryP}f7pTx08jH0e;yU>eTqRdp|$5z+*jnrGPmmu0Yz@a8Uf~|Ke{^Mru zWme*3dV`7LjS%6{Rl(_j0J^>O}_A(&75O<2(5dyU^p6>wwq+i~vuDm6wvyt!J- zlIB*r!4SAjroBY2UVeV$X`8=x0RpE{<&)wRM=FB5__X4a5r!n%RwYMyHCrOmQ%#yh z-wdX>EuKzEo|p9sP8>B*(o95%pS)-cEK%j!*%~VB5cq`D9>E(r1}8RKk=bXOO_b|6 z`_{Rx#kPxB?D@ssheKiY85-$RH6R^KZHz;*v503%{VRGeGnyqaCTfpx);Cu;)qK#m z*Jdx~@Zj|~>3Z|WNdJ-vww&aFN;(|wqlIi|IF)rbNPVoIBCO^$Ojdd}T%NJknpXUZ zJxo*HuREHdy!QE7CamVf?WmHyvQ`!v1?>0kR%`_b*abTDE{|V^*9+3F-TT>RQ1r?D zklNGOx=3xT;1c(PA;6^K`bw{YMy>n|R~G$@$xz1_-#d6W=Ohp9Tes z7vtU59_BH;=d%9-LG43#SPsB5=Wo;)wJ7i=ROEK!wb+o>)qAkT29T}6Z>l&AW(6vVA? zHcPcje;@%%1}Sdlt46^`g30pt#PkO8_#qUVq?IO@MG>O@gq%5Kdi=N9Ov#VMeN# zXDWybd4}K9I6%T^Z}wl{>oLrtuEIH<%QmpL`Z?L1QtP$f5sl+w2I;%%=6 zY&pJ4xn5ze7f+{}99W-O{E!lrPog}uGyZTe?UY>uKV?$p+cf+}WejCLFUI3xe&HK2 ze&ABw(O}bLpqL~jCY=`7z&ualn<1*+&dKqTiq9ml$i5iUx3bKmtMQb?mD2jg0VU#H z3VN}+9re0PsA}$rGope3Vun{f1yv!JZ#9i~OuhDTKPT67yQ}Ze5I6Y_>fn;)T_48V z2~>cDY#WW&+doG+dfDy!26JL@PmCHB`$gxJVp!Ra>XB^?wi93?B9}viTvK_EW2sp` z)6|ysH<;p?+M*6c^X4+gSrJ65#8zjcA72<3n;Lgpwocnxtl65TQEA9l)tYkG(w_t8 z6<@!O%9tQ-shE{;FDJ?eb$b*X#+k-j#=%m#7S2m6!N(4|xQ?;jbMBcNdN5hi#E9i< zwZ$hKtu)>YrT7XM$l_t^ z#FNS=?{-$r8*;n|xxKGo%My^A1<~H8YhlfuO1ewvV@g=t)~M_n;RP|o z(s`lsxk^sfH;`F1WRS4+6&y}N@`P29#z54t^af$e7fjmq6{ZC|s5coI9WyFzrM#E* zFh-OccMKPOG$XFFUzA#t*=y$EiA4{HxD24z9MPGA-olQ*f~0b=iFOqBIQWC{wH2IJ!^TO-yJI7>UZiviFcFj30`O4 zlSF620mxQfYcG?KH5s}@L(a6#lP2ZU2V(LfO(eaz;RGjNa7!TQZJ+fCHw%W>Km)~P z79>H+t~}Mg>;o3P6rg!ejm()L8_9m$$JifMWT^^{c#S>hH%9mBJ}whxYWhZQb&qnx z101HLQ<_Z#jW9y9%F8SZt3MQ(6X^1%8sy(6|6u!W1Zu2%s3Om|)9T9(3p}(V6Mplt z2nd}`AZx(x&vEhT{UL?A;ukYr#~&j7q*-Z-Q+-BPd@Z4@WD0Z{bhdnY8A#+u$ykxw zHJkGL4-&CANQy!iUm$powiE5HY>;?D6}P$4a;7TsDZL4T1V&DzIWX~t;0>u;h7+-6878F#R{uhsFnbJ9dth^H?r)v!c`V2b{$RDTBpQx}IR zRV{X^?cTWSk=OzG;e82`Q&MAWJ@Vy&qU^2E`T%iz8l|DI(SEYB;-e6+$Frc#PYCH} z{~a7-ym6;Z{vzE6rb3B~Ic^_l45)lLBe6VFc{gx2ZsBZ`U2Z3LNdW{J;d7`N>@YcAvHAoUDx7qSFI8~W^1NhjWsusc%vIxIdBUI*94ft-YoP4)5VLZ*t;tK3 z(2lB|?KZ8(W1i}8dMtYMt0h3kegNluU0tJxnBYZ!P&a>2^g&l3BLV9Tr&Qu8t$F-q zUN#xiCr{E3VH!+W6_YtEdd>41==*s-ATai-{QVk(*y z;Ve8Ny9$BikgXUHq5m+AHVC^jylq6Uv9*l##w6h*f(Ya&lPC3_65Qt7h5@{x&W<9= zlxln2`;f?2Ig3S+r*(PA1tG&}8i-V@YDJ4;y~+(~f}T0I%VLU_8&b(LMaoOc3iD*C z$CYA0`V?=P;=KyE2V*|&Fu5-JbHY|4&2i3pNc*}GKd!o>pen`-94IuA(3=6)->39mm4+}nfl#RUju-W-~!;E>btX1FN1M8e{RdrmeZt}xh zHry-w^a%-^46)hzH;|T54*2sJBy|PkLpRbS+MmsT#r6#9U@05B&c;rKG&jd?8wdH|O zXkK-d+)2uJ-Z9Td!+5cSa^1J3u64Pj_J2*t);k;C$N>Ag*tchyY2p1jAAjEN{Is&q zvVj+OwlgO$5ieUp083p` z0=k@@qoa|e7Xt3nbf5HjkGF{G7HPR#ve+?loA@yi_aT5!xF?@)@sEfYwO+K>^ zTVJ?d$_6RzEyOT%bB!!|B-G*x!bL4QU2ID6vldo58MQQxxdh+|@s zeIAT-P&h!HgGOmpe1AAg>0l2y2DH&J>oK18@WCz zLhe}oUTFpYOSd|gtZShoWQl~if)uMkH#;-KlwSJn6G-^;;}A0W6KnVr1yd#SM8Tag z2x;ZMa9mB1VB zf|_*z#RVEsFbY7bZqf1i6J2d8cS-np13?q(mmebyt@$CT#^uu#MI1i#9m+^}ZTTSo zog;tezWNA@G~*9^I~D!r02gl;|E8umtj8L!7_zL})rW>|d=k(;gosz@g#m}Y$P!IO4GZ2#%D<0IHCXI#QNFNuHI-TkA zox44gIcd3*Kd+qUy23s4H1MITDB2yyXoZk8$9z^oJwA^vuAO;UDR~raqwHP(94>U6 zNP^86ea&hF|8=;a6#$U&XzlBn2N4kuCG_|oUS0f5enCwh{*L%jq{jIDy7#-OueWU1l9<%Rp;NJIY-D4JDAKs0Ux!e}Euvni0DZk#i z9Bz5B;8|Wz#~Vk6@I%%UxvXrCA33^)xQGQO{xeKGiUsR?95y&` z>-v)W0lQsxsh3J38w+B)zI&y0%zGY3c$3Ca42$oh1aE~)aMgAmB@y3eP-FHi{_J1Q z7v3?CWg1^9Gi_^fpY!}Rvv`Y>m_D{o zM2cuqYt>-)*klt8k?Z2JU>OQ-{531<%stgJ)NyZw7_k4nY^2^s4+UAJex zOkdxd0z;T~a_n{W`d6K;j>g9y(zc=*WZUJ##-9}IG7CbMo~@+O4;$qxd9d-lLOA=+ zM#w&h)$dvvGf&$%9pX?kUyhbr^)KHiP5c=?mm|Jd^i74^10uM8Oo_KI!d1O+L!!{z zc6{_xV);mLT4}d_8N7sk19P!T<0PqCu3$gl?vG`NtLELfJ@29e@!*Fm~8}sO=cX$SCM4U7Y40(4qNu1s-=DsS$67vi!f&%3 zZD)|*R7QHWweJ4T2#Z-~K#&W>7kY2oSr(HdlkqJDk*q*7R9SUz?-;1jx}Uc)H{7C8 zh{`ZMQC&NmxLcFAVZ#F&q`a;rtrta( zVf%RqdzeV6R|%@MmrE)^6BSL*eL33_aN3z>B3dENFKI%o7Z-WU)@YR8Q}uGOH%V(Q zVlYw2!y|+@8II{=k8Vd-Zr1&ClvOsVyOp;&cma}hY1kzZ8rm8^=^Y<_TP_jwUTan+ zP(cb%a11KHBWrq%_=3Z9RZxZr2pAU%t@^&hJ_)WY~R`Wc`8Zt>TvJ7q3!zuSpauH9DLPOL5{KcwX{w3`Qt~^B%o>NFw74 zWiJhe{@CN;l{+uqzl5~v%nzw6A9$Y}jVn^kEb>H{Q@7C)SzS)Q!+#dKMq`B-#V3qr zR5U;H@5j}q*A9rn9Dq=6s9#JGqQ*d?@`0m@qzpoe^5mXXn&6ui74A{&!^cR>FZZWq zy`bNXDCxbgS_wa-m)N*h>CtKq^L98XkhKbK{5kj`6(7fW6z1cF7vz_`kJL;L7Nhvm z1w87qKQ8$k3e=Za%0tCyac2d4%nsyIrjQ8ecmmG-;u`o9c+ZRtf8}%5mhhk zJoCu~M`#a4{-FNS5g}hmV3Ne$vf@yeSaGG)66FH(_eYjd$D5z;&=wl84K(!ShJscSHMbtnuuG-Hx4lyEZTgftfL>L66G#?q_=501q{CXpSs&yrj#X#n)TmNEC zL;y;*Y8@Yn7KJq}%AR}E^E}6}6El;MnnH#f_jwFj1QFRs6)k*9!5;iqLK9YHI;o5K zvcYjX{I4pfZODw#sguDmy1mFtaSsyb#-$~por6>}X%~xgZ|-~-;c$j)vrYFU9RvRf zwO^PxZn@dZAFnAwn4#!eizajX2Vj%Yc(Ch9+rfu4rlo* zZFv~lul*z?e9xpCE&((7oog}4TuI;Q@Q!wTK|kjGI%?6x;$mM%I=21nLdFFxxOs}l z+EKw;Dp(n<_e|pjgS~RtHT3v917Df^xWrQ2{!|7Qy9+NhjNP($$=Ld?R5HI=Pr~)8 zXNC$Rbu@?Gky(s!e{g-AY7J#%6-{`sdTn|6<}LX2?tc4rVCim)T6VTmw#DITr4fX2 zf*()}mJ`=N3W9E5UHt9fSQgD3q_D^4ceRhiWhS9;yW|rt^5&8m!du7g^$5F@5(E#M zPkQhLrGv~USK=1M7?;-0NTz;@C&y7h9{+6nURDZz$mq_r!&mx%yLH5KoN0cGFsU;u z@|Ktjuq|Y@@Ohlv8r>oIYQ+8CHZ&pHIx8_fE271Ociwnm!< zTL!AK+t-QaX2GHN!eCUQ(+fyx{xgzDRXnbScU5FkBtN{YC?5uSte&$8%^LR z8%)n}e9V70@rmV{pxmXsNJq0bW-NI#;3{+)x$~ikkH`77d2F@Xucn&dw$^($%~a-8 zJ~VLI?=P3=%YT}YI45;7y2qe}vdzogIiJ<-c1d!wszqu1UVD?=Az=4n_2=@p|yvp2%|m|*XIvK z?&jzCWlffx8IK9gnsjjIJSEJHm305WAqU9i2Nx(*N3I7!FKM*gSt3T=YAS@yb!}+p z`04~@?L>$%>o7b?1%IqgCL8OUy?V3xpe(zdgsOb4tj|v772Kh>wZBN!FYMnkdHEQ z{1)^hS5AYcW=F24iJ;8Fv=i^v=G0w~qt=!P3Kg++?91OWXe`vb&)41VFi{J0yfwrp zko%4htNu+Hw&4SM9}XtUxVTOkYhtcfbo2(blSg1UGcx7~^WJ8zGHp{m5V*Fs8F4WI zoq&)bGNGNQ89#3}vj%r^%u>bAmoVpFI^5GgXQQ;G$RS7-cG#bN!9NPguF%b6(k{qT z1kHP0W#QAfM<7UP)_2yP6zNKXLbvUdz^P zVW7}KRLmNG`SyXti|y5Hf(do8JG~0RM&?Y)T4yd`JmpOu4{#vec4huG-Xo=-hl92_ z@?5+>h}~8~q~l&2G?6pdq)2)#TboD9tY%G${thOJ6(iXd-5YCj9^*yBi&o(;sfnEA zs0R-^%jK+a*T5UjMAgSypTJ+Qi0i@A@2#&0+) z%H;QGw$GIsrIDdkz>8^Azk_#zO6dN45s&D-1|HbOY4FnNgk-YO>c!08AZi1C!#`Uc zJ|(qCXg(?mZLX6F|y%DY$zwgDnBiGdp< zcEy$9TV8d|4ncW=+RW$!*`>TXOv2RERUai-M)eXNw!@Y+DIjM3x}lVF{S>wBgT4sL z-uaR!<>ngIFAd>7{OCmbhO}TJeXuO)iG^jTKWt}p4km#gdT`OVV(OmU8RRRiEe9th zIHme)Y5uLQP$7L##+7NKHoaoT)=c7(rWaI zgjmm0!lu_~wR(AYgOastqCgLsG?~F+JN%uKmgkF(=@}j5>qA;EPv(5`TVCfzeVfi8 z73D^+@?CnRyv5XJ2lJ0n6 z<{!e-kQc9N!t=L=IldV~w81bcK|qbN)D8f?~|dY5xQXthNscXNW&y!TGb zL@{43i|XxGa{Yq*Oiz|~mkk&9izdUBU$nfoi}!zEFz-L99Tasp`>D-tL96HnnXueX zQ?33)J`twEWivPehK>*$7-8Tf2&l_v3y6Z4b?t9nyuTS|Vsc z1SN^!m;UZmC&5Ba>MRraGRVc)ba~`~N@*R66Q>#$UHjmCe|U(Cg(+2Y<>Te7S~=pi zyJsS_#fuC=US%$+4#j4yk^D^K-N8-WzSOtX%4{#r8 zh1#&>o@@5UZui5ft$X-!>s=>&9+}0iW^Yu1EWw`Il9Eg!Yi#fil148SOc8pIw>AZ!~q8xnYtjP&!=V;ragZ*Ho(*#{0eVSX1bVhZq@^*fnv`F+)qJ=Nm zd>4(*7C#JtTiwIaJ!;I=Xi7(qh%>$pH1>CC#Q3P2jhFPVIXW@W6~GgGV;zEMq7KhY zgYWms#?(p@^TS>|VG^;u16I1Ur^5C+FeLfpqT`%$V=RConva{ys}%H?4~yJZ?-3XOIEm8f(v=nTpQuu75TjTzZ+aM%KlN)}d{JPa{;V6qPJjD<^U_n|76AnUF;+ zJO)gDerLQP90=}k|(&a(SfrBQC6RGk>_*xeIs+{VUkQ)j^P1Z@11#9?+FI;QvC%(Baz1`&%2&83WVf7 zuD1PHqejU{;hddz>a5M#11=NwwKq~G8JxFhw)jqz!QZOM2VavX-=lru425Y_^MzS{ zr6Gm~@jMD$F6_#<6B0F@q4u{=>0jmUWG*md@i`lfdUSYl_LpDreCvH z^|(=Dzsn(Vg2O_#KLPV_90FBCB1WP?zPJf%bmRMFyUySy0>5t^Qv(#m$Dj(6GMX~z z8YUqXm@rz>56^jM4T&Z_^X2ZGb-Q12uYptRbG6yJO|m)pL*4jv;OC}90x~eyWX-|| zp>2ny_9-ch?-Y%Zk1~|lID6`zn!NN`-HQA@Mm^=!TeZy^3Mk5y?Q8N;;a zT<1I5?KYA;f_QYQ^hdBEdwHZAIMZzRgr@O7c%wH-AEoG7CwyPgN5)oh^IYAVENXJA z+=?%;nr>F>JrLQrFPFcq$zWs~q2`XRJ;7F_{H}wcvCD=*?)`*<*WC^go-473!hK>M z2e0V>9iyeO_l?6f4`%7`YNb#2p+HaTs&r@|8O5I5W6(*pVS9(!QCucGHCS#Fv(d5P z^W`uVeH6b5(wbf7d`2AY{N+50$CQ*XhHq2sFrh;gabnLax^0eyfSGF#U^|0OVojuo zMi~5>BB%~kexWClZ*!op+>f#AwNBm1o^%7yBjS|D>}JeB8TIPBP0vm``io5v^R4g4 zy=jh)LOo@#6GE^HJ8ubfvAW0xX$|CyhJ-0LG`B;f6kuERWg3eqSJ`Dc?_uE*GbU-) z&7*Q|Gi6xeS-mE0CaD=zOY7}Zy5fRDMe)ayQhTU@v6|MA}H|8ysvonr@6Q=q~&W^Z`9%2k1U-4c*@EcQ*1)CrscKEo^h=dEL21 zBOhkz-Cuud$O_-Cn-&z}z8Vrk_5qe=&5lfAHgf#zL;oE6eAK&W#NA(XbsWVN6sWy) zl{)wWJ6EKF3o(lX8wF6 zyFB_OA$!-PNnbNwAyKOH&~B{bobU#}V_q0wdO6GJ*0AYgM#P&EF%F%oyL{IEOJ6Ft z1U?@K+3hiWDC86LM8=EU+JIl#2hGj?ezoB@`wJiN&=xtGbLzMK`V@SEriHH{a`NPA z_5w!)VO~Z&&j$#vVhlo`^QrT3uPMeptt}L~j<*1uJO7dKOwWT0Pak^#Zlez!8}O^I zu7k5s4BGCAoVBNqYS>{(PuRwlFPSyDnWA4|gcm@M2G~#*JFgWH>vSjFjGPKA$cTHg zof+T9qUe((ZxT}lOg7%72LQhSUj_wBtu=qbum^TmIZ~a{ZH~>>z*e|zLXg??9?d2z z{m(p_6O)d6X*PaLdpnVZVEeD{a}rem%6d`du2ug zlVW$OFYG$kIVdE=XyPfanF`wKvP_6Fa@Mo=cT;Y{SjB2k*$Ed*HiPM@^QlP~b2rCj zd$lf>FG<$2E-@~-Lo0$JqiFe6FkC`sj|mxYqt`!ixRLGbvfDaczTC2`%VZZ*J`FB) zJ<9WUh@rsl%n_+=q7{~*zx2jgrp=bhIUP<%GemQT;~b6AHb?7btT!<1g1beS0@G0< zIX_Wt&nT>O%{N#i@ew=q;Kt_Hu`dMqj+>iOtjX(|$d~?SNMqlIZLJl>p{=D-j2C;} zRNLb|x^i_Ul+ZGx?=Eg8Ts`a*W-%xvxLp;n|2d#%x{t zocj$%e34-=!2vdL5-b4W~Lh-qVG%0}%!ToAVju5AFfWc}?EP+TKxJ#3|##8MinYRw3KMQ#jH(q6x;V z|8_P6T7L~TeNeu)km>ix`zQ7;sJ1+856y&87k_6rE3UvQ?Ppbh_HzGGR<;@-_OQ^f zDa)a95A;Xs|AvV`JSsyW7n% z;sP<8^CUZkGiozoBivs;d;kIqmgaDBdzL)kqbQ3Bei60W>&$a);?36rRXt%vLv9V} z7c7Ybujo<(~l2}cFjZQ zUz83>GTPuQm7t5*vDZhQW0aeMKggrp)PKYD?bnS2`;j`S?8#hDtJJO1hK11%-&~DP znLi4oq!!Y6y3-qfNg`UyL;Wt@Ik@v-iMctn^oNiJTa;?ry|#w4;To`P%GdVwH1HF4 zLND&CWiE!i?@q@YNkVY$Mev&XXf!gElf1#6>8ms9{9+(fU+a_!$PFKr8bY4EZQ%iL z_-@2R_d)^N%C|2V&Iz3=W_IOR*>^9*-nWHpnhp7V_sr_zoXBv~uoS#e&pbhs8?OBF z3&Q1DcDj(+dPz%iuV-?xNRJ%RK6+GcRp@Y`oPW3ZjsCi?24%5oO7=tVx-GbURVsT+ zzghg%zV{FeTBN09-?8ubI$Ri5Psgd_m$;SK_s2Ps9$%MnzHnv7WTT`l`Od{@KLG|C zCCeEHC->W&`JQ|R+XRujdw9nD6%|tu0gZ^KonWzq+sL?&)p%v9LSjMQpxU(WB5#`C zrIn`(__cLjY(R`hV+PjH3cD3ssi9A4y7s)*q$ygaGL#e1@5V-NC4*^lk=)|+DUGoP zlLQI^_PKspU06YBRPSp$blDA$ofjFWjbIbm;;U0ULu%tTwpF_`9>DSi{%zJAV_(pf z%kWcuU{YWewEWr@@3BaeXJU>lmZqbtmpn73H$?mf3P8Th=cdVp%dL<|l!!NXyNLXTlpw&y7Y5j^<(OC2jk| zabRpYY-Y7hZ>n2XehF2W4-~lOMOoRGAaMDj6}{Ww_D#6D3W^9u^;Upx?};nQ8On0` z`UjBdV}%q2Pro{ z+LgwWmepTX(VmD1k3zpXCfdXTBsIL!-fNLD&VlRBi|2)JgfGwq80mbcc6s#lN`hF~ z)T_2xp~Jl|S-Z}lA)!ohxkvxeRtb37r2}3y=FO4_!SW^!zcx3S6myz-X?j50j$<*% z4{%I8lU21JrJVn)8>s25_Ro0iPA7PeRCw z0O2NZ=XJu?L6my9O$FaMIn~QN8q`Yo0JEke@GTi9i&W3K*LB2KfyQVEuG=x+E^p!? z;+fsVd7_;$_(s=Ok&;c=W%tOZ?h`;&;j#0`OGEXrgxUzv@*g7}SjAC|+q?Re6=1*w z-@;|<>GmHnyp?0T(P4Q@_@*7;Cadl8*jXqrT}@C# zEEYJ`QrOYDFc1S&`>#_n96-(t)7b%lvA;g_WRLOkq1rFK1Gpm3n9Wl-R&mE%Q(2T8 zg{R>3&l@F`IN!SeFlon7=E=bofDcU(2~k;-Vs;~)HsNKj11CU?w{Mj@1bPI9=pWqBM-~y$ptRZQx-1a8%=VnJ@R6jVPH^lVJXVD1Pf+q6ZT8dDJwsOg@MPnQO8u1Tg=-j#x}*0S}8j+ z{rSp}TlI+I)~`FR4NS(7V#}3V;^S3aH-rq?v3s?0Ze~r`fcVOlOL9cGnQae?i)Ok~ z0LH*}sXrf=EC^EtO~Fc*cN3+eFJLPsqvSD9^E9J5qI^l+LO`oRg#004as+?|Nj_}5 z_b}rg10cw#bsE^`bAFD(yVg74;VS@+-R@Rt2f3u-;FseS`z<_6v)BS=1YI6g9NbNFYf0JB1{{EbsOlv<%Q%t8TF^(z zN9{lnvH-TZd^oiz4CduXu$h+*={aU%haJKDc zrkIv{UCE~sEWDCt@8De%4g_~*!`JRVZul@h6>jac4mkmp@%(6J*Bqn+nV8|uT@)f# z+q*$tcThyiGxADo24ELm*nv)s%irtdP!4j&7Q72+YFv8Ff>-4QrM z_G!#PQ7!yS)ktB~YVJg(_j-~;8Y^eZR@EFP9Rmj`kKc&nKb0TG`8d9yzSEADDh<4_z<;i09EJeKh}l#elwu_>uowP)@LKV3nN*w~g_^BccAo zN^oZU<5(G*-F5%C&)AKu=Id2&B?=M0{!(sH{J@&^Kqy|% z8ylzg^vj)OfxN+onPm;kur@<=2^=pLlJQdHyUQ;yToXvfK6GWA>C4ZTJ#mE-R+;QU zaQS&JtNRMp&DCAC%}b7Ac8@dlsB=(2k`OjM{WA8uSB+TZ<|{k-F;F%<`1pJ|B;99* zwR%K+h}Ib8014!SW@-P?Vnitj*2}Dj3&xNb7_N?Kh`VneQfhjr(KG%?NNN0gFCHtmm{mnH4+L+1 zOOt)M^#S9Z7XDy(wkLe9C4a=~L#?FSEB;nOtL2W#M^3k3xc=U}KGH0X53y4*x@*TI z@h+3{V`f$BMj#;;r_EySeys!u<&}q@{Yk?nEv)}!xkCi#jM12Gvz1199>ixN!S&Nt zwS@N&v(J$S)$Um$DC(PfH77IsoMjE68N{+Wj4ECErF!o~~KVs5k)mZ*6&Od7xh~elL?p1mY zfL{-qOLzHpn*jzeI&Fj8}qy4Rqct z9B?27{ct6sFaTBx|5nus*SWL51Ih{eicnBJ@bE}|o&Bd*<1RcYh5dv8)M@VO!Qh8_ z8b;$C-%hERMIAFir!NE`C@eVNS3r}Kk~iQw?1^CLw;RR*-oL@gP%L$N{mI*PR~;Q+ zP{cbIKSaY=?Fqs|!;#Bi5AH1wKXl<85x%S*9(Il4Qde!!Nh5z%-|Wd*xnFc)Med7j z^;g*$QLyuF(k3%EyY90h2NpRZHnD&Jj&Rxz*HqEOog&Gvhq|0;6*mp4MK3cg-vlT(smmf9yc%KuLfD1vnc~~cD zQP1hH*9WS~#apa@KAsMk^xZcT#RI{_xlj>P?|p$k@b+kzGG2FY$JXEEI^3NTU>p9H4OGpcjOYu@z1f09p3*0Adko z;ZWpI;rd_GcklP~eX~Fl8o?x2H%St+FT%Fc^_oVFgrek>tQ^iVbdG(LHd(y!b_vh> z{(Bt`f`a3%5I%jCe%{6dH(DXUa0Mtfy0vttZHoTT6W13h=#OLuy9{xd`<71n=;pmc!)XIr%?It*Q|# zAd1{clblyDMu+Ay-q|yTrY*Bz2dre+z~P`@Kfe`sRgw#O0AoBB&h}*WTG=hSA#gq4 zx7sGuRlY7>DVnj`F0fAp0DumzNK1LE6~sK|_Q^IS3*R#36oNfC4;lhBd?O>IK@3E2 zAK3xrA8(+qS_*#esMz;0in!(2OoV2#Cx{D6lkUH17O5gk7>?Q#0hFpHcUOk|bWiRd zb4&66&h2nSi|c`U2Bt)`!amx{-us57wqIX(fs^BEzAkcL12NF8j}R%$9+hyckap$H&1iV+o^O1+rTn(8+I0) z>nl~u;rc*nb+Phk%yIK%7j)9l!_4f7T#rg{JeHmAu4(?g3&H9_sQ2PMvI}}>vZuOF z9^#2-h*fR&)XX=K#Pjb8V9ELetvQ~1hUka0>{ao2?PSzyC&RDL59jJdDGXt5^Uq zGvk)J)5?MTA!Y=okdI=1gu3FEyE3RN8xJ9R6v0v$mr&+ht2VzX zYy4&D^JPK5fnvSt79-y(D$VR&_tb*Hz3h$q@DoO!u&~iR2v><3m0aC0XF?K(U79nP zJROJ?P6x2X@|%EuBMml@ZxX=mpm6=L+wBkr_;9l~8`76|VOxG#5aG>v>Kpb|Sl*ka zD_hOq=#Pg1^hOIrV6liE+bA=yw_5K!4;l;%gncMG`^S7vx!D4pJ)<6dE1zkmYLNvH z?M?&1?sUU|gstFWEeR*NCzfT@0U*?>Ly+4^s#UBiJ3A-*`9l#;I-kMdlsmrer6^C%*ws0tVQVt zt0OA2jjzb9nJQ3jw2!X7TF!nn2yPunoEc&an_VAone=kqNFZIQ@l)Aj+N{0lyP=}f)5Y^KA-%+S$4sI&O+524`hlYv0Xc{pF=QMUNVlkTf% zU6(6)mBXae!krzh+<1|~Hc3cN(-Ts#WZQny|Db$Ro?-4JIYc7f0d2SUs^UPNQ~}`c z`*KI%?@!c1Y1^NO3bbG8)?Im}Biif_W&-Q~LGIS?k-dHK)N{mbKomWK!u=aT<(j(} zxERLoqH=X2z>88b=FbDCZTUZE-&y57;4sEQj)1N}s_UCv+AOp{^v#|C#XTufkjB1X z16|+M*`Tky15$*4%qXDuWU?vR8)(4A4f$`hv*Re##!|9uz;WQ7nOjxv2Z}<&s?;J= z1YL9JaPR|+rxCziS@hjr8eq0ilz2R({WH0siE|_z_s8_zy8s&EVdw}}w_6$iP9*vg z$#cq^c)`bz0C%DnRTfY-b={o%>Du}Bn&p{aK=4!Sj4$vJ0M#hPS^k@EruINPf;bnz z9_pyN=``L_(_0Iz1MbW4Z?wa<)~#x{(T_o07rvC=B>f~KJzzVjE_M!38^0+$cLm}F zD{;n_j|5WOgBBsIz)~F?D{2N?yuW{YBb7-XF@AaC%HiCT{eLYPor+Rf4&ay9CXGAw zTRA-N@##cRk{$thP^D%*u!!u1&O!^I0lNYeA-yAn3&P}qzT7a-IviPR=e}(Y>DX!E zfif4QGn!ffNv{H-OQI)G&}#jx9!?{mt9&N#6(a#CAyj4cXu3>>&+B zrQfgZ?={!wPLW^WSvsYE&GEOayQrRZ)YTbx06U5uh>B>0KNB!w2v^ajYy*0=mUDjK z4k!l`JK)C|8H)h-%x<*vC7f0sE*!Ijm-&9TngRTxK*!Xl_9Jd1qI!#1y&bWCsR`bB zl~PN+|7u$BZSLQn>m(-;im_$X?$XzI`->b6NVeWUbh(0zR<=e2%$Oy4Mq41jZt|-1+wlTPvHhKZMxf9=`Yij$5Kk_K3XBv$Zus%xRWN5OY`t`U zh{~t@KAslv|7I`O{NjR)*2{q`{a^rKF@tJi$~k?NKIqB*_!D@VJ`sp2`vKjRN^v_d zrga>J1CY!9e)|w~Se2EGB8I$mp?tdf#bcWoV;(`j1C0U$bLe@*v+47orPJUz%{s57 z@h>&?PD}u6>5~0=SMA z0jxtUOjw_?^1^4&ak-o%*|*r{$(Kz@=RQ zv(OcohkHOJjAW0FtBbf!fLl&N%Jaw$pipJJGn_-<3;F^ix~pA5==0-=9~7TgKtKz0 zpM%nkOK6xol>C5-cj?fB4=;wlW#%jlID}`7#X>o^H=q&yj!?Fi1`5aeGhCkx_W`l@ zC0`);wYR-IWMd1I(S&zEWu+WId!sISwI|rqMtk%KpoB7P?$I7Mk0u?OY6eF2P;dOM zc{GROFrdNA-cP~HZ{R_xX|6GnbrR~GtO9s|0P-{T@2)030WN(MAM)vqs=t0;zeVkB zGZQR*o1tf8CoKatxr1HGqR+)^fLLD&>_>(aH#^mmT-mWslo?Q)wV1&ouCf9K@Yc$( zHOScqgjJtGkku;LVE^cJ8YLB_R)rZHBq{+)mE$37n7Kk6F=of-^VB?~myP>sv0ed4 zbE7opWyLotkE6rLp(q5c-2Z}2rh^7(`LP3^FxpxX?y15ymd@vY_h6rRz=M7AHNTPd zR}ZFQ%$s*KFgYTSjuiMcKZnyBT@-NbSnZLS9R?~yuCX_z9XGh*BR7DKpgYLF^f}V* zEAhVC13AIvQ85lmP6WZG1UMCxl8~K{RVaS;l#AhA&)m-sl_eO+$!k2*ac?WuM!w9g z`bP&m$3paRH`Zt<7zg>WtW+N%0A{!P#vD{Gd*6)Sq_)yc6t9&k=vjLg%CP_-y%|Au zfUrl&PexwGa51iQzc+3Yrq zRjI=^x0NA_7C@1O|AY-TgN@`jIR4GfIbv_nDR=^5Sw)(ZZ-j+uS zb@4UL9?hI!*L{Xr2Ob{=lGQ2aIIG96PHeq$32Ekh#5FQScUbDhfc1a(j2Vy>CIe&F zllaz7{Ydy2<43h~e%&=&R#q&f^(wn!h8cdY^skmDl;XN`f zT1Bwk0gffT)pVA`FgGCZvqk#88W`^&7OgrokD);!u@qSGypC4ldU6g(mg_soHZEe3 zhaSN3s-g@e1S6c+*JK_~X(8FkZ*y-h^t7h4)T1clL;;_unnuW$pLE{xQ3KNmsNYbC z8PxAjyuB8zdS(UW*jvc30#HyL_EOlrUwfQoGnD?mC`fFYb1kY7L=1r;-`<)m^wx*# zSIJ*+AFPEcKLOc_Rh+@;eAe^yVaFfw&TcGu4LW8>=iduUu!Fx*&(avf*#>4Po9t2u zp{=R_Ndd~_mjNkV9FofhaLi6QICIN^-h2}gG7?fI&g&0-O5eXgmj$3UT5L@7Q<+ML zyb=J|GcH!#$EJPSl#dP$&%(?>1!33~cu8zwB~l2W5?BxZSL8Z&1Kc#OL%l44<3RIa zm45h{3Wq#M8!HVY&hvJ-l#KLd1JTp*m-~IyhtrTyJGdBDi`gf4mjGbx3S=1_$E6ZOp)4l!(Y?=xQyT+n8 z@2c4}U^3=_6bnGa>p-aoRw-#^S< zAzynLa*XC37wN~T?Q+U2`r9t0;~~rSM(`{0z^b!y2cs0Q2PfgMBpiY}&yRt=8lqeb zs^0~r{cPx@9|6e6Oo`@ottTLZEjM*Y?eXK?Z+AmXHn>s-)2niK09(*lJ-KGc9}huo zjf$p>2le$?j)2WuVxQ%KVyVvkV~uHyod@k}Jd#6TrwoJf&osFCd;hwn=~K%SqyzUreheLjRMJ;KC1eLn%YE^E z0lQa!ZBwX9lsMo{Ax9t|Kb#YLU)Zzol;Hj)z(nu>Aotde_gEms-jy{AQ?eZ$-1OB8 z8Yp*l>#(zV0Hj}Dmm<|I#UC`s!s#yL0w8^IC5$)9JtO6DxWh@iwBzqh>iyxN7c5br zlYn*;Cvr&!&V^crqKyz*_Jzr(LbEZs4CwV?dKM+Hz?2kcQ#_)?7+-6)Oarr{#mZG6 zR5}`=(A)|ERr;+E^4uyOGqVb-N5%e1#R!wJ`S{hB%w2$E!Fd&9GjWNb zR+J<<^#ys$F1Tq|H`bqY-!$fXsN1MH;@&RRN5DkE8KT1}6@83;9sijASidZStWV{h z4yEVa-q|xvA9nuk&TU`-qKdNd;o5%<>VB3ZQP&8gWH3J-Nc&v%!m}o6b+@piR~$q1 z+?8OMh}M2ka%dcFx*?Q3rI*6_X|V_KEl3r8?CZTd44f+lLB|Z;TB1D;?$aI2^Y^PJdcvXOrqU4 zbkPe9lmZ!fB;+^7@?hQny9ec?*Z4Sxr*p9m+EpqFETpQ^yMRjlTTme)|)Cz%*(ckfEoZ9md#n>`VLFJ9 z{;PmtRwcZ_e*WU*6kZ{)KqHBUgj1*9-^kXXF}DHAynfh+L(&0(RG*QQuHsGAtp8R$=( zmKCFgU|1F`_ZKx3YVKv;?o4tCV`vPAsw{<=3kp&hj#yp$alM9&duZ=OV61X~O|=HI zX+8Us&TEKFGZc)HjX@Pc?UdTX$R?i}*Mg4oraEHCzJrb#xjiK(6=e+)9oqVt}+gkIvOUJFGi8ASUVIoXdZ4SVV>UH|Zan_vesN5?nl*eo5Vi?Na9qE|~{T1j8kR_5RuILA^e zFwrb~L0LetykVA>Rr2F9?`*@o`M&tR#tK;&rvsXoqx~O3toR zaNxJWElm(Q;uybwpuh}Pffp%kT?)tHG`%QGTkrAC9~FT=e=(XIV{*D{{Yn!)Y8RMr z?)hIkY=CiFp2Cy1a9S#YFFedlUQ)tx72h`W~Uqty;gQ7Kt z;1hV$x?A~wy@6CRxNG7| zp>Cs5)wpZ}hHKn{^~r+@8*Q;~I=B#KKpOT=^(@y!#u!8Pj5fPJ14^T~>t`*N*hhIf z_+kthTmo2*umh{G9r+!g0krE9iYGu>>i%-@K)h5KYhgWSx(5imc17NLgMf&TrfQ*SSnI1?3s=r*MHT>&D_U&nJ%=k=+tX+8JWQx9_t^iWOH-B(h` zEn5Aa@2+>*=S``qoT{n1mMl>Dx0+cKzlu13ANiLd6-;`|S6}8_w_>Q6SdJB`yQB#( z)1`)M)l&;h=%H_Y?x;=X<)CEfZ5)RxCm;eqV@I8lx|Vd@SdWvlA)^a_=V|)y-}}=} zNcsiA_KSi8UgO4cW8!;!o+Fh#Ah(*?%lG9ey34u&6Aa5A^;g@&De>DOZyD$wYr8!J zhUf`9S4w(}V7o4U>3%C$(?0`V66zztIBHTv#CdriSbzE5;>6K_P|@%xlq(|Jh+*di z&|u#J^)bPyEYAMkxa_webZ))N&7<2N&iw8^rMT)3RtmH;U;D5!30(*jK&lxOw+Lp| zXZX6g7*a_31Dl*`HFZiMj5ZHK0R&b~;ny##56tVCf)e(yp|QpE3T6Ya#U;MWrTP4z zXeOuVbr!jq7EZQn2Qh*_*c5)hlBUevK{jp{rVa{@_f|DtW|vdFE-a!)m(L*9>URNG z)ep~Wsl2NutBsQbm#QKKnu88^&^6(-efCkT!$8PY*2C(`eV?BBA)sHki=c5m^+o5* zBMMcCmn4@zeT+O1eE&p({#BqApGNb`KwWVjes$I7F@YD~5hyOo1Sh`^e9dXq=jX|? ze}0y*`*L~T%u@kd2s$WpiG%wRvXxke>HXwmHt{%Ip@TjZMX%D=&H+~mS@$uOuXjrw zyFV9P@bEjrmg_Hj`0D@3ZMoCf`oU1t0Ex0?T<1QHZRahg%Yh}1> z229`T{^tc)P>f?SHtDj%4TH?eD=tAcfB+&$Eco@vMM^w=BS1S@DZqC=Ud1;1OrMfy z9GwRa_4T37kL-XB=jwocuYpuPr*2?8d)spHd~2xp(S=P}gR|KyoJ++g<{xt=h`@Rk z{I;{Mf=xmdNaLm3hjthk0k+{%`sqp{JMvgn3qkl>rzC)t#U8U+xuq~DVtA^M{0V8n z=um-zo1ij3IRU^KN_C$k@2Tc~#UwmT{7RdIO}bMtohD$O_z>|mCGo~R{c{W%7Ps93Zg$GFIy+EJgRI5>if;U<9-A`4KmC}^H|8`&zwLmd zHi5pFMOdl3%~y8A^Xqw5 z7wJFCE>JdHx}Cqe(39DoEaUbDMZE1IOj3_R_vTm(-D_KQ@tA45#ME|MQJlL(cshAn z|4yZ_;gKX(m_h;c(%rr#PTXD{wTeI4al2Z^b@;&wt`ZhxMD_Ypd#ZfN-EDawXT~b_ zE(29pu;gUscWLO{cB`in@(`o_)N!UQ>C>pnQ9a+R4y8Nb(Apow@M@^w;+uO0C%kErse2~K@EQdG&GRPUL66XzHA0TuNE5uG11gQs8WjtMgZxgrs> zyQGfZE5nysk6ion#$lm*(^1LFNQB3H*C+ngYXY*SYG282J^?)3PeD`o$);AKmQ)M* z+U?pk(DKihmtYUCF0tPoj}tPEx3>F6E^3J>MXj!kR2YNlaa-6CYo%WusVLcO_dk8s z|A6qwWBmIw$rpEp+6rIJDq~I4LffwDun8o-d23r0?Gsg4i6%#JD!>hbV!lQuEpExT@@?BUtC4N?JV}A7XX8R+b`-~Ubb{Ws;KDc^>{1yN3nUqTMZV_3? zeC(=k730^<-4CmGQe%xt|1_7;$5E8Z()rK+r!6@>SLda`bEMx4w-=^ZgiQ=oQ6H|q z5DdhWS61;K+=Y=UftM}x4vkuUcucJ4RDzh5rC?1@6*`c$Udbk}YsOJ_$Ws0s z+fh;WE<6M0fXibL3q&}oMp6~^e0xH5|C7F8jp0$(uF%o*M?t{={uAXcBIhgIbm=pz ztX`hI5{JJXD2dez@>vep+FRJ+PTyevQ5dj3)qMI(R>0{NSmFJ4hfJsYS%Dq5n!b9p zT`uyN?U(_5HPhyvM<^=BH=c{VQXn4@O+J0DZD(n1yzW!W{pEzG^YJ3EH)gw%*cG(P z&>N7fN*U*vUs5i1ODjX|4~p!_=&<+Qc<+;=%*2zmq4y(qg<-_z zKC$7_mxF z^7rnqKRE65wpy52&b`I|Cxd&!?ZBgh9gU5d+!Cu_0SAf5E67ts`h)%V!By|>#9z3{ z^!@HnHhC}S^ywSbHl^3-nqPf4J(RkZo#K$!c}Fn$Q;TW5l-@dbhjtsc_vS*fCsO#(Gie?AU3&EF!!@@56Zx&HST z{Jzb6FLcr}&@TT>4B`IMyZ+PB`rq>=$IqqSpU-x`{D1FD64L6=HGqQu=F*+dSAgO0 zk(9s#qd$KL{F}2XV^|XOfcl@VRF$VS#wXj-6f2seum8n=JQgTQ^+mn^Q-yubUw-dj zjrnaTsVYjb{>quZd;^sK{Z#+-!`!Q<^fn1A@nxs~>ihn3SR5xQ8Mf)W4gSwZr2|G~ zQ4X8A@mJsXmqYcv2;RGzuG*jDjlXyXXyo8!Q8IYVyCw@IQVC3_O0C dt1az_p> diff --git a/benchmark-results/blaze-query-time-comparison-20240202.png b/benchmark-results/blaze-query-time-comparison-20240202.png deleted file mode 100644 index 14d8db5598bcfac7c3dcfe734e554f2c1091b1c6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98182 zcmeEvby$?^*1jU>D4>kWfOM!xgCJc>skG7^5<_=~C<2leAuUQscbDLRG=g*qNOw2< zp4W5s*&ngb;e3C5|NQp8E-q)oyz@Tqvz}G=y4RYQ(vqUs7YHt#I&}*B!F{1er%qjf zpE`BsHta0;4f3+36Zq$})g#e+r_!2;Cc*#M>nc9blaM$?175>Up`Ru=g#o<<{P3M7 z{MT!d)3;8Y`TcveQ>VO;r_lfT9!c;o^j|Refv)+-zh`1j|MT78cQI&xy!!$?=FA_j z&)kMCV>qP_-S@oteI=_?r|>RAKd1GhTfwpgPCXF1D{FsxrvF@ys@x|39C(~tr{A<( zhLK-4nDAhDco}wgQcf5L9ln4W`}Ed@*r#YdF;_9K)7%#gv!-r!y2QIe;&}c%{R{s2 zO{XzCy}3RQQGI2h$hfhzPMni|;kO}<97p!R& zOZANV&MT&A8_gc)uSD%T-@Y|==Y&^-JH#wvlx?6Gn}WPx{Po4XA4LP5JKml6_BaMi z?w=ng=<$Xz>s0@o+n?^mjFviwO~S)*9?sU))oH0Om(#OXk zYk7h00V{5)>X1A6t*+uw&>4nO{pa2Nrv%jQ3wUQ>@*1n%IPoRIr^zd0JsnQGroSKK z#9RBfG)`t3{??7ZRr~}#m8?+iPvLMz5C+?h*=Z%K6AmXg{n*0RR}ff?>TEdM)~VkS#xiR0Aggok!H-Fo3cV~wng3vK=yg%z%(;sxlk-n@Ok8GjJF}#Ah<(c07unfd zMR!Qp_h(GV4_Y^Wx|T(^MzbflE|drVfPEc@{I$mJltSd(_pVHxHZ)ASSCYJ|KUN0! z_ww79g*oUe1(Q><9x5B9=C=CKrgcp*=l9#H6ip>1w`Mo<_0E)8lo8u@k<({4+-{tE z?2Dbc%JK{W_w%V3d?Uv2K3LsxGH0%6I*(cr-~9`1ddY1yx82pov+I>+4WZ1`&v-HE zbEN5u-o4IGh#9KV@*4*#s6Ghj}dJW0x^zt$sF~aDz zZvE)U7sfj4atdlTd0sHe2uBpI!Y46?!cS99#PchiWox-JkpY8m0&5z4Y8g0bZWkO`>g!s zLjjaOEtybPL9=}GLA4OL`AQ)o8CO)6-cZap$<`t6qfuJnO-%3Evh6uJ+SkIlar`?& zmnL6QMH#IA0HK_;kCaCvh4lAs!>5gE8M&^WMg!&B^X2^95+3~EoTalGw8V@_hMSh} zuB~*I@7+n}>Y9;^HiMPdGnVrRqcSN2I4F{bJ6iWrO?(Z#YDnFj9YN@mG{*(Dx4;x# zZ+Q6|W#AK+y;>>PH!m(E4%iodjW zQ*Yd7oS*dB<}1OA55}Y2T-$r?H-@~jpNi;uQzm7h%-@tct(#h^VvX-?5AK)ATBOxp zOii&Ld#~SGOBEe)k!PVo?~I3qk70Fcgk!w1>u8WdD-cj9MZw2*F9GzsBkzS1^SXwM z3FU6m(k7s!Q#0M1o+W1w6QG>frO5ESrwW2jS2~Qnr%D!3QIc8CtRXRCgM~(?_v9!) zWkOpD8V|&=_o!3BEL2dZ^b|?M!l@n%Pt+{hno3TH3sz2=c_m8f!WwDS#Hl+?cW`S? zqxsXvT-MUz*C;)t+{nVwU@=i*8(F!R(`m&Fgpb$<3&4wbM&}{SP3>;$vduACl_tE{ zsYlE^2(4&vqRE6uixXoNQk?DPOV{J6qMfs52;fonLoe`_+H~7&I+UnT8Lyt7yVR5v zBpgGB<_I4v7zyBhxLw^6nAw~t^YP)dUWa&QbhPUhvZFjPYJDm>J)V2MBwMzhddqDQ zg_q!d=A2#?A>RBr&?#C5y(%8JQt~^7S)hAJvkkxh$pBoiktgbk~5CAVG$%eQR=RnqsI6PI|aeKTwL6W_ZtrZq!!nyQES|&V5+0RuNBehKSbh_u(A$eTk_8=$C_)^xh{? z%>rlAy_M+M*;l5^jX*#K_vTQBvlTkEB*%~koxryEOVch7qL%q9+!qvDh(PF zitNE6I*{GV7$mbXL1)0IFrFH#6DdO3;6ql0j)%w#n6H#UQzT>A`XUL+iTJ~h5>$~* zry4u(mzgOK^_@wFo!W!hfVBzU&XDT7?7g6dRTY_^nm@qhe~sm)MgvVjb@6Vi7^Cjc zVi(SrO8fj?0*aEwYQp-o#oTlkdQ9>$ylZU6iaj6>X#3zrqP(>mQ0Hwyy4Y`+DxAUV z>C>!&K3~4wKhL(=n_W#{^oiK6j~SD0Y+K|;?@*Q3xNrPHTk}-l3?>ZDwoui6DBoqG zf*Aw1s7qcs{3=0a=rf1$Y<#$B&vgOCZm&l6K_1NM(V*s7pCRDEqOi}Y7S5+A2QZCE zxox$rOl$0O3BSlSfWPwdk?l1rcgZ8dUH37`L?MhW58pD~uTv;{!%+C^&DE-H5npLF zS?Uy`fh>D*v1LI|tOriBxe5f_w#JTIlL;z4HhIP>Cb{i(_dJF=zawxdK5FaA!oyjS zkL1be`Y61Mmg;B|>K2Susgle`=e4nBuetY}Be5s53arzBKT)&o*3HH-Lcc#AtQN+sS)b!3ecI(Uu^E$nc|a&D^WH#4j#JmnkHVk*Z9Z z6v-H6^ZoV~yecH`hO_LfbI^1pohM&oZ_M52b!Z-&TcMsBnZmiPd5~M);TvB6ta$AX zjn74VCbB_QHh~eX-rh^4@U%KD29r{=%buORYV6y zndtU9@@;1qd~sBA=jC?FMDLVsVEgeg6#RTfJ&?Webzb>_;5R(6HrTq`YO;oeg1Uol9)s&!3Vi(X8>zYW zcs_T(aVXpw=a3RD|CrkUeD=_BsEA+$VHI!K^7h!pBmPQ&tkM#Na8ELTM-_|)t2_*+ zh{=RvyI_^Dmt#je%O7wwn!I*3zh(>8EQ~*{GPor__5GtLav5@KOHSSw!r*AXL(g8F zvAkZ>`d|em-Jl#0g|U84!6&4oBM^KxM&3E|w9+khO3@=KHde`*=8QCIThXfV&SNZW zBDoqL`lR*T8$$NL#OSxvjT8sxPpHTul6AeAwqEB4(_oGb?m7?Vt&p$zk8@?GzaD&y zRqopQfNIp2I_k^mb41e0P5Q&@RGt$}UK##&)KO~A<6O%mW8-G?toP!nqU^ONU57Vd z(IUPr)2`dIArxPGqB`E&aiaY|eHibx?AHn8&3mB9LUS6N+rqdD&Ozf!z`nO#a@(lV zOCFM+8ypyQh7EBr>{U1L@9)S-Eq)gz4N6VlQ4cYpqXS-uo4PXILB#@vn_Yfo)6#-q zu$M;&e|m&Ho}RqYo}JSCnwwdya0_eF4f$J2D+)P-TuHwYdYm^?2@vIvTUkvb3_LP4nFxS=ZSBLF}SDG61T2? zcmrD&dP=D*>&dmVQ1w_=gXoT-EL)11t{ViJ#cx>W2zO;=3EMxBG_sHa;jdh`$qOAP z6)b8stay2DzO>co{T-^_oO*_0+1|KXoZcXA;g*|~u~0--U+GNhyl{65#pwDP<2-F3 zHG?SrN+l*^LsFI8#eTIuM$f%DgW}2f@T{b_sPR&8X7ilx&vc{>CM%@NCwoQ`8?Z>%~QrqEP1uKFW3mN{<18>yz;!#o#x@Ky3NEL0vGOu ziZj`DzCpd**+k{vWrA4E=u6W#=N-$;bcM^ET}D~&d7uz(lwn5j3Ez2%y$ISsleq%L z^45p6LH9glKCFVHfV{2h%(FZ^uaACn#4JqYpip4e*Z?j|TZ~wJa}arKU~>mHqhO|o zna3bUdo@nc)TrRbTI5?5EcS4aXwV_FY9SX^JZh5HeUmVlg;e=={LqzX{9vnoo6g2( z#G89omi)a=W^m=tDZ{v5vJ!0(xNS>X;Ugt7lj)`8mEbtX8Y{LWZWZipO(!J!;o+W* zc3JDD<9Iu0hkl;D`^nqZ?AlJeghe_X#YtG1h;UZ8Ss5oG3Fq2NvL-0)aorgq4Y^qO z^BE?vsLURg9`zJESMIhZA{4OB?t#FyDM_drCqCD>Q_W6otA>A1#(kHtHLG7q!?3!<+z1HvL|;a9Arrkkbf1ND{u{M+5lVcZhk zTTNy4aK8HIp}JIi&g9S9gK+1y_jxo)wrfgj%zrxg8anD!`%$|@>!_4yd>>}N?uq2z zn)tZfZ*<##@Ri^t-kf$Tlv(|1r-nukW7#zGJ8KnXT^uHwK(@|d*J1V>`y{wt)Mj&` z&b!}UQ$8+fbI5H6aUCC_EB71=TGrw^2TjOW??}N$sex(N$WF--FFpbTUc4#$$CZO< zDU%91g{mA;JS!Rn+|pb6o<|yumxuol)kon|iCsRY-wba#hNoklDg!B3Tue`fR@Ea9 zdu{w5bx#{vnt9$1TXiVu2cf-yKTEc$Ft&A&_v{HP4ulow+9*yBbNW>A)ohtLj}R)| z&nS@U6GWd`LbO-0^_f>WXBra>h1tu4^n)Qa@WazUuBpUFr`Ybf1K_|KaoMlJ04qe* zn7P4F_EfzJy2IXDet8g@D4fFo)FUyuJ0F)6Wt06tEafunLcpXc;97UHEpY-rXkiIa zGY?|q#x<6~^p3WTftrP~Ao^#YK$iVj-X+^o#VJI;;|>K*s26M`iXsD1^=zCzs`vRN zzEDQJD)jN@!j5^ZNs814Ayi4?Go_tU@AXx?@@=)XrmsWrKHpb5I~Q|*7%no1o=Poj z0k}n!vhC&zgz_!S5s{}X${xNtu#pnsrwbshq)WblKj;NED4Vs!42z#+uqN?3C#{M{ z^!|bu|GT6!)wv%BQR6MGyRv6bkAuS=Y}=fqHb!1qcS>9AoXCBCwAakPR+H1q!e$AE zx;A9<62qKY?w$+UwUwLGd4EuOaSV8XR#!X4I|?0CcfHR79^5@WYy z1Vy(O_cwy#xPsbPv_Hh$tkN4fK;dr3eG9a**##~2rBH&95|AYbT4NiV8N zd(Uzf`;1)Cp#6wHHCnRF5S$q7AB)kS{qG`(|W`RB0cK? zhw;&3Jwbzc0I(?OmbTV)^DHmckS2i0jG;K8j{KRCjZtK&h_mR3(s=tLorz%%eb@5L zoymmgU{m%eqV%V24>IIULBRD@1)~)Q#v<4K}GJ^eHNvFD#NrPUh z)B4=x(=!+E?{2NuE9l}1j&Q%rhv6!0r&X_0JS*NTgQB8hH})kX>n8*sP_Q~g@;b__ z6NC!AV{N1E6+2IP?!RYLko-uXHLyhUcGJ?REivkSc^myKA$jX^Ge3Xw#l8fo=wRDi z|HI-U1L8XC7+Q4ipt#0l-4i_#S_LqRQ4^4PHtA<0bwgu~bqMhwxFPm(7f4t-swwvd zP2o66s8@B&#s-;kgi*IWKlwdWUVMq83}8RJOA-#k889_s=Y=obOmVv0Kfd4d z*rZ^RHI~V}ouufgv~ouo{Z=3lVh)rQ!y@x?g*BO)NaqD|@SE;548KnUFc%Ue`c3y< zz&8w2QOpiZ8J2_iQ3dvqJm~oC#Xh(gAC=X3niIEh+p8(85$nO{qL!BB8OEn|26xJ-jJSEu4KG!^Sy(ABAO%E1-%fzp=TeLG_lx8QVT!$>8Q{TR&i*kD0^Ux?51_l z40gV}LM7juYTVyoA2Na#$h)SvHO{}UWSh#n)RDfR`rwTb<>P zu%gwCLE=&C>2^ z;WpDKk7o2Ol4yX~u-m)EBfChznpluz>a&`arH{!^&rlt+V%qgK(Z~q+-+JFqoDj@*JVV|)n!C`F~x)u z(E#gU$7`f+Zqr%><<_1T{|-5^ocB$VR1jbE7HbkOqUr;sh~GCxjj)a>Ub@JPp!bQ!J&$}_6bF3BbDY(3kw8v? zd|S7Y8I*}!+5N&W6XaaYlKVBRremi@8w}ysc}L$p&Wh1?^@kxv6xp6Fp_05e*E-@i z2fdujgDY1S!xeW-KJ&2gi;k{i748sF9y}eoJbnp*My%-cRt;!QgJrW{8l`!*B!ApZ6y9BtC?ZZ?I?O|0o zAI-(a0#M-45Ahx%-d>;A;L^{XX3I}Yb@v+!2wVjVHxs)O(2ROddK0PkoE;*W&D);$cL&zXLZU!Ko025& z0Vj3aSayAnA<8t7Zk^3u9|AtqC+BVips287 zN81;ZW>r+#Y13p%)cHtHf8h}dRUj(?E_b<+8tqBrYAylUC1s}%YNv?EN;K! z?~lM08hO0Qe}UBrjqqS4;FNn2E)GRXujqL7K>$I(1yd-lGV!(@s4hGqep#W4UYzuz z$cNWo`k`zXZ62fBW_`CdmY`FgMGY={W7a0jE&_ndx}DnqFM6aLpee#Egir1dqV$kF zh47c;!M>@fACiZeS6MZbZlvOeB2!7>kLqZ@uFiC`B(HEh`S$8+G(H1Y({8%kF0C5K z?BgDziWO#Ee655HQhHjx!xO1Jvz9&tj&zR=6%;@WkZ9MvY={;Y71YtdFEOVVY-OYt37O8L~$v9Op@ zAabo~AJL)iG@E4@BlCS0Wasp=1XW+;J(Gcb3}*OI*C57N`tF5dvYea$I{83%NiOS( zMVS;3-$v7@dk%-2$Ja{$oJTw#2bH>SW;epC?d<*Ublg%2VHA4Nnb?9yUzQFx43whZ zzbI5uia!=$gxYo9-yW0#v56awQ*}ApH`P*`zmQk%T85vZ z&4Wd-8v6cha083Zn#;{?l_#dj<#J3u>FMx;BeK zMOx@dQo+wZezvx0=4TES@7W7hu~l#rm=xKDZNKPK`g{i{+W0gRf8A>?UzjAj@H5&6JeDjmj-2y}8hXK?|LXIz*md_b;8Q*^12;lc-Z1)Jp<_08|QBli1Ox+Ogr^>Pz zep2Y)V=Wb!6%pJnz`{#fxH+gmSZk85gIq=^iR;D?GP{vYjJf0qR_C>*k=q8jSssxY-U=U%S_avjcEP zY67JkUNk5Mj1!#B#((3em+#NkbIZw#;vF9SKqFL@Q+iRk(bS$LyqX`RpkDZdRI6oh zcg}9IcyQO0`q$Owk)-6bhj<#!7Ts|Gk7{pSdV!}N%1}I$Wz6Y%X}B1de8eEPNW?bF zc8}`WvvyE2B2xjDJUjwQQr=>+W)J^(EY;{K8mRv|AEbYU#)*@OE$_?IH~HG*b3rJ} zZamYKshbcuG)0L+;GSA~o@rG2k&{F47z9hq(aGwEf~6vO zmqiV6cE#h&Ejf?zrBiCMhDlfxqmjNt*L6aFDt(8_Gi+)%?=eB3KLU%61m{Zxe)Te_ z-`Y@%uY+=wZq<`&>G$88VNDX@Y^nB&NIo=$Agx!e)m)*Q0K@y9R_sTnO44Io@6Nnp zjE-D-+BzDd5v7gans^nTJQNmJx-smln_xKE|MF)qC^v>&&YWoG4&F>1?ZX6{g}p5P(&{|> z+OOyW0-e&?oXt}QtMUenO47@BXU(z~)X=+z`;(nDAlQq|B>}(*(blaG;sy&#F6l>D zi19+@QwhXqVT_q}MNQ%#tiq8=a$7eW^T ztylf7DgT6KR;-8d21! zUuSCSj6yWYG_nkA)~(xo^@bw>-NCeN5(um&qa+ep<%wON)F6_m$GT-_nIya7%hYOEiKMsABISXFv#A&H>0Wu7aH*-&}F2iC!6%#hm@#;qq=|q20(@@GA2~T&F1)^Mhmp2{oAT|+z^7OuRlw4wF zHN^T}1gf*6#qCcBaJTn1YWU;v&u&#Wd$}~FEATiBmbql_)i5;&-K|UfinuBBnzc*{ z6b0oLTa3%HI}I!=+0wjEsmtays5p;@WAfJUqw#dD? zn(oHCtuDtcWi`#U&5*w~CtqxuDV51v_HnVH#-A`mE;*IwrfdE1Ix`=fF5SjALmT}( zcW=*}mJ{X?f{3he~V$I_ID4_b&e?8keX%KYNKJVpqpm$418Na3Jk?C6vd( z5)tb?W%4*)ycSIl-uF^kR`#sXJ2n6|({bPmj*3K6D&pQvcHQ05jeZaJLtZWJ*n-bX z%EJA~q)S-f!@e=v=;*Y`_NYSbf#l)Q(fK1l* zSc*3&R#{UP$GCo8lBl!82oyKbZAZAdY^Hw9m>D_G54#{gm1k|E7|9Jia7H*gI;$UP0xCSx4FjhQ@8H)Z+i9;6ecSu_lD!?q%#j!u_is$5ABz7pw zJq+Tu&6NY_!67~VpENXgE+`3VJ2#$pP^XV9_|GK>=wlSwt)T}T!>N{@ zM+11Vqa!FsW}9LGjAo#CWbfDyZqWk(B-`We*GFy?BZILwzrU9*Qyj1H=Y#!^BWEB} zVO~wY)OG*QB^~*YgMT7W0C9pvbZXRLNgez7(AK#lK#s5SEA!~`4u0~V%VJFfMaU68 z72d43W&%=M)c-8MeZ0es^=iCB^jv~uAD<{N0(O^eF1wxOUVa{k!N z7>I(4a4^1p+?xM*(}6Sg7evNm-}{ec2>1dK{;ePj=+ zo*8!JZ!wwZQTD?xt-EzY;-FF%h;N-FxH>@tP|lA>VsWo2MdLwgUr=jH+e5VgZWR~B z>%?|mPaU99lR0gYw2{WmyzAm0Mf_>#!w+LIkgjWX0A*`ZHd8oi{6bM^ZSfLfzv(c2 zmSVmNQCv2L2qwQ*Lr}#NqR{4mggw{Z&tC$ABE^Tx)o$#y~aX`Tp+wqYabRPY))4y_J%z zA+i#m8tCW$7(~vNjT4-j3lZ0#91>agj{39R`~WiCBo8@@(|ert3w?>Uw`z5BnCs4A zTl2JH=8E^L9)Lnf+nxckmV{EA-vlHV*-91D!qKc|0J8-{Y03(y=;-xo08-lu#9|R% zkVy8g0JudPl-V0kOz4KF$qQt&r8FI?7ead;Qw7Woal94T(fdcxnSU3?iv#S|7;9gn z^ita%m-T+J`gT1}tZQ%9K=y6@joj-Un}+iu3c0)5DFso|^B{hy7&tTaV>)7Jd{_tn38doUQdX8h~|lB_ON7jKBN*tZp<%yh^^Q#6{Wx!Dv30 zRy>4cBd_RPgZ&VZS>A{VVlYl-0d|0oG=qwZh?vm z^ugdXuqEFt+~35<7)RG#aMC5@&R77Po;ZlR zf;k7NeK!ja=tG(O`TSCo=bnja0#x3lqYItSZ@69m3E*lR8seAucg!6Dfmo~_^txNNH)@XMwP7ns|8IR3U@kp*afXHew3GxTC%VZ7fFDKg-G$J7q>2Q*9S#Wbq zZEfl?CPi8lsF2M{M&F@E8}L?vUUV8kH-N)i_g>r?R1b2ms$0&Tej;ZZQ0E4a89Ss{ zAPH|PpuC2a0i%+T*oB9?v4r?io~cta=4%qw#Jk?7t5pGOl=H+GdBQBT042#>W*(PD z+B(?wcb4_rPkp4&#Oeh!m+O`(fl`2E?GxT}r@Zn4?QcphAhpU#5N^zMF!|at=hmij1|y|B21smtpK=glRD-L(kj;F}fJa zca~bof1}+$ERn+J^h5%P-md{9{;G=~o8gbdtAb=T=8X=%E#sgzGo3P@X$pE88ax%t z0B0dfO?xmQ$}#EclZb20WaW5|j|n$cZ-VB4QeNiuqX}F~l7Mb|;c!M-D#=bcu}=-e zhBV<#Y2O9iAyKTz)=#aj%+{970-AAybnc}sfN^NiRD2K5JP-hHsznk~U`jy=}&_KZt^-?KCYpVfB zYY)*2TCP-8#Tm23)PN1pL+)ejaQS){_eU8?*!!A zAkyq?hyuAEuFKX6b?W?#IQpg!eVP^U=?I0OF`fIvCAfm-P=&rYw(=+FjcWO2OL1S(xcy!z#PYESx!Qhi zS$&BPoWqB z79nUB>PY$oAVp4_`GL}u3tGl~n$NMrxEqa?G*Z4QmbfQMD;fhyn^`c2r}rfk2&P8g zUF;h>#;yMduh`$#ct1Vc)X#Xtubu*t-dQYs>AX>3m~N>sFhqdb$Rr?|O8IDZLwGaa z?n>JV0QYqTcgMEdfkJ6j1my9+6nV-xW-KjS_(wqF2#)A-SBdRCTJnVZ<%+_5|qQ?W&4iAm{d$LCx!%^B`aRV!UwjAa1q_~Q%# zs^>d7MZXd4`y+`$`0l`gJonrz5{YaI&~Hf$-V#2LB0(~i-UlXhbsuZ>$X^6vd@4YH zOe-j%yf=9%Ft|A$9S!k;ZaTTH7Y|Cv49&k5C8^C5{+ZFGo!AF@Ao6pPIJ9mufB^yj zm<3>MZiPD=<+?S=3POTbX4LySnwL-ufKf`Oh($>24%8HbtwltuZ9f17jd{8HH}Cma z_(NeV_so}bq;!$&^48NG^r_{0TeF8X65GI(AO-~LDp|;G0-i-#TB#z$@ZtG(z1hgo7Oa_ zs}tffAvw`JQGCH{tP>40j-d4RS>>oo*)Wg^DUM%inLT=1WbwR*{L1%M(?d+4_7JES zBn*3e8!+SIk!|sD&_ID880~Xi3r&EZw%rUjeqba)m$SN(MzR@Dk1|X*9{d52#^wT#3?t#*1MQgRsd)PIRSB##Qn|^c7=Rl*DGc#0z zodj*M5+Jv0uJi_sElH$G?)yaN0>G9?0DXe7Ih^-AETUj|Wgrv$yhLO%2j(oiTbutT zzG4GL1U{2O<}KqVl&PXUX!?-A6j9C@A4HPZCxE`DON$f6zE&aebO2e&}_wgucb1S40pWODa{-?$2mSe)xh$tS?ie59x($L zS#I=z30}DzZTCcahZ3p3r;HZha4QeW%wD`kFW!u>0bo_f`^2_{v_}MaVVByvANVn&rm0 zqJyNPkQD&o8~K%AfZbj0>!UvM1Qm2w@%deyxj^?|D=<*e&{iwikG{A23@XFyYL1g} zahNmZ`+G=`#{31a5MuG=sSe!-;0MI+p#p*}RpEa{PK*WnuY6y)p?sjJ#7bBw=#e*QZc+9HP=%_zs zxV`{9$S3?Vd(dd~&9T4SZT~9iQHemBe$Qjz=mwoe!)9m!r7$SbX0>Ou2|RB70Wkcg z0!qVxF)l0H?TP$MBCl<8h0IB-U~YJQDT-j*a^< zYd-!WYax(B|NS7xYtB1)x<{&5Ea4tDE<9PECLWnEWCEs{FN=?i?oe{e>rhta%T)^fHc?$KD?3 zY=}4QFN&LNOJFYy0%ZwC`rIFnCMP}ynr5Z;_*jfo$ALJM$^{@g%}JN4`QYNe;d?0R zg-Ccg03=u7{=s#kbNo9Dh{13#RO#xNlaLXHQqqQiUeGq5L;^5~`2;YdMz&wRa0&E= zJOl)D17(NNAThuLN`PWNfFw8CmIGuW`}6)W0z3xQ`)!72uOuS97eMU(5VZM;L+$X7 z0mc^ASDO&+8Wu19{?P?^^%4MPLT^e%^*tF>Tm%T@??Ei70Ev~zOV8^u{0!QJs68b+ z{>^?K`jh(_<_}|z5itIN1uCB)0c<5+<#kCaQ?)zxf>sXfEHrn8@ddh0KZJynIL*BF zyhu#otK_TY_mRGIrL{X}BN`f*u{uAbb4uj7vy)yBxgHu6gRHya*TB=1{f zL#<(9PIF_T_xC^@<`KYb+J1g{-UiM>wBy>$`~FTqO^b$xJy=6s6HwIJiy)*R!cQ%< z8k;sErf}XI4S7qZB{I){T!?wx{xF53Qy8&lKlg~d2nQ_g{f4B|B%nv?vs5%6{i9Ss zcA9+mSn_jGvk0NdJE%0ZveG6MlE(;Mb?<<*_2IM+3p9oZ47t$o`QUHVU1T@UDn@Ts zQ+VhE{I%h!5Qk65D$fC;aI+_Qz`{=e5WjUs0UCf5B{rO(8D4U5USH=La(vtePSQ5|JeRhyZf$*Dn5s~CgFMx z;a}hWP%M|I;V$_C#=kw2{kzH{xIX|I&2=xp(f!r{eia9%#qjkqK6?S|kJVO_UD;cC zVEM9kDX(roEl5y^kBL)V4I>e80`1|g3S~<{#(euy5$m9$^ALhR%j%8+2M;abUz%&+ zB>>t|pqUELV2Aele!x|o1U5n(fR!o)%-TYE%@B181EU!z7OV5d0t7+YLi3NH8ZOvS zevKgLu;i??iX{L8PH$2$mheBg!q1$X@O_ESGTBf(0a>mzSe4!bk}=~CbGokjZ6W!?7)JUQ0}X05I{*A!iO z?fB<9PQvbvfS;36lQ0%_3(}}AmmS7AC)UdSpxlsSaZc_RTps5g$ zabcWXkww83j?RX2hdYB0?Dv6>`yl|bwX@!m$KX`Q?G?Co3St3~M29)uJXwkZH0TZv z*x=K#kB{HXV@$!{BKcb+Csg3Swd8NZbuyFsx8eHRmz>OH{U5n^_imEkAVqP}_krDB z8%yq^(Dh$}^?-)cHb51tXIO?t4g#(M*z8m^GB7`Nk;!1bc(rCIV;1+I_tze(J-o-~(rWd9Vc70R^1?|*YNR1*46me7>g_|* zg&zFgtG%DJ+rob&3Yi+1-Lr3bHhd=I8*W!L&o{l)?Ui1ut#B=`Ynu9Yp55IJksgs& zWy>yH>GR#M&zm*B^dfP*Rl2SDi2OW3nnpQao^khO_I~Tf4UydJy=h%r-JZ55bXwpT zx{F=-p|;ddF*q|l?Hj#2qmN8>+e&b&GwN?|#|5JR^!flbqWhHABPh*h zmEGj`NsTr!@H!@HIuO|Xd2$L;1Re-_fO54e)DynQxWB~6Y}Aob@&zC&#Lu5U&$Jw^ zD_IYwS3zK7W54FPz?qG!!r#_J`xhT~Oe|^c;QrmJYmX>}rh&zAco85{O(9zME zHN8yy{M@tbu8qS+S3}phP%$1< zJ>TCQ+$XM$ZV2vcWbB5zobAC;8wQX|vnSeHTWE|{yx|rX6B7=VWW=}g6nghOC{v@` zhVU=;^z>A>C?ChTIzA_oCPve_0Ka(gqKK81)z-w|{#N>a9p^|{S=q`4zG5_bzwzey7>vnD-^it2fYS~)LtyMs_}n0p4I_kYuDW_b%sUj|YqrzOR0_XFC z+wLBVG38_W;l%2TL&ii7Kme+*x4F)rEdB&il`a9@EsJ`}f32T~MDKo{oJ@#=Jox*A zlV36nkOp7&wr3~y5aKWZ8=TMdlRWt)R~Z4>sN_B3fHeNK+35#0ALb${va$R42d02M>Ia%YZoMiO>80 zN<{<&7bQ+0^lF+Ghghn!XU|y`A;Vd2OikS!eaHES;NL-02_lNN{v$?(2cNLF)QI5h z3}2q>z)p^hD`ZY+_`?XqM4*xAt4gZ5oIEhMpac3EfLHV5}JEXcpKM5EC6{3N86oft-~ z;6UI|5Y_nV7@piHM1XTX`lFrwueIlp-NUZ|SB%(Oh#$|`IquFUK0W`sctjCc9wNqo z&nNG0wFK)8Kb%1%L`2VYt2vl)Ih5spaKk1ldAt@}Zi(~Dk|1?TLw12(Z|J?tVPjNm> z|Dl;or_s5h`tbCdR>vZ1|7nuaRsS$af4u96(C?$yEpdzt4D7g7FyPv0XlQ&cYQ1z~ zB}qiOzv+Lu0C=aqP`U3-XH-;_0TTHV8i?DGrbY-N=SB*%{-Qf;Gu>}_%go2?HTEp~ z0P4VrstF+W{nWp=yZy1am>a>UP6Xy9HzkL(efy6~{&Qc;-Pu_vH3eOASE1fR$Mr9tRsi)g zV0F4PVRz8FO@h;oi%2Gc=D%2t47>buFd|OKGNj@cysDNM#-gI8rnbi5?sbW@>1xe; z7X2){8x08cCke^Pv~P-uz=+r1pn=UFph0RL1R89e+{eSNQZbEj!zIJn=%L|E1|WGS zdGDAw$fzH1>RvqRfd1!in4-|Di6|mi^d+ym zd78PnxMaHR?`<@ixxPg}hi1~q60hjk`62 zGB&aBe`+(>KvnZ|B@Hd5g8NpYpZ5%{R(?@A*!6_-7uQCv-R{EZh7a3}mdWt-pe=BMM3z4*}zU6bsC? zjJvaLKwn!FKYzpQ`cTCau}be3dEySA1wBLqNr<;{<=2;3K`TL$^1&9y}X zhyupqE8Rk9x_(}MF}**sUtK6=S<_u$Dqp)*{22&+3j(I+~)GH^`E(a2UXGt_v=Oqb7Wezf*u;%J)MvIe!WIPeB}|qwihEc zir`7w@FX<#25qOd>(H#+uTT#ou~uu#AmWYYPcWFW zcLOk2h`<=n{Trm55kI1VZb0zfclMR8((B(s&cjLgU3tJ)4iy;>);3jC2t;sLP$;XY zSk;MHv~+aH`2cKr%GWregZb)a|9?%fCN6{P?C~ z|AEs~QGi%*u~%gbtOcvb_+$0Y#J*nr9s5<_!ts|)kPvqpf+&VTU;DRk^QD5}>#z;b zr6FXgU-?V&`QY}z|6}j1qpJF%u3<$`5Ca7TX%nQC28F9qN*aW;A|>5OUt1(ZN)QC4 zrKCG7kdW?_ZjhAbTL&fV>odOR9q)L@??2++b9S!1*4%T=_2p`9jM5D=cd1X12USUZ zwOhiWR>j^{-T+uYHzs#do*+4S@}ygYyX>idJ4}w9(oyp-8}Pfu49Wza4{sFV^=k+Ihfsn_AN2y7ujGERLb z06hhFllXC)Xbd)`u_6>gMA)>_;i|-@?_rWPg3n9D8EuY)da1p4^$a@(tb#58dRn85i#=38(BTU*6%y3`YI&k@I{a8BhTLki;sQi z)N4!*Q5P2%kJI?*E0?0LB%$;#E%|_RZubE{kTJ3uP-u08wM)+jRUmr%`X1aE@9x6E za@Rh5 zR)4Yq%b&*Ts;VK&h*-l{Fa1W>R!pYqr3$Ri+1j#wuy+FwVNpm9KVQYs@0M>E_?9hM#PT>LpmTL02p9|MZ{V8KO%$u)`-~pR;1Vmx6W! zVzuX~5XP+voJDrH{dRxMJs1*CfZoFE*be;8jQ=zZ)d(EFpwqkUr~mw}ow?y)*M;AQ zt^^go8=^G>3^TI-O~1++O^jQGEp!;bP&y80j9Z181c*y!iIis(#z!p+9W?8Foc0ej zPyX=A^$Yax-A0xM`aB*WZ;s*zF>aOA0&)oE+%3WE1;~-_bPgcIOS+^OqXX!Efa9b7 z*S84j^6A@fhknqjZ(xATs5$s*+~mW4MRm-F;tbaI5T05jtprJE5mC0bvU&ktqdpL` zwM0h7!m z{dxAleeeSSSn@!Js~u8WT{?H%1=>V`kKVuE*|Na(KEz-rCaW%LJA9sNU9uyk784RJ zOaTT7k{Fx`0mvx88La;Ax<5Yc0Oq{o3fE*Bq&M?HMxb&Qp_-KV<3EfJ*wMa>J*@4` zKiMzC%joo?npCkn%Q%wk>rS_Kt2k23O~8>P9%iJcA4t<_gD~w0KwSo6kmfNAP+(%o z)KFt(ZKrVDWWx3#%-*(a9K81+7#b2wKmvL`|I#>r&vvid8{8QIm}vy@gi7AMF^Xx& zPf{+b)yJ!smz5D@m=6{1CH{9-$T9vBOSo5eo8OlWH`MO|h3w$WnGu&qraLUQK+VZL z8m{2jtuDzF%7rd`1pE0V`BGNZipv~&D_yD-sLk( zoXy|R0lH*6a5S*-ga^VXHeywO1!LO3fiVG%IApYkP=N=qWZNxL%SiYRmN0V$jPITL z&cp_qBhB%j0Z6Rp{BQ`!?ysXOG#A(8xz$+La?VaH-j)c-C|u@)atM#* zu`o8XVk%3+y#&bxJ&mxs zSs!!d@AmT_9Rjo{z6Yf3-5Befu1U$>W6^9TYXeof%Yg~RmzUSgAZCSR21H1* zx4+*J)bE6LbBGPgS;j@z%@Pj?bAPwu|2V^j%7KUWzY()I`{WgvNE6HYT8ybV-w$En zUYv8J2ciT~JvPeUZoi%}PcK<7vv!(;GmSB(B1v5aU#@UgGQ6C3(aqBzg!y*%L$Cf`@eVmH&!bv^!NMnzjgKB z9{l48v21VtvnBu8lK)$5NtsmJ==SBJm+S|=5%R`MlJ!bF!Xux0$j1DAF#}3C4`4|b zC&#jC#x^|D6KTvPeghv9>i!8aq|I^2-{;+0hx@&E@`nbfDsD|BY)e(#>y998Xn7*q zi4U5t)-mttU;h>JN7K_tc@bQJyG1q*uby%b@I=eYq`b6MxXlhOBRFU*tUHu5gGQw| zfkc0YbPf~JUWS^f1@5B{Rpt*M`S)DbQ@W5tbI^-fg%Z>^G!qjOuhC?J`a)fTrsUN0 z8-Jt*o)PGzbrDhVnln5$2dPX~$P|#vv@}Av?Xk)d2mj^n?J$Cp3_|t<72N$H4kx`# z)cw%V(7F@@6@;tMHvo#vdl8`#8e^zs$7R|pr3g~hI5b{O+J@6!w+TTku$;@R{}RLH z_(K{cem-{z0w8%pWY{tl?gHDYe<4fw+hgz`5$)OghDBbC0G1$07tD5}5BthCZQ2wE z_kfllW!nCpSj7psA7tbQenm0d=Gu+92RLato~1%G3r`?QqRm^}g`fhXQ50nAJT@|b zI&M#REr##UUU=|3H}K~z#lplBSce@l9}8dV2aaR2qD2J_|W0 z*LWrSQ<9T4T$z(9f5_djU!mcMFX!(3D-DOB4N*FEnf58QIbzU>|Mn(~sJ5wvh31*t z1*4$f*A9Gy8+z_HK`%A`(S&=XfhLO*V?l+I&$1vPMK zn!RJ`7=$jV-M%{Qkf`$Z;lIr}f*FC34nnBPY?~TxCIiKd(_+9w%0`5!M?g-T#}YCw>VS*}h{BRM9TBL z=j7Cs%S)w?oTOXt?QO7{o5j-JzfT=I&j*!ef3K){iK~OW+H@<)%r*+oYkC}Xh6{%X z4rkrn>U6$hG19hc`qJE1-|y7{orH3~w3UZzp8>-${*3^^pR@ZbD3Y7HM@v_iVXg_n z$4tY!1xlOkLFM#4B49jW1iYzv!zrda@QaEXtPeLpje>#>BE^;yXMV??0e07XRo5zg zW@{GS?$Bw$`Q8yKTDiFoZg0z*E?I8J=a3R{KnY?OF7PT2FVf$XX+MR(Z)O|gu`%r7 zn6gQCKmDVz&rhp)#4lY}q2#O1zS;eKvErazK)63a4oo>PvxvFNYQd<3^=AH zl-}Ch?9WJ+ZL$XJFm=_GNnP-|Afc!Dg2Z_fRDIJ^8(Ye?u_Ku4~^jeGGnS2(~jji{DgF?51uT=7l z3$!$UuI?+5If+p!KCuY|G%JL6(YR8BI`z{I^cG6Q)a@FO0I=sh5GvY1V#zuCd4B7W zQ`utoKSK#?BT#GYnqdE$2z4JefZx~!^2DSh&lgX92ZCZ2JIlD&p*zXG>|2I!<9Wyh zHs!qS8qRq~zNatdky~k*BZHvy4*A~Mvl77`wNlTIovXK(ezWr_+rPqoF9~T2hT8a` z^5HZHM-n0S36Z+Zo@*Z2yNw|G^BerptPxvg*d?A@Ja1!3K0C2bR>jr~P-i+tU~p+? zb(Y*E*59&|(g}#3y8igB7MS1G<0{Il7Vm!pKzd!~4HME{(EC@ifyJEDdw;yARPxqM zL^%F{Hm8j>R1!OIgM6%q?@OBQ=$kwELAJe2=Dg;E_o2MTwIX-kJy6{?p>GtVo_rZo zU?*W3Ogkwev>HB}X~Rq)y!72i)Hq3LGYzLZ-QMcd#yA6&=3PaIXa^jTqVu1&@ZSg6 zZXC&ZM9$9nK1us?R4VhJhjcJ(=GR=B$8q|gjD#Ge61M+xh<-q*G$$Yx&^#D4e;G6Q zZNj3X8Mp7^U}F;y78X7MV(+6!g6^qYu`Ew|S)tt|y4uLClh#J{@^t7|`q5G?3tP#%8JSt^nO!oKf8j&%r8w(S6$gJeaDP@4E*Z(>en zT!WC#QQgQ?eSH)1C3ED=x>ld{*B^3mk*<=zCc%jm^ErNaFQfdW&sh)m{GTDpTpE

S6sc8Ke#fHW$N&8HeW?3o?7y3X@9q$m4Z!tFKpzkyZqde=b+d8*>Dra<|J{cC zHvxYctN$k8&x`fH9q{v}{q8US0~4f$`qP*PR$(nD-8-;(i*b5%KoZM~l9NovQsnDR zR^S^z5cVi8;!apXmAy}dp4<_yU$Kly=KZ?%fV8o%S;AMW4oAJ(o8K5fNY$yF!^ z7Sz7d3TN;`R$SS<1ju!3OW*6==oz_Y%w+X8}Y`Q+;sXWz)Np;GH0mhE)9Paqy+P{}@llpKL9uY!Hub#ZEO2)2}CZx8rFKCtA~ zCi|uS`r#ymQ<`{~{vdDDX*$krG|Avp}~IW= zAG-1u5~kQB{{JQ10GGQB6iq5Nv_V;c-753##Fek%mi{W-qK-nrqxT^8N`l<3%0Ke} z*h+Y`?99#(3o5a#{GjG5mt$vpXrT%6wT^+LNuvH((Em)@Wmo2n?qVM#*T-fmA>iX? zTz3sKGqX|8SewH`ZID}XRt0!B?NDaU36WzZqZVs<0(v({oosFYp~e)TuYiQfKWk3_ z(+B~Lc8o29Mz4@aHGUmF6n27JZY?|Of272MaBd)fVRX8~RajJ1v_-Wb$jP0NkVTAA z%};k|XsB=CGZm1iTtJud2u!OdBU7?navO<-m9=$#-5F;Ns;&3`1%i3Lbj4dC&Gzng z^%Mux#-(7Hv$5q^Z|T;PBT*mmiKh9y7MNr{1#GvPvr>C(&lWyYv>7)0TWqFDZl0)y z)aK}^8CL~4!mlVm4-P3ASt*r=wgE)4M=yrHo1YmU4^06@93=DSDnebVj=9Uy> zJdrwmblZR~#3Sc#rrLzFnExb=wLeJzLRYtXX9J;#{utyywIS(hOrnsC(+UEmqXQ~S zJhmrBAwcxY61}o}hgfoYI)2Xa#b|G_8%G4>bQf|ak52^@sCnwW#}`Zgu;tpBGaY4j z6fHkKq!I+8aN%L8;np_eFDgvl^z{b&4+cD!HO6^q9ffpJXr+s_;uv;OrZ-?a>D}SL zW0p9>FU}7Pw9n%agi|he@9pJFbMuj}>&W3Rez~Eo3$4?sG)s5om*&tnj@deri*?E5 zedr>f)s`Oy_R}cn)(S#SAi%}%5d&~`_wj^}0H!w&4G)(Pb!9%|wO|UB@Fy(@a&`ye z1h>C)TVQmw-nM9+6m5j5<6gS-S&?O8y5FmNtW_q#yvv^(U4iQ{XOY^p;4Sn#K8xu1 z*gpAs_K47Od?o&ZdC?o&_mG}h>WO`9Jm*JMpu8V_dhu-51?8YSdD80-NVGgL?Sw7E zxYA%%*~KuXOe4{;mS8BoQPK@PrGB(0wW>!exq_=D0@kXUJfz;MlH(H1I*EpQ&dd#S zJ$Z4IfYt|9aQgPW9Z;@}%Kn5c_9)m7@|`C>quAKlyTsTCx1O+@w^Rhv#yzrq+cqF2 z(Y#2SoF~q|I?!X<`{S|n>tlEMRD~Qo9h>^c@wVMdyGU*mWm^P!$BX*q3s3*mU zM35wndsD>cH$x!b0~ZGF(e9aWKH<-q{tJ`@NX{CRv_WwdO4354ccz{n|xp2eqx?1;ea zJnf+7`~bH?jw1uF{Q=Fh`!O*w$Q2TU+$-w- zfdS?Q!MI801~*Pj-6DXsqmazj`e#CMtdO!Qvgl+=pS&nCC;hAHCBJrrs$QU!7@sBn zLrMtt0RTHNzq9xsL>m8r9Y1-%-&=+M+t4Ph@bYo z{Ff12+hQJyfMs)^9?fYLI)9qZqj6g>f(v*a@M7Gjqy&TTqYXG>>L6Fs2fa<)=_yzmFkDX^DUF8lk82kuIjS2R7j*uCzND}Blp zATqQLdyx9F1oO*VClusUV2ix`#R#mdjPL&i?f+zyM`$GNPq^~lwV6gQ<+m(2h|1L5 z*m2`yhg-oxP8znE>-6Y#`~@$x0sQnEF)e1QQ*VxftraJ}>L^PsH%%T^)+0XgcJ zV2sJGD1Ln01A{#^=P^&Kl?rsAeuHIn7<23#vjJJZ8PQlc*t_=Lr`KK^jkWPz+uG}v zPFX3|P29OOGY`xKhKOjX@4rGRp8Y*s^~0Occ6$i~>Pn-7bpO(rwXOuBHlDhT(|+z3 z#nsl9_4*maok)P;wbQ3iUmt(`=Cj}WkFyo`00V(+1?H^^fZ?4o8KA|yJO!;3ENXP> zW6U#47Z_l80~Cboe;CE@db@50AuR(9bBO=WlkN8(`C%Y>HDP%F-`FC7mrmUVYRsUf z=s(PS$Zk2gOWK_JO;7lsc3~*HPE9b>?yat@w?PWJLM8IK%BrfWiA*16At9lP#@59t zpM>S|uZXgbDN>?$bad3T>b!6W6lqTyOV>DMfbnc?KzrfvPzsP(;LlfhXM2D^(Nn0O z6PE~eMQezYPea2SYbUyqO3fiDwvRy*<1PV%V8Tl>C#5w(P8 z%69)Vh;N)DR<^ui1q}X5pcp(sa;nlv^EHq812L)(s^5gq*>pRBs6^6klx5keEg^_* z^AO07db;vYMo5MfF=a3y6&jJMqrPd5m^7=AiTj}VP~8RDd!yqq=MJiMn2mp2Iqiyt z1B3X_-{0^WcY+u0-76bgrjm8jUY@!=D%V~gid0C9{{RsQHwB7>Rf4oe}*tJ8ofg*`RMPU~j~qT>9v%!+!sBq{_neF1L5CC&on7 z{pG$u&3hv&P@+*z8~6lCA(hA#J7fa-i?a?3kW`-8dD3H{Nep9aj|xmf3E#qfsyVqJ z8F}h9&<(^(ZHG-Np@NmnjYFdd*I@#I-t<5$%ipZ#vS&_Q9xOtR`op4Lm%0r=6=!b7 zMnghOtXOlhP08Xqox+y~%^RPKB(Q1h?qx_P~4O2suN z-7`Hx+#WjWvyxnOV;>y>~A{oh}+AM{1~-f zBq=s75p$U?nJ;&2T@Cm9c2VXnSmVT69GklX(uwY#3Oq&C{~^gof0~`vKGNGY{k*^p zu#2iCSU!4+!Cii@{RE@AgwZnkgPi%nVX^~O1XnsUA*IX|R9!-WC1#gSjZiBppSI3- zFYhFTzogx2`z6|DuPalDc=b$LK6Q_mfo2QvEE0Ho#i_6GYQzo3C{4NsyUYMep(!74 zkWJBH0v^}G{xaQdvCo=3jf19Rct_5;&+#=~z;Tqw2{}D6bDH~LV;P@8A|IV{%#y(9 zi=B+t^NSv#{a-3GyeOYki0GU4(0dgHx!Ycvy^$7}mTJ*3 zeJHuH!sOYwU*T1tx_n@K_Hx{tQNe4$hL4q)X^^ML#<7uez{RQaVZ346i)Mq3 zvW;cwncZnNZ)jjk8mUO1an|S@>%B(hX?O6(xWX=^|FO`2Ph+`n)lzB0zV@HH@j?Xj zKO6e5HJo{sw}&W0g^YW*mP%kfi55S*&bA;_5T8fm-Mf-GdctI$t&?q620!+TseZ;b zZcGgiG1+{pw&r?bhRPShHygK*OC2o%apMZ$@wg-9uT7U0GDiH|eGto9*aQZ?IBfiB z5@c=79A*jDab>bJwD>qMVr=@kzgoNgam`JqyB-orcoQ%g$4;cFo+>)1Ao5f50OU*)uD7i67=A&oJ2Y>qI}-^^HWqYLNFI3^HMSI{DQ;p%;li zP>6E5^e&lSJdK4tdnT}+!&J08^Qx|j!pX2YD)viTu|DGXI-|RFW<7Gh;E+x}m+-G; zY)C`qnh7Qs%F>=e=Gw-ewHKy7ow2OC@vUBK_Q|YKX9KYj%Y5F7OWkR02<8^D2qf9? z`IGJU4_?3~lIbaZmxh$r{xJNgHSFY&HmjyJ=AJ5UB5heU^&7-W46|5PigSh_C3HjY zBUWs7B8*=NZ`%nd;!79cZ!=clnWIxXY**Fd6!=z-v*Lc2V=Cbl@zDLp;&BV!tK1oSnbcrKeS9*JuQAgR}el=%TxEL{QGRhA@wCs?I z-|y$(o`O>eibrb$)6w6&L3>jJ!^=OYFt0hL%kiBfajz9Y8*9K!@kA(R*3so-4yeyp zyHqZjK%QZrS**3v%NARfZqT_7H7VhyQL-6*-v=UHB4ix8_v`y|H|-SD^f!}{&$2S> z%KE3X31=(*0jI!RSKC3g`rYrFaiG9@-2~v8DPsmmZ7q7)n5YwlmLq$B)8tVCE1T`u zF*oF>MlcR>HlR6>m-$7L8+pIRC3muq7TYCe302aPcx6GzFpSA@4lRlR=lXUQcr-)@N3gq@$$ew#A%@vXWUola~?YNATz z`K1|j&Q8v?A31ORsVNYN%nIj5UzrVK^tk<(53a2xQ)vj6WYg3!58UA;9ZqwYyky($ z+?bDK7kz!BB8qPA(YZis78^uL^wm*AQQ<-~e@jRT`_D51vgsmND_-iwkZ)cp`}m6O zvSOxwm-9t05F6vg9!P^VEKo;i?(;`p0m}z0$}XM{(UJNqQ}P<-h)ZrLg5!^Gnm{`U zr%jzYNV|IJjuHh?=%(H7ch&~|FUu?U6CUiwOb%(x^+U=Y!*29?!hpG|$!1B|jh~Vt z{y{&)3)zkDuI>NZjIEk(7E5F|#)?;CF0qq%2zKD%Blv#=q*h-5KcHU|%5hmrU-&v! zQck(ePbtOSFJ5GS5G$X?1*bt(@%XO_!2b}qOYMd5TwVKU7lOT_pW*vim)}h^n-dV{ zuJO0ODxZAq{KpfRgc$0(K67=%CwdtFadmyyfgSj88s9s0Sa{7I|7rW8mA5u!^_8=6 zvrEg~mZm2TD=&9TpV8Qs0xO_R-#!U54Lj9b?2;_4GY#psFJfc{O8Ad4D#VBDXYYH< z2uR-B=EdvZQ~m$2=Fb3lVtAiSr5e*?PBj%EogEf)ktctcgY~2G-M_ZQLErZ4+el;9 zH<-XSfWNq!hGM7w(D3j6#qti8>b=xaX^hjr?<&9sGEB*7z!a3rB>(`Eygg6W4ATun zq7RGme|<9c?y#7we)j7nXLZ!@OQ>*VXVoR~T{6*%enP^+*Sl~0eCA6z!3h_cHGr_gP;xI1BIB&yhX`=;W!w#FZa<_28Nd{px7x! zwa_bE>C3KD0T4xhx^4_4)|o;&TPTt&^9M7HX~=L0>z4LpH=MZuD1gMAkzf;p(-6cUAhhq*##`!^@`zn}3`( z)w*T$2x{gr1N~Hxb58`~0fO-9zJ57QW!3wW%^+6 zal~9Acvs71PF6fqrNG`}5b8H?D5~0@Y(&xMw8!6$)2XwZ7cI8*;Eky%zh6NRS)aFt z6Z_{rxf$!)j6ye)yd0j*iKn=IrH(FdL6juU@%-Hs3)!`mg`U*oFBVw)?m2QWoobbO zXN-F-uWn+xP?5|~DUkf*OAkF;ijMTCu!TF#5Gn9aL}8Ax%q@beQJVt6X5!#6$* z8q&=h_haETvNt8^9M33(`qVx$lEdfHWO4>dGAcT=cWTzMx^F-aZzVc;_n|}%nS2`0 zI}ZJV>@-Hjxx;ik3sa)koR8ym&PsaZxX-mIQZ|d^FsMbjNrn}wgf7-SjPIV0R6~(X zyb$dC%Fiu4{D{}9xARMuv9@VWqTpOlg{%Daj^UIf@+}vsIrCn2=qwc{ZLG!FoIthB zB-#mmMSr2BD(gHIUD0Hk)TR-SlJT7AXNtR+?fvi#68^MUz z!rE=aBJ~kW0!7`T8cZ9pSaBRF-wKBDColJPiHF8+`u4a;m!p2@ zax2xzXnRq&@mQCQdSy97q1TpoYM=_XasF=q`RrQe$-!D~m5J;;VxO-G6HZ=s2>xFxK?Y%cvn%V;i zyVe2q=E6r7<%?+(Zw1M`pIgwX?ZY-!i{i;nz816non{Qa$8&YhIO&aQfoWCCezTA8 z(U@honPaPp*0b(F+lKI+o?WxogU0f?x#!TQ2BQ1Vs7@o$o>VyH$W(^N*CjBLck2=| z44a#F36s0k#Hs{PT7E*C=5E|lC5Mz4(7t9tGFbU+DzR|h+$;L*hjmxSKN!}pW~etJ zH;q_NL?K18%JWL|42}8VmRYk0PgkSpvSgIzvz$7iWSRgQWA(lyo!ZcdeK|a7pkM|_ zShI%9tMsjpkk-!&G6f{&M1e@cWR@uv}C%?Iw%G*!=V~E85*m+X~g!%q4 zfBZZFb(saccrNBF5$3V@A79eF+V^JPKleTGBzVt5H!lB*SO9R@N@Sqq@|Vsh(txLb zl38>u-qKP~t?hO`DlT3Vm7@CIH*v0pdkq=-%?d!JK=f&@plG)`UT0`t4LxS9sB&Rs zpQPj;dC7L1L-{*)%jzfLhu&wa0K-18l$Kvca@HcE%}Njw2By54{X2SLw(8rm+Jk`$ zRTte`{S~MX#O=iLIY{-oSWrvUgad{y0dL0dJMZi_J%i6s5PiToE{7qbIP3Bdpbo5?qbG?=)6;;=oA1~itnO_Wc+qFn*U(?O}#j04}tGj1&^*HZFBvoTx z5TlsDkmk)k66GB!E4$7N7mQ8If=9+L1J!j6TsO0UgUCF#+$?N8hBjetc$BOk5H4!4 zvg`g1tFJs&ofyIPQUb@;EYP~Q{JeeuhvJ5N6~BXi)hmzq>)J!r!>;wp!to5iaLdDN zo>y%nuTA(?+^fqWqARWk*&t*1vcOI z>F-zH`sWVerrMt8mG{CN!P#`^lf>;!RZm`?Musqp@N;TgOMTJb3miPj0Bexec-ejV z^OyiHD!%|lxqE6 z|KsS;isbs^;@|Yt?ZGil+32F!v^90S=IxXAmCK@8Shp3!b7Hj^A@@K^rbg~df@{b> zr+4aaB9}I&|FqFynn~KA@vvra1ckzNW3s;;>&QMv&)|3Je65*&Q7I_v^6S#@t zk@fWm8n3;TpOjobAj&DT4Yk=duQ-@3U*lJdKl_r|R|*BVTZjg1pw^EF5J*Ho2wcl* z*d#z9p?KoW9|IJq2P2&373fCfwzAM2av$MBl!AzYoJH_QBrfsd8s?Dmr?mO}dcwsb z^PnCuk|>Pc?b3$3?^tYVyU(|%#0Uuq`vjS-&YPXL|C5{VA(R`WC|Y@G>l#X|`UxkU zQeYZnzr(`#CL{Ww4q8uCD%EV+y$XbArMxs-D?7|bLVsfH!@)kdQx`?}Q~Bg3VmzN5 zHff0G!-g~!%1!9z?V#O*(dJw@P&P^ZCR!mS^z9B_-I`e__RVbzVlHM%_5|$iHt3;32;<`QPsX5^^+z z#e5J?Y6BpmZDW&pfMf*ZXBG?99K9JXUkC!}8#{Io0S+OQ;CilMm&pw^3oAkTrV=FF zady@8!Ob!1AUb}_DUO%kGZ-1HZaRb)l8S#tY!~@7ED##$O*m>TP(-Q%$a+3IPEM3r zDnNO=c_m;%%t!bjfarur*onh1u^}4Sn1@gfr~xdc?FSIOVfw=vP_nKPh?IWHi(}ql zG$;Gc_91HsDUen<(-RQT0=a{bmL6H;;_R%hLukQ-NHSMKy}t@5a~EG!Y}Wnm-8)5D zis4D~E%&-Z9{}y{=De`>)gA#-?dpuPv(ZP~=vyZ;UsMK^gui1xE>xaPF*>qYG+|70 zR!dDSJqkL7ZZ_Wt^%At!%p0?aIZ=W-d3t{G)gm#b$WykI zB3mS2`W+2jfS{qOSWr-qXO8uftiOr?LEAE`j`kdNTT6FBRx5%&epp2xy)wU|kF7hI zKE|9-(@2=`9(*cln^(w~$-uL)*Rkx&!%n9cw!Cev&V`)xON%~y4H8R(LNgCb4)ze< zFjw1Vah2?RqGWIY>q{53#KiL#l_PiEPFT@A8sI`bJOfJPSG5OIiDF)xquDmmaU!1XZ#!TCX+RcY09k8P1BAS%#5A7fT=c{B|A8> z8v*iiDs97a?s&SO;;_<=$uzGGyq{#}^1zR@I+Gj2zz@2W&U_vC<1?KNsba{$e@T2S zi%p8(d&XIBbDO*3w}E#)rsZWv0|OtT;-yxuNbz#c=Txs+bj|*&LykKR4PGZ`LkOUJ zpV2$74J^GO&SqKY;9pcjn0s^q%qrQLV-^V9*@cMqA+$CkSB~L{M8hkxm*wdeduyyM zS{K4NL=24U(MO$&`7L8Yuo7RDeGsvGeg~g8WHVLBos5I(Ve(F1dR~h!1jj9Cc|dMG zt-Jf{Xm5X@(N#KGZa%BYS*T)4$sO<^flQ67N2MUc`RhtA3Fp~tY^Ig6w$>Gw9#oiK z-zF@dhOG5E*L`ox-@KPdTO{NbqOy9k)WSca)~wOj(~$DmB49|`1@mrw zlBYXJRGtWtZNGU+xX*aOxbb;^+*R*~2Y9hI(L5qkIbR^*5~oP%FJ}LVH013bx(0r4 zyRbQ_duI;`2VVL*8{Inu8&7J=&^d`+G2x4oBk8iC3z*6FbsKEGoW{Yic`8x1B&?#; z8Lb`ccity?mvdoj#CS}l6i$|w%;L>>k3k2#-ZPtO0Q;j-U77_CTo>yh1HzEhi4+to0bFLHX@-WRnW=j2kfXMGlI z*hj83)DkzoxiFHHJ7P0En`maeKrvYhR0M68T5CP7)_D_TYT9iolljnBh(9>Tvh;P5 z6m^vsOx(-bt{G{MV7|P~_cS7dO7XVEBZzm(bN}gVCG!kF$*QAHg~rN>gL+b?GbYGB zIgjB!QM2Q%rSJ`DeZb|U(dXSI$0mEP-%f*IR*t{=%3_LvP7sGt*lUD*7p;<$tsjlM z4V;E8$n~Fv@F0_zFy)PF>OP=d5T{>{+za+0drCP}efS?k#TSNb`;ePaDO zHhavolHKy7jRi(p2WPo9D$Z( zAP~q$pR_~bF{kZ(2bClu(RY~Sm3$L85BmknK0z+;%LKZO_u}S3N23j)m4_a;2X?hM zvsL-SirB^eSPAByU({$r`>hbQX0%>AvLAIepWH~nK4bm!JwM_m+wJbXPH^Rt4e&U# zpy%STR@IReP&pw0Q4efCmcNVq$x|1EBMDS~X1$iow6W9;WvKa#9Utu zQ8d17$Wh-LEx*K1Rpk(eQSOd1XzlTIK05Q#KyzU(nC9fs_x!)e!drF5&=R;Sq#5rj zBsgkKwG93NuOJaXKf~Fz<3K3SckmsG8^@ST5%gD1}{{NCq&4~!Fr7P%ek$6Sl;`KIvAT*S1 zM3?Uf-jDZPv%lMntE2nvhrrvwtdn~qbn)7szf&x`N;|mrja1U#y>_|7@&tOkB|)j$ zaRbmRKMUGzAd~j|&2~lY#95^=VgCw0zI7DXNi2=h(YOR8sEsO}KXOCAYw42&2Qrpo zpAyG!*aB5S16x}*XG%5fW7kk+Ku9Q27O0NObZ#do646H#B$ic|g0VlV5(nU64?6Rsy*c>2=b~0r~^Dezh)Gc_yN9%uJxmDM^6*pC< zvKZs7gZ5Z%!#ce8s$BX9C@S&br#E`M@<~PA@JLY@mfY*onJWl)BGV&ySW-0uZgxH3%@TuYUIXuCjPwF*WT_Pl#O2DMX!xOYDK045Ag5 zj>{;`9sD}Uy%G!{DY01q#@>M~_&Pyv5^8vt5@l)ifQ;MslEW*rx4PQ8k37nFs1n@i zl43HRhz!A6wkxgM*QZFo*nH^{(k>D!bW;H}&CR>n3LCy|dPclXmxEUY^LI0jUjEKZ z+0rAAC90t?m`g4oLR7>|5cZuW4?SdwP+MGR#+#g?i2in+X{tNXQWcO+{+pua7LnAZX4zIpx?FXVG%g^h+ckhmy=#w>Yqzt#yF0pfq z=$XoSI@nTZ^q$piIlHzk^ANaMlQSOsr-rOP34XxAC8GFr$#fRyk;r3|Fpe?5{MfA! zm))XHs59s$|B9x=(fhr3!hI7$eWc`c7+l`sxq-M``#5PCBPg+PC!V5)1ulA&7lhjB z9PI*fhBuarpmhYZ0Tx`(2TB>;3mBesRwVm8@+`*>w^%l?cS9J2&jt(8%goMOFliTk zGlIA4TSx9g$0O&2H9j3bdgE@xcmoI0i_&*553(K0yjV!2uhJNZ3+OU7v&pZM)%kAB^);G!<1 z*6y^vmn9pHtkTVZs)8OdRgnEdNbHxk6RU9x?{*1mClV* z%t>}KhC`V;q=|z^@j$Ld!Ix}zxR0W6H=#2-q%i^JrbTjTP*<(9gjirl$v$9oUH@tU zQ1CSV;h_|&DKAOs2gn!(s=)4>(kY^qRlrKylq!;YN>RaS?-7)0E^^E-(AnpbDJ`c4 z1sXs@dNRzeD1R1gHQj`>zswt}Bw|86fwpQJT9L}E#9*`37UgbuV?4ZZgY51*_*^xN zesb$=&$b6Jd;7Jus@L7G!MbU|>trgi70wJ!#<$|iVA0Yxypns{eB^1yjH#=G zTJy=hg4z9Dt}h&;yc3tF|LgjxI|xUb>mAu2Y~1SNLV05wZ&Y->w=N3H+i@E*8}dCcK=b*Gm@zWFv8>z9aN_S&Xd zGdI0S4mvxt^f%6|J)PDS24XDH^YM;X$d?T}%p!$*`lhd}JU zvmkeb^|x&#V4QcuZn<<*=z;*Wv-skQ)8TswoQ_OhAhgFBLpDzQVQA9<>3rky6i-M4=t7hZO3W`_$2vpv>Go_Hd)iiHU~-jL zF?CjpmCLAN`gZZ*YmS6R1w}*?d{9C7rOmj(?3;a)oJx_L!d#AUt+tl1SZC|j&F7>t zQ9&cD?|T@wtW87L-rlYo<;F!vPoitEo#IYP+{FDNXDR4_qQd8T-jeSer1foI90ed# zFtz+nk3M#V*9-KXuOcNxv{F+Y_p!7Oi3bv4Ih@4>m=OC$4iip23vo^ok+al>8Cjv$ zMLn~coyws!mYw}lp<@=YNj=Xl>TeUij*t$TE#KrdiRz0Z13-e@mD^Uer)|RBveUK- zjl#a%7 zI?MW|jRIjiC|~sMpzdyQQ0K1q)=}GX-^9Q|XZ61N+a?(&q%Gq1oT-lgqC;$}9QTEt z!2r*32kLlUVI!x<%4L5U9=gc_Et2AHwAr6Fj1AdoflG+-vL`c(x()A;xI6)*K<_B? zkBVfxs3`U^mzDjBWM2}Oo>bQT+;G|1anI+V#H+UVxF_TNF7E5*xbOv19`%BPszy7m zs6{O`O5!csMd0?0TsxNRQyr!BHGjPA9{vw+>^wNJx?3W?0afIH2l#Xc%jCsn6 zURD;-2Y=W5W822=R4ZE*W!R#dW^RhS=e{+?7=4PGM%Ah761bSEkKh;)?ZX8Ng8SAES_j}E*$GB|i zQwiX-KCTKhI_BJ#$HAv94lngE?it2;l~iNT6L$$--Iu<y$;RxK37evfs|B?22xx;GnBS=Dva7?P%z(9E9v{V^Uw9U7{+!yDnHMaa z8`UiJ^=l54F=M5fw3b+Z5gnl2g09DLYO8lsGATmllc_uQWw-T}3MRYLGOt>U(l~7(Q=t7_?j&W_QVn%<*=8J4tI9omF?i==$_-nVlS!6J- z-<0S_OKa#cdA6bYO;K5Jo~Qk2h{A#+94(L)@ZgnD_g8*}HWEfxxsBJFgKsviIYX)n|{Bt1eOn|WLb ziqG`9@D7V~baeEE2f3UJduDaLYf|~))L`_6=x60cU;Vsb2|wBo3l(%)3k~PCz$%0h zZR?!_PDKG(L9oZR`(oI?9Pcy;^kcAq4ap4jj6NC6kQ%&T$*o-{+p+39k zm%FHYI6Ou#sK1Ct(uS%in5D)Y5?0-SIAr0W563sv)xEs*1XVuQrjg$dEVVn;I#Rn! z^m^{l`XrFdZL~i-rU;3L#t%?8b>?8b}B;4rvGBVId3KN%jr<6xbm{5 zzjV~2Dp)>c;O@V;5JgJtb%gn=^n`oGD<#v&h)mA4otsWuHYjHcq!cBF2T{ntA+}}` z`~%PHudEg-x>sC{yJx7ss5yqG=|pv1Us9$V=*Bva^V~dksdN6Ig2dXUPrIew%Y?>K@tB=ofQROdpDq;$#%^jj?%{%TksFTErTY^+Z98 z!!)@jQZ_cG*d8*8;{%?qT=@5TI4~dcf9&y9FA736c;zJ?v(s_C6OFNP>?`BGIxXPV z_>cEi&3MdsOKepqD7#l!cCJt)&;-ktOdu7+fX^}J=jFvGDaBtqtG)YC1+(*AO?fQgu*!5cv*dAzw zp>2k-GrZCoUmPZ1gE@bR9X0@C3P_CxgIqRqGpGo$y#^r6t1Y4NfA-~+<~T>fsv7&lSX0cUo~ zDDR3LShKJW7hWc~^x;ziIb_c-Naj^KRQ0L=Ase2d#LCU7+L!)8?2Nt)EGXPt_xYKW zoF!JVF-mW{OIC4rI~g76dZPTpYB!$WM4&$zYyuKhWB|5(X$wuqThRi5(q$o#x4|ne zCKe2Jr&yz*9vYOo(RFw#fW!-b3y8mt6v_g7mhb+nLW0+~5$4TgbPmQN=#YB?N2n() z8{^DbF#L}J@nz6{dld6>El*-0AfDMOpTmfbh=@q`N?}<)X!5&0D+M8n2as#p_yEZt zX3lXqrnty)=~mlbmb(aInLQ7o;7p^`_PYwXB$UHbvU6kjByYP6-o{s{o+6sqZeaPA z%wp*>A0OYdy$s>SbfPm`;{Bd-8c2Z14`(GD09NgU0Z%~HtG|B+VY@7hEG-&1a_Crq zYWBtS@e*F-h7jSr&_^182$F3ZzKRHEtBnaMoSUGRqQzN%*?S2&JDJitA5XLs=UCRL zo8aFT)+G-1Y!JG3;)~$bxBxrasT&}3y!)(RhI#hVinw%dIkU!eLH~+EG z`~|Owi(Qa17Db*hWXnJ?m-p&{nkScwCG0fFS)*^{a|yTFYv#AR&we6^N_=U)bjijMn303(L^4|pG;%Veo4pApFgD2!639RQ@m6EPY zH=Z@q*(UrNNf4U&aPa@*>AT~pe*EuCilVsmwz*s~viH8Idr4MGMfN6}%-pM@T=OP7 ztFAJVEqj!`Z`Yp5yw)|Z&GmcTK9BG3U*o#&>$#ujc@A3lq}6Y^cCAQf)G`DSSxP_S zxKt+2o*+5pH{WObcFdwx66PvEUhh9`Tx&b{(onTXZ~)D_E**S5gtAUHkhK5^0G?`B zAIM@A<)_>pyxrdV&2l#TT&6_DtG3l%>_l%h4_be`F@nBdn5`wI!!vkpLft2;Y%@G& zc8Oq0%!Pw>w16zVRA@ZJ;-2Y(hswTFc2pcqh=lrk(8_zHV%kewh<($J$Y)EoD|z9P zD&?{V^|5C<=~RUl=zJukKj-%LSyN~UxTw%@ZFBD912f+7X@fuKXVhgQqnxX^=HKY2 z6Fqz*qv!~fN5J*0aG`mf+*wD+X~zpDazG0L@Ml?MVAe|DHbjh`jILXu3fZD^S(1Or z$h;#~yNv0EuN#X|h+6D^9Q{IGZl52VlPteJG8#Q!$s(-~g`0MShpR--VrcqtrEGeJ zO=Aje{Q@~-_kQ?*fGP#)KLI@6!L&cboD~tWa3Jv@OV`ntvY;@2k>tt$W z@fiZd?DyxKbnsM6CS-vZtnKAZ1Z3enZYkI}u#xliapBByVdWc9-eqaA%=V^`ld2G% za85ttAi!r+kVLJJ3R`AF-b2Z5u)vV55N6c_)J8U znZicRT{Fd22bn6I9=1LEcwoNSYP;1b3lSnrnk*)h1NLpdAWy z*~^x@8JKe|39X@91Jf@fNwUdamHtXzvnp>~=GAY#dGNH@!u;xmv6nUDe>(OBj+g&v zTrHiBAlG(`d34nJM{2E(U(xv?TF*SENDZ38B6S|Xu-P)=iwoT;gkAm-3R_5 z2mV6Gy)+hEZ|*A%vcA}YC#X%x>>CUtjC!M>tiR-N;vwrD$nRwyDK3@PTo+YJXQKTI zV`r%!r~&v!;mMtvoCdG?_WSBVX3HKj|G)-hUL!q;fNXome(@6|(!qSr7iX(*d_q3- zy8riSbYUv_*7t!TGUCG5BVEZ6$e~yzBd*v%f15W-x?jPz|15uS5na|!+UEqQsECIV zZo7cNWIdrUa!68b|D}irMnWFac{=X^q^GBRvyuQ)`tO-?jc7O>Bd`ZCo>!b8B4 zn1abb&rvfUy<0Vhr7as^>BkfMs+)n>7fjnnfL2Hwv9hY6kf5G+`1-D^PRnh1cqTTm z-)BJld`J8##jUO65wWHWrjs&lFuO+gH$;404F!b-H9*E%~;5P}pKS z^5cAgLe6Pa(_vJ%e|W<|LCiqxQK@*uCOc57<%uwzv>s+|KMmP?nuyT#ogM?9Zfug= zp9_o?*Jc&x;FdjH(cfm|Cxl+gw|lyF{JTtdpROh9G!b=H97FaU2a}sl7m1(g5P4;N zMI%BKV-`oZokkc0 zL(Wc`&R@3pP=TDz%aR+uOYJ|qPdsNTi%BsTZVz!4X45-E0`Ui9*h5GKyO}KjF(?4- zyYV>g5qK`a1F|i--wx=kjpqK0$^W_;mE5-nVQ#ogP;`DyoWp#3Xp)+ZsB5G z@}o}w1Ei-X6z`ttiMBiizKAo=#|$K<;2I8^O0v{GzJ z69bZ=XU9>$ik5*GZ!g3f6q$WN*O7XCBSV*~$Jl?R{+s38u`{cWlxn}cLj64&`QK|* z+{U)}L{)}*KfsIm-J2?P`EJ~k6gW7BdvsYX&X4jQsVnPL`?zoBw69x=Fw}AJzE2^B z?t#M+7xT=_PEwW8@pD&Zn{~KlMEo&MALJyLMPdttf6Sm&Uu|y-p;DKAsZH{YyQt!Y zFZOTT%}9i@TvRX3io7sCCf9M%ZYoA@kSC)yRBpLi~MSohN<~z=qIm$?Fd%os7RVuQNmxONxP}XUhc96t!s9mFi+Usj{ z;fWzjrS~~Rk4re6o?6r^z+xT^Zl=YkdD9o zYh}A7oY6p~v~d1i!y3~H{FT)g#@EYU$#?~($TM9I)7I+P`2mGWRO5dL?pS8i42Y8= z#RB9f$q~8ac>CfW=q9Vb|4GcuVN5sY$9aW~-K_KATLEVW$jyU^YkVR>>^D0q@mV3r zg5g3D=K1ez^w7yjNx!)c{!;OfE**=9SD&D*`mn)E(uoAe)1O~5UbQytMiD$IX*6GI zzB1CaJ|hRuL|a_eNljKk1!8JeVnX3fm5u52^UUG1(?3TWXQblulHmgjpKqrYa~X!jJ9$t0h+Q;D;*#7xRTg@S0c zxV-)h$R59UQt4B1WO8HZu*j&@v?Q_=jdaT%e}kRPK+s=#tI8}x%E3jO($e@BYsZO_ zDvuhfM}e(FNzA8wYDo0Yg6d`I$(zdkemMc@TN+fHMpviuiO$=dTGOd?H%r1$_zIRmDwSu5GLhT2Ed#DmPnRynQa z-aIwiQUykPgMeY4-hZ}jIdp4-kMUxfmxj`@tJzO9O$*@)yMlVELtmrd@dbN_9IvXb zF*%sJ*3^IEVms$U24_ZJ8|Kx;DTF2(l>2*$^FJK;NA#JMn?A@O!y;^dPYcg|dx`cn?7wkGP{7_cf#{ZUGXq{Uu1P)sgAS!2E!<~0TC zxGmWxd6)74{yvx^{>f%PyT4V`{G8L8-y3RiplQ1Q{^iv5vY^~Q)oXz4`RT&_Bi<5` zV}~hokvHKGb*YWasZ}GQoFGN}oU!VDQ`5dn)5rj)+hon8V=y|R^yaTO!^+>=IN#ca zkhO4?+DdnW_X!|aBjRNLBQYDOG?EWDZu)CwY0wrn77(nKeq9&QF>ww><<<5)|gPGc5l5y>6zMb4_KflXm|u zmP;VB$>$RFZmqpsA3`{7n|kqluDm;hXBq$N1#V}Yo05~S?7Q84X*^R+miWIc>|oH< z)Qs!T_Vqi4Sbl!jaeRnDH&<5i0pD67aMEw62q z(H+lkrBy#Ly-xfl24+^B|3pNSmeIGF+2$}aX1Uc5IDJzT+u$D#FYBe}73h;0Ql=t! zDEcN+=iFZz{;B@SrkqyIIQ@tDV7}RY%08R%5hXv&^Vgr(d|tWX)$y%=&|g1_VyQi& z>kc1-4cS10z!W$QspfN5J^xUb+|50#(+C@W0V%F7;XI$S@tXQ2e&ZqU1#!7oxF^=b ztJyDGcn7{GiS9&1fB8~E>MfKdsO>PuDtY;-S8`Hy3fH{wo9OvwKeOPNzbQ4k z6aj1niF+3WpBufnU_k6S#FQTRg?RK;-!*sbnFrAWW8&!;goX*=S!>V8o|Sp{*0n;* zJ@Rie^>LIIqW2a`V!z2tYl*rDbs=^v_w;VKGJ@reror}D5=P|@W=kMCWPqmV!hKo>7Yps?seyP`%B+PFGyW}o22YF24bM=V<&-F14_ zAeOh3yWw&;FFvdk(fOa{EeiQJviuHI+JRAaV@s{OejRq}#!q4Q5HRj;W7gMV^6lJg z;<*49k#vlX@(;bm&HW&#D|jj3(5uF*M$Tq!N`$*Nuvvb&gCztlAlnz-gJ+vY&(cLR zPr!@t$M57&E~!cpD)B)vXy^Irm5s6fC*N3zF`60uxP|V zdvWI3n-*7w4f4xdEV8pY*Q}Qu%(AmgzTs@0V=Ek5$d2VWlI(}&4?4RKc7J-U<)uge zZ0bi<5s+$Z8yCixR+%T#s`T4OV3eGMVJ%r?pQNSxhzj<^XZTm1rsoAw_kD}#4|(6o zDKf>NxSjcOLmS6F7At(abWAgJS}g9oHtv-EQkhj+TRbY~Z)oB3?oFONHqAM+$|>)j zo7T>Ngp@v{naolz^iij2-^7*gd{HUk=z(L8kO4_*YeU{%lD8v|zcR({RR&`-1H<)z zru+?nsG0jpUOdaE2U;eA0upkmh?j7NhJ02Of#u2nDH3SYQ(KQfV87KonMo zR)gBCpPhug>?L-0HHc|x&Xl%je8cVc5c(C>WWw;L{LwizKhPF2?F#3GJ(s5A29*mo zIy-YVYCiKF76Pqv7XUVIdy)wdHQEeM5U!;q(D90vU zR`FfubuFxP%s`^C*9SD^fk0S}+P7{OsZfP6x+J4CH8IyY_zc7IiaA1ICIjx=B78i} z^Scu-HU>Fp7yF3i(r*uj%Qr#H7?t(_*tLTAzi}Ph zNB$*$-zBN|b#BhD`L+w4#s)(k7UxVoNt%?2eHyMNCxE$bBbmZ3H@1_c1nq@o*y%i; znD@T30MPMIeiaOXscE$;-oPq~8{68Ar#0){EGHI7Taaazc?h4%!aH#$&}mTvyt9l2&(9ZEto z>Z4fWTL+hX5kKWH{t+^We|F6Je|FpjBkpo&f#2O}kok^mT-l90_S_7bpHTPL)B_i+ z0p^=*kfxHve|{`+Qz6>fAJ z3bMeEYo;^zF`|{c><%`i>2BQPwQQ0O8St+h@X+HH3Q1o<@eSabPtBu?A7^?`X89}! zXzG;4RRPwTXUJv>0sHq=xek&y9H=j^4t0T@7dV+_&wDyM{+vpWvCP zx_l2Irlv`4NkfJh10Jh_YX!BXAFs{d+|jCWE$@?CX{QQ1^kuO9ubtb3_P$!E!pH6D z82LX6&#!IYupLpFzlrRv4kcKYD+sXn%AYTIUEQNT@vVx;K*^z0Jb*cJn|mGMiRnTL zwrY@tO9 z<(N%WD@w4Bio;XA_H}+yhW-qC$2zt{k0rV%B88f&oASzINxtAQBpXj#$)M5x>`sm5 zf$dQcsbWyyrDi|pY-y>1ytRrZov64Jo*)0$GA+&hfZY}kYv0qyq;`XE6Dl+vAX|9 z{cg6)rHt{d2;iIOGf#gg?o2{!G`uePm)-G}L@SM+<%~;^nl8aRrm+saz49HJl$kNQ zEYKaER!?$Jn`($jxwiJSyqwNoJmWYkXG?&WMzb#V^aYfX*_YRQNX>3H-rSu_X(C9{R=Aq3zsf?MEsWawbLeGkk9d;`oZi|Csui`0 z4=`IU{1Lzo z|Hn}0XUm-IVz-CMm(81jZ{hNgW84$B%5ewh&qYuF^kgv(huw3fHuVlG*ul>S>QnaX zh&RYYj2hE_AuWW3JmNZ(TkH;G_*@W6H=m6XXEL#id*MhMl^~+>G`o|BQnOy?OA)e4 ztLLZ}ogg0wKYeq3V+WWhM_qG$v|sTE5M6aBbQ38j?C_FwNj^rGpm$%j&I!8-nWiQw zCWV+HG7-_c{7qXLdChWpt}ge^_-3g@{*=ck_~y;rdnz$opMN0nxgXO031QWv5&RxT ziim}s+0Z=L!5MTPZH{O@@Hf8M=I>KOoYXH7)F*XGAANNTy>!QLLIyUdIEyE`q`B8R z7y2bYZ|{m#HA!^u|9$t0N<#|&Okhp`8J3qO`gnyAWp)4jjRy`qNXh^hCRY^kEszuL z2vMhPN3qEKG{P$Pnd}|EcFMDyWn^PIni@=66c0%~W}4t3<;hQR2K>6r%(o52X=4@U zqxquMQLPJ{i)o1B^d@8`V`*$z&iNfHtlS_mn|M}XfJ&{JcwW%hQG`3fLZe@ulQ)p_ zS0}h9+>Jr`NziB~Q87ke+nJGG-U8VW|;faL2W0^H`fW zNen#YIw7__emw+Sa+^z7XX!C4G7DXB;XIaxAe-~_pV!hQV`VYmyWhJM-a=_dq9gK(D<&(L%>=eN zE#_&WlUGr*rCJ%eq=u843I@%K07%3^Y z8{PYZ-s)Xok;VuKgWuSjHH##GbKlS<`^eQ5;tYn2+1d{Q?HMyFX{`?4*SC#L?L|M$ z$YQ+%x6Qys49*Bngepe^a@au43_3qfuAHIEJ|ohpb!@4*N_RlkdXXX|hWuF6W!)`Y ziKL+Q=a|bPOtdKyfA4&Wr~Wp`<5eW91yiK;Oo7>5Fp9;r;zd?%k}tntvMKK@iSOGv zp|(F%t&Cvmu&PQ|V21p#V&oZ<5c{%B1cm)tBwr!oMUIKJB+N*DQ`rv$vc9Cb;A zyUeo-!xyCrtQ{R0W)rc+S$LGGaY^>2yIXGrZ+Uq1Qi5z0A>2svdWRn#0@t5Qspf>E6Q zQ5{z`1BV1XrKD4$tmpg zj`H4Xy5LVwlyhI@)@jKgF0_urX#$q1oI_KMW-)J&G@uLUAssQ5YL*b}%Rlj9;^U)r z@mWVga~fg5u_)pN0|Pe({{D?;yloPrsV?tMKJTjtphP%|+Op=irH6B?^o2-dTYf?> zs2*SJk5}V&iogsPKd4F%$sEj)`t#Yq_T|%Vrq@(}M|-#GEqXoNI~X3t9px6^!bacq z&iKs1Qozk7P(1W_=e2@+!`?`NLn(f()9V_QKyk$p^#ce45;k*ZMrV8uOKm}~B?Z-h zwH>m>{bw7YM1mt%(n{W135d7O=c1^pfUdzvd=?uJy&Pb6Z$umCMjxp0Df;(K%ex{v zO((#rS*?wHDf%LtZj?2hz428_`rCA+tFIq%iAxo|mE5~65T+Gf09x5moSy>{)J4uZAvL{8w@QTC^~WmSD-FZHW@ zMZf4{%2B^mE-v;MzdfQT%3w+3jy?y{vRPeW9|hm4POci|k3(5f^2xH&QP%JTb$^4i z4@3O4=HE-c9lvX0;Rm}M0^a%*CvnhTSv7o>|!4$-<*(_r=I9Vt^%Z@PvT zk@9doo!`d7s|-=SJAQDW%_Sh`Wf3PGO|)M3X<>x+=ozxRRT#20m8Q|kUtJ(&vZth+ zW5g0XiU2CphZS^na=!~GKdL^#uM=nc?Rx{V(r8n~ts2oTlI>^FX{n=Zngii=k6eS) zI8^=VRDs@;(}r~ZGmdkL`okS{h7r$HQ@aCo3r5dZYo6}CcE|#e!%g=iwX6aDYR#Ec zqd`PUAphvHa;60id6C)(fN4 zDzYSZw*D+xN$@Zs-H}e%NCxja=G>hj-%e}0JRvz_SF4v6{EtnlH90!?gqZ(M9sPt* zVx+GKi`Z;b%^$Xg5ex^fXW1G07 z8%`_{Zy$$Ing#rQ@0lxx(?AP6e^gapx(pj)X7i-X3l*bXu$36s3mnN9>TlX4qLwLf zLYJjD@K8jqqPWhf64=6c_?}@&* z+RpgJzn60yeu&CT7wIe!RSBlIr~TmMI2M$t7M5Q@r-;+{EJYjFZoqFt;SJo8E+)U@ z-00@xD#MrMX@{JB=pDR1>t*nHvjuXOj@1IYo1w@HNxL0w$MQ`_eJ?-;A=>P(#d)6R z^jV!i#VDYMzvyxrrOw^oYpwHgd_Kp_6e??2s`4|^57D*$sgNDwT@`hcA!(|1U%kjc z@ST=}oFmJ)SPOn1(9X5QC*pU!w&hn@5D0Uy@Z<_HScL|k;hHp)TjU*G3tH%w`-bIh z7a9UV)CV>hZWU&{zgo)f_TQT>>szO($akVvE{I2!K&T zcH3d#11v!%?21SLw+fuQgC(LeqU{F%{SM_E9jcF?LK3dpG@xMM?m5xs4(pK~Zwubw zlu}!+r*vydSJax(D^)wj8MY;G6@uUy^Zak?THlDc3A3#&RprA{2{ z);M`H=DZP_7LsB)*Uq%0Vtfj#|EaJ}9zy>-LmsK{uPXZX{&@XlqJ=o?n%YPoG8YsJ zJMa9+00cb+7k!XVZFwCX9+}ekpN8k3U$0yw-nf45Xa=rIAz!mW{dJ09eZLv4&M(Xvam7Ay_=8rEl=l|JEGGZ7&3ye#Mx4N zDP3^?$ZT)DysrQ@d5giQY<5*@!l#ElQiNx`>uUp-Vbd2c;;q4XloXpG~)AuxW*ZpZo^%lR)ltftj!phG$WeQfOpE(njjhl$gHQ(o|YG?)p z9$I)x`~Dln(v*%P4glkF8FZyx*r?Urh~$%b0zyYS&L&l|{*oscx(l$XOFL<)_u0uC zzJG@U<9)i(WJ;(=PQNjpqx(pY6f|{RsX>UO69yedccv15ow%mBnC8ne|H)ogkq_hfQdH8W5)lMHQ#wE09fUdWw-Bwt z=NLpl*}8O>R$#g6e}vzk07eax5XPGmbM6=njaGu4-IdX-xX<8rgK=dj_0kOmZqX+> zbebf|q}eM`8V_a9$GFd7{d{zXQvtBZXsXmhyu5LX{kaZ{Q@o~OiBeoAhjTZD72sl2m`?TZ1a%R_N*Q3K9L$x?)lh;QxLEY|! zTEMuwa%}EBOmoeJi+5+rSe%IblyF!}*pfXBExmHpvM4f9>8vvAz=WD5=U$v#)5rEl zTGp4ccgASJIR@5TuiI#$_$#iIT!eP!;nNo{V?>$qtC@Hy-1GsVG34!B;K{zszuvo; zrM5{N6FU!1mb!HwZO{>a?sT${=+fy9QS1(dUeGw=Mreqi5Ix2?jlxFzk^4N|LG}A| z0N6=pCZ74X^*ic!N2}{q^;(=uwO0*IGmLVhBPuK$#oh!P5gfr>PdNMV_Z8{iqNulP z!2EM8qbCcm1GSbeuN_P55>dJb6E4L#)L}HD8kEo|EYPkFQzP@EC4~+8;V1f0DQ35 zI_7GXv~IWYk6t2zrUcc0ht(2b{T+5=_;&lE@Gdoi#iN}p1Ph6>nMFKgMBEtvD@&Ww zj>ni%-nDEggYg%CO4co_d(%dH7m?VPEet|Q1U%9zWxq>27()5X5@2b2#IF7`E8U+( zT1;^@FZhISyT3fuR&yPnO_gIe! zgc$mw6tDZm^IFkkCvQzSt?Esi7j;+GE;eoKnUA|8|MVdfsHMPlGaWT2XP6n`hYRfd zYXxsmx7i_;e>-JLdktM*x) z)Qd)=P~0$m{;0Muria+A)?tr{P|~$Dls`XhH0LZl7MtnGFQS9dq#H+fGu}`{6Ms}N z!ypczS5X}rz-CF{j!JWz*j{WKBg3U3|>n&v!5nD{(7vmAAz| z3_EvZ6XGqI3St3+5~j()^SkI%08Xq!uuD=A<8qio3vUz8EABTG45Ks z5k+^S%Z*ixIl?`C!sajNi0c75e+n;SdeErsXiP=DD~l|LN26W?*vNy%v#1$~EYzz|#SjKkkIDmN#-G!skn!T0ny>wkM|Pwxw%yt| zh#Vo=<<}TK`j-I!c&bq7(X;8*m@Pm}{G<9Q<)rz8+F)(*xyw4bElUcWq;Afz(WiH(l6WR6a(seCMF~t|D z5)Q;-?!|%Q?P>4PGgPb*QMZ4!f}?Dx9~-m$@oc&N=;)yTD5MdT8WKzItoBSjk#!)# zGgr=(f=-#@a;Wge)~5Whc|t@bxFPRm!!o0JIEEmtc<-hPe^Dhc!)_l-fpvQV`aAOR@$5kc;KexE+7r zc~W3Tl(x{gFl^8rb-$spr;>0Qw@U4tO_1LH`Q|@n3Scp!&tNR|85Yvda^vFjCXoqA zR3%n2JKIb1f3DYKC=Bw(w?W+Hted|nt!QUJhF7RC2;-Cdd$pvfqPs9DYmr}mU_Okg zmzJz8O{)!Zvy#3?QSZB z-{$k2^30L`>ZMe5{t3ILsI;-=1k5hjEo(QjG1IvgF>xU{lH>rW32{k_@nB>isT`_Y zae5gy`y3&w^mtPyXGyB&zRX@;qbl|#>4^~y#!;>ce*)$={=j=7Ua14se-4|#voZCz zFm(MYO+V+K5Ffql&hhCiSFKk*PYT`3WDOIp2jV+lcVp;rZvddbjV0< zg`hg?Ll58zz*#Rgkb^0!>HKUncl)8cr4K1cRTH13q%mZ()5$??_>ta}yV_2_D0b}Y zbd6IJXX3~Krl&f8Kl}1CRK!YnY8@7oznfB1>z?H?P{{5^n{|g9CC+1IbMkzG=Q!AX z#BMrM?&7a&2(CL!BZtR&6Px5!@5duZ*I<=O*6%h<^_G0J%CVj$n(} zEqtlUp}LuSW+Y>N4j}8NwWxnah{qn;$=YcBRe;@?gN~j?%-0js?E_N7G$3 z^J3KeMDe=yvM;rco)ZR^6o;flm2z5PC{XaRN!iJS1WafUs4Gg^OO| zBvo@4X|}R^AY%H}j-viK2tSpvlQoZ3Uz~Dm4}lV+$h5kmle*yMUpiP24VacFmvytU zcfsQ@P~4=9=49@fBLq|bpQtGcG|MupX_;%#5y3229U$`iMj?^t7`C#Wz4trHeE zx_DlZCn6<3DY6YaK(7-S33MRd)PQ_OFYbx(AC_WY_pv}KiZ>xzE>`Sx3+J*Al)ZY7 z9izx_p=J8D_GS50%fp4iO|3So%O0+*L3DTb<_@~Xk^B=$f(0qqIt}h4&!%u%AsGJpp1s$-FUqaogu8Xn|!((BQ9-CwMdp8E3o6bS!>SO#;uqK zO~DO%SvvHQ02-r7Q}Aw>&MGY9COKx|ePSlu6(%?LJEuhA&JJzA8{u;rT;-L{r_#G+ zxk7pY4knmqx7_>QeEVOEpGL1tU~i2dN!CWEJy(17DKT4E8r+_=MLBP+tiv#=%UFLa zx877g!|CTK(?y59a;I;o{|c=^`9^;;fczeA2P3)M}jIbuBKXCt{n; z=l1%Y2-JJ_+bN@SL0KyH*3XaR37O;)09oh5hJ6WSgWtARcVyHmpW2)VRElf758I-H z%Y^!|c~v5989pZ)e-CnkIZ#V{i&q=oyk2WwwqL+!YN-`|)K+9JlJ1TtH4 z>Jurmu9pozC(6a(_JwBrM5=Z6fZd0_VRm&kyT&HB&;&FPlNA-+{_i8pMnr2PF*)gg z+%b7&oWy_IIlOyiy&Nk>-rm0{#Gy#3+002zgDZg*bS_PdN2!}yqdcpZ9~gG((UObb;xajW009+!vd2^ibYu3~r4 ze@pER;5;28$slg+g|<_Op=rb`R44s5bJShxx8rOL`m8W~RuF@z%^*_QBmIcWhN)=Y z+r}*d?z5Cd*{_Jd3Fz3Fe{2(1`!Unty#%j7cE7qT z^NwG@Jq}&ChB{?@6tWS(eBBTbQazs&v&M^ z8l-;7X$^f2#>0v?N6%CyprJ1+`5W+=F9&S+{DWzQ7vW&0@)~y9QcC|z3Fq{GxuM;g zxD8-^0X!WjzI$30rM3EQbt8NbwxCuPyU(B?`TA97$Zr_p1P~n0sJv|Vp-_pXFY1b} z{sZ*d2zL3@p8+jl{YcO3#gdc2FjlFd-{#hFemYAOtxLgUSE-s7xtOWFz9(%{)5H3w zAJCQuk^|7i9`h1dvXH=pY#8U3ZQVQ-IJo50YX%FnIUv~QDeWHfhT&ra8GZgvltpV8 zXWZS@949cU5=^k&g0Xb&`xFQExjaOClUgb=pVF55T znarF2(#Xq&z25IuSw24x09-RRN2|S2M21cnzMF+oN-&WqVp;NQ(TR2y=(Fk4g6_n< z=<8tN1hGKg2AgY4>YlW#Tt=J)43z9z_UQnQMU|y;{zMfzD(y@8Y}&NU_>kX+{NEtQsk+*5q4^xv-et=^YiJXq~7WJGKMGy?`C z?F3}yw5=KZyyuiv9*q;=l`*r6gMSMr90x&mX7pRz0~eGY_jCJ<`W`to>3XjFP1QB=B>TTNQYHV&c9@y~<`RU_)QI(uCU;iX9j*9la9Tb1> zR1AtA%o$2O`PZyS0di4MbO^MGPVYI$iaU2=9{J-yT`B#mk(#Mbsu3ol_|1o|BQG5%g z<=Uo#G?a?6M|sq1KJb~3kmdUwK-{Rbwu76bp0-I~ayHPni;{v*V)5t-sVS z?*hd50(z$G;G1QY&f|P#$R+Q<2Ki*t zAcPPw$mD6*VmW;?BV@IYtkrp?7pC=@^Q-D&ae7(6mmIzN5#tbA-*@3t=#BfPxTVdJ!07+) zWF-kGSjAnKtftWueGylD+RWHMiYV7AP^U>>iJ2ldC=RUC=*9U3t#%wWTnOx^Y4zG) zMd)8Ef6%#>YVr$;|C!zbWK|0@fKNo|nG{W*tFKD*%|5l-8*n%I-XG5trF4`>qGuy2 ze^TH+3Ee(dRKx9;-;4E<9hFqPCaqkxkO24gy)1Xy2vlcGZa;>fIFqh{xFRn5wC2%> zl&EhP#<{&91LI}n9~=aUv$7vCo3|V8TmSTZI^@@qN=I-E)aQ=cLUBBEY+7940 z{VC-l>gO^e@UH_;t0~Lpt2pP2I_uzydLNeEB0?BL#tv??-X(WtmlUuw)8~F9?f&d% z*0_5j_r(@{po5U9jm|AQD^nMKUT59jIYfBgT*b;wn*mY3 z^pU;&k8ct7UpjlaGIVYPvPL+pFi(B%n5LqIu~cu>PMg29-yMEea>|PR=Pii=Yc5xk zfk|9!`x^=WY|N{F0G*B7y)~c2cuFva^ETD-D9Mnf<9~(vQLw@d8Y@GUw;-;=dbE5_bgR@q6*P1G5YgS7Ex?C==ZbdS7Q;J=HV#| zOZw*gkH&1WKc-v|2!g&bb)Wlt0!)oa*;TR;m6rB^i#(pB4!dv~H}Qk9VFCr9mt_E4 zG^?@fG*WESchp=SzG_q7w&%rFwSc8UL@(WK#H{{?xyzezy4e5G4SV{i3CcmvSU5-x z|77l_!=wsRWkHA{g(S_?;T2qEka^aVHDe?v+3moaZnDA`h_VsYWA%7S>v((z1b(-S#YG-x)!tNQd)zu$>y35+* zHRu+54;3;T^S=%SXx0nAs!GpU*BuqC`tPn;jLPG7kw8C$BY6Qs4S7SKT@K{|o+W;q zg-@X_-FjqD`Y9MzkSbQl!oAHi1uMBY|a@69bo7b3i_+r&FF3@Xk z2*i7cTj*O%XJABS7P^Hu63LWEz1WXo@b{$?_R`|3Jm0Ts`l?+f9&*&K#(=`O%OFI{~7TM658TDM**ct zM@7P6bs7&J?V{C{qO|B=CJsJ+YW|qx<$7uM9Eegal#eNeHJ3Zu(l$o+nJzu3@XxL`4@El;$ z{VuJ$PT<%|H4rqZM(f8Q*&EE>5WoUuHy-<_zYOnub@?ZG_`M0R0mvwZ09*{YP6RW7 zhDA{i?YM_s2l=^~AKSlxPybR4x?k|c@&aw-DQ#U=R|o1jZ`s)?(3VG(VZ9*5jt-Yk z(pvz=@s#%Whpcfu;i;G-x`J3KwHCe?kJH;+ips|$eL;JfA_tNSu>id=(TjJlqOF63mm4>)EBmH7z=acX9B^k%$9_3hf z<}J_HwB%bDTLef;#*)W*}TV(Pyf8!6y2ZppJ6KH z4y9eilb26#+(fDLa7S6oN=ky^`b{oDq{$5CMmdY^d&+f-2lUA$a0eKszpV3DB^2cU z2$&UEF*r*@*c|PUxo-&8C&n+7r(;dN(D|qks2f-W8P8YLw2AdDKvGzHzi*1DoCbc^z4q*OIZ*0$%L#k~i-8$0{45?Hz z;JpyEEdO77-x=2AnzgHl3O0IClwfE!nuR6?(a@xcs32WYic+NaAc_)-ARD`UL(a-Z6TkO=NFoutTl69q<65IxEM7i@)XCWyxkseCer%z(e744 zoqi5Wv&zK=D1Q0A8k~9fRpmQ-_)M1d!u(b7_cP&Ivsqv&79^jtkF3$k>vX|{r6;#K zbdq}Ar@J(hCM~|VzQ@-Ts`OzlqmPRpwpve@ZqGNmV2E{!p-IE3xJ|uXB8zt&_eU`G^QRBCRBn4*484!PNiiH_$`q(~E4Ovt zpGy2W^fF^hi@03OotXncb>YOS=91jb8~`V{P+h*%OvlLfbNh=NhaN$*&^o^-zS~ma z=Q!rn7w%J;Q5C?VeZY9ymRTZa+3e;D91|!k67yhjcqkqr!s4-Z4iV8TdRB z;kv6J44ag+#jUkl;lTNqDM~&9^2_%e@jJx6#19Z&1yNJ#kcZz#fB_caNl64-==C83 zIpe@QzmB$C?>(Wi)VoSCFf^0w$` z2hh0A8tk+abg4b`fYg8{baLL1!&KC)|N8O;mFgNhW(slUZVHa{Nlj+(w+b zS41h3>2bOjm$EP>fjKO_qr1^rC>^x0S;!ilE*}vn_sEZGpparw*~Xc&2qPv(2s$BZp$woQ9n*x)u#Gw;xo_V`G~Eku@0+ww>Kp zNrBTRT*+6E^ilA))@pmIGxy!~L2kS#n9IJoN%uPvTk3`;4Of2;QXGcPBw z(;0m(Bo&}EM@3y00u+Zj)y9$j$*S-OnouQ?r1Qp4yboBA3(*fPr8|8lDXDFIwwb@Z zm8romTx}fD!5@aWIVaKR#!NWHH75vHvl++46vSRP7bradnh@F+N)`PuKJIV`%<10S z&NOucvnU`I*OYG~(MrgDB8d9DPI&(qw#Hm3pox&!nofmHNriFQm|@bR`ANx24aiVzbOw z$-Znt=SPCWDKiMNtN>2N9Z->f9W4?5)lR-=AUQli3R`9ESz5Qz$L`5^oD_R|%1FHm zY3zX%d&|`dB5UqQKdophR828txTg5lVc zS=D!2lP%}A(3;ioos?OxBQ^!JA*%9hBxMFw>ZPWi`-H&)ehq~VF$(Uh%R*+DeG2la z)%*u$JY9C!C9xc+IRDB(drE&wR8aSYYNzFoy4=(4((o;;*Da5!Bd>5eDDQWBKrt*E zcV?fz3aRa|HSTw}0NE6{g(-q;LD=Qe+}c+q4S_H5hGshNTGhW74FhUgu*Ie{SNp6P zM3TT**!wrX@FKTl+O?62Uuee99R=Ue=FmNF)Uxrx=o-NEXIA^ZG3E=Ljq~!iyn3;+ zPPlK=Cy}~_XDu6CDat%rMvvLM`T>L9L%ekrM)9Q#SlUJ%WHWlaWkY_aWLEY|T9Me= zHV*ez?b+GVW#8Z23T`}!s7M=2JFz|svjS*NR`59(XXgGycuOb~S1{Xe=Nucq`r)3g#j`|e*Kx#LYc0x;>Qb;g(VD`#Tcl~=l|EP!>w1KW~hktow%2~fyLIR*C-{(1I1swe%!7liuq7rT|Q3HFs^hE3M)(R0_(uTl=IK^z5NUdNq^N%7(;qh5PiPmZe|yLRi^$MuhSdMI)#Q&8L+9K3fZ-D&b5a)Bm!>i z^C|tdRzj`2Wt_P8z)uKynkaG0PE%0LeyN<5^3K3IlFOSS3QqMjXN zytG$!96uVx!|@0AeT(EMD}xDn=|GdkMUW2Te!g8ZshZ0VF3{&S3ebG(ZjQW=D4FZ- zM!hT-;wPxL@grJBVc5~F2bGbL3BlO*YUw0RpC6E4{xa^pr@(VV(KiHVEKbR)e>iG^Zc>*u#=3y^Y(WwQ0j-#} z^r%isVD-TYT)64s_o|@}+~bLhP=6cgH*5e_%dfyKSE7aRr0jUqpt z+RNyM3jVsG>B;j~k-yf*%Wg@ZtXK349-TxJO?NTvvu7O-oUMF_+%|8mXu){bk2iX^ zn7r|=Mj-pFCsY(M5)uMo*V9;>ua{JUnm4>9DN#dl=oqdooDga&Smk|>_B7h?g}{S7 zY|O(#j7MogLpb;QHJ}eo^YJzAT*18LJY~gRS78Soj3rwaWmi&sy`a(cvMP-eI#VJ^ zXRvP`*;ZfpwpqIQ(Oj(qK`+i{;{Dn+b0C@&toJI4RS;`_1UP%a2=YD38OL!fEJ7?Y zq@FO7J(#hLP0BB$&7yH6re3TdF_cuF|4}>KRyRJ$UQfDOEYCQ&PV?Tkh^u|OPYihg zeGBbc|2$`t_R1=<|5)TTQGGa2o^#-;kT`#u_+alUN_?7Zax z`4==}jaJ2>@{#;|owtn-@5YE;?f?`V-LziVL{CVaTYwLfs3G(MQ&B=Qc=mNE3j(i~ zYM782|7UIWxi4&n9ppL{q}$iSEjCfzy~dY!gQLwB&F~*~KK)hY-5TQg7p3YICVxiT z++|h(kwY>hD`txzk|riE_LY6mDo8k*cUMa5*lptQvv29yI<0tkpi{i+e+I zi8*;QtyfY~b$iAu;VQLZSoHl46Brb!?{$LF)K`b&pPCS+G&VM=jD41s=2#k(My{sR z&(q>NA0A@H&stDEVJE_Cb$o~PY&6c3ynH~(@3d_enL9b9tRH0gn0iX4_}hZ~=E^br zw4y{&ePc&uN2JP{Azt*ej0`TCg1uS{daS`fwXr|618>cO|OOtn9~e?`PTI>D*0xH(K0|< z^{g$W7x{N`ei5BJg6^lv`zmZz`>ISU)3roOKWR1c64@T&3cY z)aB$ak+srouMGTX)84+d^;Uum926FOcW9L&+d8D($%VH$#<+}H;$TR~=xgN>bn7^? zGWYGbrCx~P@dKg3Q}2!^_1iSb+9dU>HCs_!TbA397mC+TI zKvy)eUKl$hO%n7G==VIEX@-kQ?MK(4T|>Ne{RHLV_ww)4U!CiEYbC@C$p4nwwD-?h5C6fJH z%Vvq>j&s_FRQT-f1%$MSbe;p2#H0pBSn`BG0fzU;$&ZoG`W2a~_gP4zwN!*A1!V6r zChO1|7Hdcrgw53bHdi90OZA(0FWUI-sKQ!+3|xgyV5k&N@EPENh^1K1;VS*yX*K<7JDy|oD~Ni z-Du_Slo1y2A7%Vm4^ujyEX3?d{M_Oeo2cTT-r=&qRaA2C8l#Ya|29Sl$*dO+X*Xq@ zILd%NNl0Kdv)Z5jSLF85b9mX=C%t8BG|)f3y(fHajwlv<_^Di9Y_j>@MkS62)AZ7% z`3ndDVq?AF0KucUGyv8@Cocl%R`gx?27g7Tx;aVsD!2_%Df`k@eK5Vc8<_V$W`_s8 zp?yg7c%G0$0nJd7_Il+la#)F@l~9O?3zeY%Qh)pDW`Re<%+)t9atp(&YaaBEfxa~| zna9ldtIuRD83r*lJIz5ivjp=heL%91{ez4}w870KJ|LLLsm{d3#rApUu)FY23Dnc-^Eah+koH5X(cL_$#__ag zZDSr62*1|v$cmMp@lJST_HP#zy}zau>~e~Mf9wvl>U#cf`8s~y>Mq+Tu@A||3P~H< z=l9-SHTb1;$C#%H~HaL2d?+3B zSl_+IIK<%r*+U5GkRrnfW3yLqos!p~G`yR|Z|?YbOFeNuzvB-ndHPK-?Hfb z5HXSg#YSTSweTAo-IZx4m?RL)R?dYb`^s+AD{v&L|LJ)AVSgSdJ(@I)s;w%`|8>Gv zF(B2@V1!^O?|vLn8|~E!+re2u@OAJQT?&bTV-jL)ls$<4onFv1$8qKb;~EL+;U?qDZqL zyqvMsAZ0 zNhinEJVm#KS@h_WwG-CJo9-$g+Z^W5>Nw)1K%Y&qqL$8!Se|Y7^=2-j( z2>V(~dUyThYi(Pt9H6yVYH~09micw&8W1~2kQ;~7Cx|1n4L}n*1TwHZG^93!r|8?- zwUqUVv>#T1+jWxJV@;Scx#s@)g0zD0u9+jO7!OQVajIT$HN~$O3Ri=9la=@4)%dkw zGoe1Je_fJ(%TKx2HZ}grZO5n*-Png#Z4; zBazif-U&`0c9V0zn6^z+qi`Lrxmd7%Q8$;^7|TRAQp_^Cs|fi|vZ9s71pL}zJ9S*g zL%j#~ew@KND_``H^vX^WMmT2H|Nf`x+#G*qkw?U<*ZsX^R~8ys3(0eg{Dir#ZkU!} zRB+?sGy;1%a_nw znt$2CqX(FFVQp39?rZ$#&%XW0ybHTs{?z8bi~#B8MB%XE(*evyTOv9{p~9|9J=L@9QrM_P5Zbtmi`M-_S!JUf(jF{{|n0(lsUO2jE zi8kiPvr(exYSvtL2oL&kNYq(Fo3paD-5~z44@B{KLKCVb2;(8$;F~3+A)LH$|5Ef; zU%x#H`M_NWfdIAi4T|{?!nj)lsc2>a(}Ei!zw=Vib**D4jXZd_vV@neBaE=nG2|B+ zdOKh7Y2vnn@30XWNzy!<>j(JZI|l-%vHL~%oqcVUCIra(?xa` zKq!mI>>B%YYhZ>68AOTuyXS{9B+?!^R#a>Cj-|JR? zWLC&*3F7)_#v4GvdTZ@{C~EjhTX6qFdgjVp8Vg=ddXKHnnLis*|1S3C>C}h8ezd}6 zu|S>WPs72~zIZE5k>+S_scj6o5b+eEkVxgGjL+7cU2Lk`Be2~N?bh#+hLJ{6(hnxq zMbKWf-7gt78QVhu{vIcq{FG8-bEgs=m$ea1P*tqw)DX-4WZltxclt@B_6h8Bgq<~8 zCHkr;PQLeJQkct43*H*OnC-BAa>Ug}s&aDaO3m!_?dSb`y~pMOg4X>g62)=sSnFOa zm#HI}i)_>}5j|0fi}@F;{p(o7UT5r!yY9JPgYH_8uy5V+nV34q!`wL$AIXR@;Fr^@~r6o0jjS ztT~&Y00HH5&1h4vLFnW$^G2Cx(QNUURs@Fw$Zs%w7MjrfRJIsr_dLk5IemLW%+eU< z5{p$*;h9pSzEv5<{Jov}iTxAb^0ajFONROMDrLNyc9YCgiqveYJ9LJ=ubFeT-`jfR zEbXSn=AlX2_r|e*xeP0w!T(&;wFoAwrn}QcbbN#LCV#X(>air?4!YGGD9c5HAmsO= ziE&9Y#n1&P4xKUoxX^ivqd;iW$bxwPN$M>kyEArAGg5ugrSOe$oOeL1EN%mH|EWW^ zuaAsffng}u?QXGEW!3)NXf{{Y4Zy5KnEX)cltSD0aj1fdHpu%N0&TO*h*12otakoFaPX5YiR#`xcIZ+xJ zlKa{4Wv?WhP`6-9kFny}u#$Q?V7`A&-hMc0eQ&e)&eTN^)J$YdYO~GOLdJ@;U}(|N zDDy1AVM@=F-)rFU)FIO9v1l*qTgboST5oAfcg&FTxQu|&JW$y)PSq8}MbJCG**hUL z%s&6v`vaZhtGR;#-?9M3K$D`0sEmCHV4>V=n_t3w9--9p-WI30Kc4aXH0m4Rfw%%2 z@JDPhHPXdQ%l2=&Ze4&zX)z=m$_eURlXta}`p1#d6&GS5>Iq3E2_&FKC+I^y_bRW$ zD>+bT3 zd3X(Z|h&j;>^4oHNiNsaM>FU9GLjy(x*M;c`c&IsWv@!wEm zYcPOW7N%DN zw^l{`P-{|iAK9;#bM&1bw9J(>jvfghFs-7BU6)O2j{w9+=85$U^p+d3gq`9wgHY3P zRuU?VV|9s#qAU4QjLBy#wg}Es19u6&=(xw1-L(4UMV)!3F4hYLfXlm2y(IR_3uAZR z)oFm~9}3%O;UfWTLlH0@NapFt9dty6JcRa}w@+%_+Oy0m@!JL^PTj54w#%KCB}~5&vDS z+juC!0ZCohos%h+)GrU0F736;GS0o38Lk#| zwNH*J^g(o6NCm*#wu@g@I+K3j2dLur(N=)@_89nV+Gi}_-YjjGFEJFg` zkrXa?Zn9r-+0$`~qAE%mfsRe-8Ag?k=RIq91zJUP^@7%YaBJ_aN-3YAlD5uh(LG-q zQR23dCq0D@Quy|h=rw^`RVJvC)F}N3zpW-{#JC>L#C5T2iqp?^xWLyrG^k2=zSo%0 zpZ5}K5U>Hfc3-p^fl4avI(+U;g-qMs z3un^DXgO=Ch3!9b7eZ%3?QYp%(6qho2vz>ATrJV1W|iv^`-w|+hwEu};n! zO#p(x1cf)yxO)tt!JMeat%3-=uHW?!^5vjbs>nYZeqErR{nOVJ>nC@jCRs z`l24WZh-wUv|2r#BWL3LmZ-@Yo)a+NYG05C8d;np>_jbxazUy0z@)+4>al7GdkFPQSaYFp6 zVDl?r&%{}|*o*IuTttR%Qyynv7a#3Aab`o;55#aG8O6^IugT%C4GBHz4ir{(wk>Tt z{F3TkwrBb3&|9IJ4UD09*}eCY`v07!-#MDhEU?%L1?~`}m}%F7?5A)ShSE#i;{j3O z?F2Kq4Ln8zSOEnv*>~UE8Zis9oCC8PMo1JT)=0Npj6@YHP2FLX>s}Xgf-uQUBx@^W z<}+vnMt8b+`Ll-guovH%>XRwWZ`5xq36~v%PHGEwaa6eX#pI=BZE@JrRUYHi2gUb=SF7 z5tmp)u|fgN*ym827xk5Bd$BNA##55n3`Q{oP3|5SzAM}&EF(9_S1wiz2UAWpCEm?Y zgY$GNzt4@4K=nCRHoce6%`oQGV~9>n+n)F;*w!Zev#IyUW0GT2(LB^D{|pt?Yyn8N zZp+gja&&wR>#=X8JYM$WWvU57Z8Ip59xp@#${MBPQRsP>DWynQ>ezi4Qd^1|E`pI< zlX?ybA}^8g8sHhklRhci9yjzECx^hDPm}$TS#B6cmVQsXig?6`CDuA%g^0~a?M13> zER68DmLo|Ji!}!B`o@6Et_`5mb>>cgB7~rWRu0~6ZC>`t#-#AMFuSP3Pm6>nCF^Ek zxao0u?g(0Q!Tr*>=FvH5!?c>O9vgr2@z_Q#^g3kc{hA3|`Ghi+Y$T#hdyTX-v*LFJ?X}%Rr2IW&v7N%ws_>rFa$%M%L2y)>DkxyyDM*=igZ`zk zwvf-cTc>H~<0L$9jKhS+7}EJ$=foqOO7eiBetI3Wb6@rHo}U^VTwuaw54>3PO#o-; zr(<-ALn`y3?Cbf$mT6aU9a2Nxv`HuLE9r~TJvJNH9d;VdxHYmyueZpszCoYPK~pmy zVl%Id6;1Sc?eVKfBRl<}ayz}+qkkAVd~>?G7$FDuU(Z+ zb8~X*UMHe>`sA1 zyE`L~MR&gB-+%2|3EQeOk@QyQC6~8Cu|T**$f=Xq`M;YQs2M7vzoaR)G6YOm-tQS z?&9B}OuBC`X#-|Udz%-_UlqxI|D=EZQ?bTSFc6I?RizIs{n>cHRnA<+*t0PJa<$Wo$|Hb8EkG;fXO#1P+8t@>U5xNMY8M_3*{(gvTi{0s zhx|Xki{YHzwfm31SAfBm~V0_WfFhQCGN{`u|$L=n6H z{C)Q+Ft6jT}9YKC@^uwEA*dn{GUAd@Whcn)@KUYz&Ncj6PJR%Z8 zLXsn*7e5eL$Q(6)Z<(EWt|;hE=go&fn&f5%ul3Xj2Jhwa6b*eQd3V{*kS2KjO{F^xb< zak*LM&PU+9C*O98>Y;G1`16}z-n(KHt9Sg_`yz+-7p_be81xp}GlUoA64Q?N6xbf2 zO?*Rda(2hx5^;$p{Q+Ci>7l$E%!AL`a($EN&99^9gB)JW1{TK=$|B|9cw0cVnlT;`gE2sb2Uk;yX4!8WG}4 z@myQXRD=>*dX|eCI!TRb_ZJ60>J^yQh4w)O_WW5e!NweKn_0b&} z`muRd!!he?E3qv~>MSd&rg?)wMK)PBNFVd@sXS7l_FtAfgi3Sex=fnk zM|qA=(&~&R8jCrm9pbL5OA#gJtu}hH%&sCiZzyOspk(zl7pa8z)Q?!5G&N zVMsH#D^~DRj`cBlJDvr&#Q_DCo0k(a9 zVH`Y;FdcgfHE|7Hi^=-5t*calLJ`?`Wh87?Ypw(&pAy;>$0DbS7PJy%NN&AV*0Mjn zk_W3L?^a77INKn`6eKDdja{G5F&O>)jwNiRQJh+O`e1EJfvmu z0cXBOUtTCbP(L4CE8tei~l+l0LR%ZNhBC5xb&A%4j?*W-^|MD>pHt zFTA2UFK9A)W+2q(ejjW?n2f;mM`~*~ zW7fjf`DuSte9LS7u#TIucqm!kKDRTw-Q-l?rEzrI4~LFCz-PT{UaZ#seYROPHlvP@ ziPEZuLwYYtH^e)S*?YQV^+)vxyu=ncZWf`z} zLnEYd#}YGSmIv8fI=*r6V4bY;=UUYzR|r5NGbi)M{98TotJzANEV43JA5yQ<%#Ae1 zW>S@U*!J4fi;@(9I43dVEqr-+IL4qk#!r*348LA@!)aturQI+gsi{3pw^_|7$xl&G zJ@w{AhsnI*^mjf}dBb5huz?O7%U$(czBY#QjfmXQ~dtOhOY{<9f^BX3`lDTI37 zuL?FRemz;Tv2Ok05YJn8{9|_($mFjJuYUg^i8kz2Djj*X|D)~3Q~I#81_zI`(@@&9 z-|8T)PUDEF_c@@iAC9&$2pV9Jk9ixQ<-m}XR!$Z@5TqXGXLO!NY-LnFg06Q0(QAwR zC(Vc`gugz!+(a)l{gnADxl4HY?EG}77q@P?mvsfD?b}e>-V4HSG4(#VVxBy2c5iX} z8s6c#q4-+{%sIELEFUfr-*IkZl_+|00Xy@Kc|~Q-tjc0NN_-}wGl)vKRlUJ7nb|Be z0zcIct}eBrjtpJIv}wBtiatwgFaGY%_O@ugGa;8@?pq_B_i+wd^q4xf>e9nw+D;Y* zE_fY=3|CZCMHv}o1}_#hMQmiHz702VyX=Y6m}{JPnC$#n$4ArMo~{)r2T<7lemMq3 zFAqosoquPoKh26B=Eq(mlB!iV?gBmET=SZ$(fWZ)3w8;+qedy_Du=z$X$rYi-e) zsG4y@A;@VWtF39gBcpkBX(oe#pH3vdQH(sgs=-KO?9F2;#-hs!Da8)+RI3Bj8=n}J z#T1iK5y`O9s*C5#5^;EJ;YO4~_ah=~VlB1`3rB1gc{PjsrY@ALTmGHSUbCfsX| zQ$@D(@+V{%dn<;5&BAmHzr>-GjOB__(r9j{$ue)OO}WVGhuPnmr>tDSw2S#tX=W{) ztkGq*H%d;wxgJ`(l6~i3V~$etbKe!PPSW}z7GlLg{THawvyIVnPA8ptFbZFY9(a>Z z85L4b_h^OK3JSZZ3)6jaSRRQDn56gdy>FC>YNfLO`kdiGx`~1340>rydn3r*z00!b zrL0ii#hWuhM#83$qhl7buqT=w=qsP11t(-(mgQ9= zGSjR+&Lx+uFB`XA+(-FYC19_-uCk58Y@>Lyuk&owELe^R=9&fYrQRYAj_R}2AA$W2 z3g!i1^)jkV{NzoKtzcgDOqhW8tFalz0SU(jl>-8_eanHVg+D(Gv1w3&$m(d z7VkIze)r+GBuXl8jCqL|JvqcqX9Q?F=Vh|RuV&F-;k`HdIh}z&s;L#6!+^+hS_;RV zhFE^faPv+7EQ(fRISExkH$nvCZ zi8`NYncIFnoH7z+4Gw%~QFi*68{QJ8n44d~VEHL}c#ViAbeI+|4b4^vucwW_lC!t) z6`ViYm3>NC$Dy4*-02*C6~qqNOqR-CLm}By8iZdhrSTIY>-HRZL zCGVPFnP_)UNlKLj@8hVJS#6Y1i1TjHO%vS^aH8rO=+&*7E){(@q6XZ1A)r(?&9B>?O0&!1i8n^d^_*w<1NM@__DG z2_3T)-Lz$Rd|4uHnI6rdVyIIO4iX*l`mQwiIh_M@I}@c&tj=-jMt85OKU=|gdYvT` zr46;8YjzAP#l@Uf8BtAppJQ5fZ;YFVg*TR%PpNbtUm(IMC=>*xZt$4W=96-@P#ZW& z#JA#v(95r~D-_~Hpa^-8_Oxg z7n|-*m25bZni?mQ-BH+s`1ToTS6b4+`Hn8NPK?HP$|O{zjFQmo?O(inSTM8FVrVUX z?F6<&XG;sF*Zi1@=<=zS1Ekkq3PTnZIp_66D?eJQ4mZu9b1sn!VZ=JY;&+xhg z#IGd>ur`&IjV7doDRweH(i-5O2Cc>C$TsjDH7WYZ;_Ij9wP8tQ*Z>rUo5V|Ss&J+% z8JnVA!u*{ar|vG1jt_&^3@J|8Q5oKonBy~~CkOD^3RBM)nMqktVVB}g<-Jws!cJ7t zeECt$h8>>}_p1@OqH==3c=SDJGWoXb&@I@&w?f=YZlMbuxtyD0+XjO@wC@``vuEn~ zV|`s#2UHiTJkXf?l1BBh+QWoTXj-0~yxd9B^z_;FH0cmt4$+45@xFs58*2-)7z1uk z*EfZ->-L{cK7O?arMm+5?8P5m7-tOt;9baf%Tg^f-dn$W zl93#it)1%4`yn;Aw;u9@Dd0><+iX;8ufJFaQyVbPmQiyKF-f1|d;Pds2i+HX9oYm| z#xptu6=ksMN$O(<+MZywr-cUn6m&=XbEmpbwF*N(eB@dok6i6JwJhvhlc<^s@Avn+ z$ra?h4POwo3$D#JONxuPRQg|XYqY=Sq}^XgL|i-)6Rg8wB0Pqd}fbMn4q|4wkl)JSO!-?`}bQeP}a{UBCX2=c0W z$+5>!;k4&G5-Z8eTsTeX^u3hfL8OTR(Qz-Mj~Sk%@=5F89}CxnVdCVXn#9OWUb!Ej z^J=-_IUKNWH`9tKsC~V*%FFksr94UYM7;o8A{cO|Ch&Qrgfd=PfA0_W8_vB|b54#< zq` ziM|JMo2c#=2C_o;~J*PwQue^0btoDTk00E|#(f z)0BpZiq7OwkHa?(vC`&W(}v+I9K-?Xt>zwt_QEcF9{Boc{rWcL$~rqRU+uaCm{MwaO zuO;RRYp*ZWyM)C^y?&8mGg7bXE0|r$QIQ!bcOj8C>n^i+VEnnjCW=e5-VNPT&I<)M zp0O{BtDJR?e4v6iM*A%XkRBcs2}wPr>$?~=Jd%}#sc_n_Cz34rChRs4+`>4%Q|EZ> zyBV$=LB=SIU{0*)E{j^OHsc=+rva`3&6ZXB-rKNULbmnRyRlG5+tV)#$Lj}ERvnXMn7Ft5yoA2-6P{dSCU6HTA z%o{pu_vcUh48Hlw>*Cb*AunAtH>W-503qqhM@zTQQDisW_6K~%5?O?T&6qa!U~wMi zZu3}S@--RFt#Za#owt#)a2ZVVp3Cvxy0$@}jx=ZT@{BgCqf~`Gw5y6I$UJu`>g_40 z3Z^)J_j*T6t3qkj;U}rr6mUJ`?HXy%a+-GM8h21r(xt))iUp^a6aVPo#W2*!716Xp zMs=g9+q2&qM-Y34`|z;P;!V;;1}gHFUMjmUuH*Wvi_@x@4{@fd9pF=I=Djl(oV;Kr zC(U8j94jkJA8LI=sf8Jv%^U8g$lYa`jN)IL&nb}>UaFKOP3Y8CPwvpbE}FW`1Y>KQ ze>fk8*jmROH8b-yTyuhqh}hG>B~~Uh=+Ma+cAqZ>3a^;=Ua#;v@$zumtZI^Nk#!j< zsgi~T=8ognW_Iyd9g)Pmhw)43*WigMVKRBwOplYq;GDu29-u?{ z%L+Pye8%i1;HAkTc6e$f`eYn!Z-P%n;F0xK|U)=d(_v;`K~R*YCU9f&C;8 zA5O1}dRm3sbQDR!Fj4eMD}0RawJAbYzu-U1tDDWhRM28wg4E(GmI`h|=kSv-;ATfzeJ>Br(lg z#iD^o;3WbI>X=RW@c6Lo zbOr>UdM%CX3oaWxJ$3`$(ajkC=61&;!yY2aZtINdQq}4Qr3U4x#Yyr6D2`Pm=JYy@ z>=RzR_)sEf>HH_ZVa@J2)G$V&gJ_-)0&Pk8{9g1zkAXK=>s80`H1Uqp@pC?hDfu#q z)x7Yfg1M6_8`K4wb^J0~A3ZV}B-1&uElbH_zo+W;?6j)X={n2;ty!%!I%MJ9(8B5BHX>BLI^HK=2ovsH9=ZJJ zU}b@R__U)=MlClJtaIVgP;duvc^Xy8QQGfR!YzyZkq|{J_7K*8^;6cWd_?Rz8po@W zNcGw`>)Y`z+U}m9)}I5A)~v3Rp<_1PhqegH^)cM=wQ>CnL$@W~sB>x@%>HI<&RTYT z?hNs#2TelnwO+HQVa5(+=;F~fSOSp!N;5G%U9qE1Oo_F4vEO*KH?Ww;`NwsoUalA# zA4YpQS7tM|l2uCs#xW;#C$R>WaJ9T#LH_|EPn-(WH___dd>vKdF5_UX4&k~HX(UnT z_qtdJ(|HlgJ61TZMzf*cjawtJt4BlYi@S7UURUK<*vycy?j^}yGe~6LXy9B1$BerE zaP_5N3uA>>1puE{g=s72iy4R{X*^l5^km6CJLf~FO!dgy8%~m@fp+orT8N_cCBuo1 ztgc`;dhg~SxZ^lK(Ugv4k{bA&mFsgUE-ad?jE9l9nq{3;8xTdwijQTrJXnqFu#}oW z4mwH*&_@ceYo_xvp4zLlSVg~j$R0#F^v4R#y=z(%i*E8{&5z3rV-ox!-j_J<)Z#f00P;kb9lO>kpCc%2UO zxxn7@dS^MgSx8FuC@15_Aj>5c5MtGb_it*X@Uj>w3>zBeH}5}s<~*twoZ2+wTeB2?&W9lJEEtLf9>QrRE$(8&j zE(xBVNABeh9#||MU`l+(-DBOnS3iEfp!W-|wDA*UqDWXrVV)dXW5LDd_WPB-$GfsTG_-6^BA* z3P}eUMEfa85S{fPyqz{tI8*n|Wn;Y~CaX=WAeRXiy3JEUOAfppI9d^#^GvTp;lUYF%Ab8a=rgarOkS((=+fRS{O*EPKS;b>n-R6DDE1G5#Mdh~r0HZ3B7JGVx zO*<<))7wBQPYG?CFpYkCUt15;2uUPa2In9{-3lLR?X^*5&Nhf2&j&B_Pmz}hN5<>8 zHPM-?hlB|%AEXkdL2Kz)Fl~{B#*d9}@7zjpw3xF|(d`af9Vz%cCNKox0fw!P?D|a6*`5ntmBpxIzpNfkkAXhx? znH2rUiWY|bWJTkyw9kdVA=+}@x%Vr`NO!2rC31(RhKAC4S#(=e=)Fk_Ot$FEG&qDS zWigtIg(R9#f^}<-8P-NCX^fJzG(9`}^pf#F_jFCa{^IvO14xb3#G|w3sAksr$5Suc zI7tPstj^RA2UiO@0=z3dR{&Z%8o@Lss+DQqlAO#zbxVixWsOsMR5=ipcYuR5GZ|eE z$dEw7S=(l5p6I|E1?`bd8VdehW+iLs>^Bs|j1x(U)R(D9R!9S8W(Xju@kb6MzGCoV zF_H^}6zRDM5EJ{HcqcKgBJA3vEklfJ*kWU5{nDHVJvIAH5$<)9oSy89&(6iab-$)? z#9PALaUx53y1_V)Nh7LMI>+)d-uFm{JG}tFH{GV;TUHp;tWRlc*Pjku?mt!%DA;UQ zy!7h*62~MF|@VT9~ZhaettW%PQ5Ag;9txS%Pl}3UfQ_BdLoH zSqr`|PA*wO{QW&Xj@}Z_iI|Irf{fBf<1teP3NGFTVQ@IV>~;OZo6Uu7#1x59q-+JE zvgjI20D&A3TBgp-!3b9=t*7*Lo`~6^yKrzrHE-61Tu{AE8~?sGl%EGbL6n!iOi^x4 zGC|YTZb(j)ozVl6;X|nHMnXrUqI!#oSVyAKPCU|^jEj*HoTLuR!pqca>dK@CG>nq6 z(g!jqYgdWdd<;#9JUqwBElfhg@?JOFE^EqlFvnAs%LYEOcbZbuCA=fICq25gBc-uv zP1(vt9Xp4~vvJeajmO5NpV*iuZy5m`KbJSQCT3~!AT}T_ALcQuCUbp z{`*Kic3}ZYq5ym!8~5#a%SGM9B!i?iAJ#f9H7n}n&4(5yDoe9obaEX%7`2gWw<4Y) z;&qgE+^PN(<<^zML!GBd*bAQ zi^hlk{z!T=F1R@zyeTkrLly1bVuCikzUiY8GJ708kK`T--u9LV&jUzKX$0~l^^?vJ z9*;*xnRos4i@#AIxY)pjT>LFc6zGIX2b|W%w9WKndp14yKeLHAjnjN0=hW@3S%be= z5;<5BGIZ*rEl}ytzyFU1sU0DtHNN)AdE4-QwGY>$U`AZaHxGGkTbKWge9JGHryy#1 z@oepq;h@b5w z%YGD5_HP0C#TIN{vHv>)BI4`M*Zz}p714@!r#>uH%V`=j3n_)zV)RPPo6C=501Jwp z>{$!%^08~>)UT8B6<)XN1lXY&!2Twp0P90P6V4!r^o()K?`(U`wev_I-#tSl4r)f6 zb%Zpx_u&}u-P9odY)05Gkn-%lS#rqaXdu91qb&)_4{r{6^YjMR32&@6RT2Q;b{^2b zru;qIz8sd5PYJ|8w@7m<+(;u5@o*0*v&yqSGUR|+a61kz%?#_~d_bL$2|CY!o=eb{ zUR$0snm^7@e&CAAOiZw;!~)jEg>NFWx#>R6?kc_>^ACsg#u}BMqMS1ZTkczBZt3Lh78R7u zknn!xFh8!^N#*ywRaHN9TptK2*J8t+9USIblrz>+3V%rCuP=|u<6$#LQC?Jne4+M3 zuRA_B8l@Cv-@kKDCDeVp&JrmAi_|4OEVAubxYjY7V@QE-gdYug(_-Dw%8FxYS_6j*MEEU<$>BRk)NsmS1_7~dEq)Q_- zLk}ySRs`q2(|kr4GViK(X=d1B1MoCzocjay{8eL0Z($t2VM|=pYFv7$f@DRtxbuH(mLE-^Q4xKAQ zR12?j5I+$~I3r|BRj$h8W+D8`*`GUS4yy6R{><9vdw_~pw!bm?a@z_6?+pQ^-nrzM z_B0I(E#x&hSmA^JtJJCE$o-{X^r+}N);UHx}p-2!VdOhVGVc(u(8qbha z(wyh@FF-vDWI48x3yytLbbyvx(51XwHvhIi2^+*WYZi`8-I}T8k=?!b;8mVpJKbXb z6|V?gi!KDsDF&)Zz+9zr*=VFEYn5{I;`D$zC9M}n7$D5ix}NwOYuF7FF4j=IME*_)-AUS4{vn2i6I#j9UXg zHDK1Fa3!lC?o&Vx zkd2tpK%A&nR$uLg=&EX2L8+aotHZ|+C?e5Fss?|OMI$%0l2hvSxzlS%qAq6LB)%Dv z7b>KrH7kcef|EcWEEzhCr&Tlo%THB}jkOz+*O(TB5EPKamgE=5ta5^;W=ao;$s@1i zWv)TAsR6NkFTIdiZr>w>?;4%Upq@O$dDq;M{3ou6^+|ASSZ7na8MkY&F+9f0E=GxI zVq*lW$|+UvZw)SWTBnQgi4T^nOqG;m#_HFH82}U@t-aFOc0*y@^MI4R=NIgS;gFXp z*~=La-Rj+XkE&ICv}fQphv;{}__X=V+zt=KW|_E?D1C~@5fgnMn4QNqjR--Y40yVep5ezLvhA=Wg=jWcuhy&4W$jJ;A-l423^?2 zjy*q9l;7Us#&Ax)JzQ?GT~TIge*v*);J0gtxTrVAl62)di}Mcy&x8}SUI(6XxLfqn zir4{W^tMG=TM+w7dd0EN{a6g>A?_~HlO>O-9L|c0V#?K|+9*zb3}m$f+Mp!II;y4i zH7C91y%t3VT(?F9$IAp=|0*=nlI#mmF5eitNHSSqpZZqC)4fZ1TxNfG!KzVc^?u8hmI(0W87l5?%b$v}7C-d~VF+UKca_?y|WUbit zoCx^H`DqG6zisDsxh9{9IC4wbQD-t>yGV<=R>fOk4AANVi`tYP6~F@{PZCVTI3E`H zE)pc@X!A~6Dpcbj;ShghycV_?J=1a{A~o|(Dq-n+Bzc5C&H^M{2F6`tf$WUCeW(QU zKXb+t4Ee3-=(^#1o-6}`B)18G`qAr>jwdJS%l-DD2Aq{(U4R6V9e4fs082IyMe}ll zv>ZmFo0ebL?Tvn^{*!2<0fVPNJFauMW@0lP5}9l!^Ysh&CIS8E>A#2&I}nHAowk-c zJ2%h{OES?G$7nw(8(V}1o94gEeV0ige`Gl6_=D;kjK16OJh0LFW6}pHZM!)Hfg>Aj zBIj)mI8DcLra^NPl4tMC0k&c^>ZFZ7Dal?)Mq&2m9KBekW5n#1SH+k>T<|tdC@{xR z_~B~dR^XscN(b% z!117`giPl<0EYDxQM$wPU1r$N2L4s^I;YF~4_$FD(-{NOrUjRJ^0@r?^$Gp!0y$f8 zFeL8Kl>V}@&Sa%rV6`B*Jr0T3?|Sm`qgNC5>r2Dct)5K7;V#bqGTPNLkTshv-ADZv ztdG9@8W3nen%HkTs#D=HQ)!3TKz2mi$M6xx{0o@9^eQ)~g18C<53^nt_l{4`=_JQA z(I;aJP2&|;PMp6-xs@XJ$I9+OxY=d$mcIXufd15ICP$teu;E}a6LL3^8YXYX;o+Vz zaoamX+E>~vD_nFjtE8&xk^RQd14HQvPXn6)KKm}-tR{>yl-&xV1whzktPQ!4VJ&6x zCS9T*A@kk|Hc=YCq2>%mGK0ZO_p(%d0g(Ot=CkxpUh=T;26ZgVZwzG<(KF?bz!5nE z=Elb4f?mL6!TGR@0eh`xwl5hM%T2z4Am_rp;#>DQPTRHKc|PNvY`DI3N0tOY$39 z-!c$S^y?achhoX%i3nV_XqO14w9+9T#*My9;ZBdDuV%U8?Gcx@bdqz-cA+LxFsW`^r$Sv{y(J zP=mi{y)+o0Uok|k4Z)H%q*~rWq#%ra6`9`OpNSAmxwu$n2ms;bjb)|mN>EN|vSbzj zm(yUfJMYi~f4xxuvZ^MBy^^=$Sg&nea0paH5m4#s+pVm5k^3UbBu~t{5#0o&Z$+|8 zc(ZkUV~NH^>N%#WgYTQ_Jls(y#GbLm9I5L9H`5Pl2g01Vpzz0J1cXuQ;l9IlWlwuysQX2oSM4!K~y5FZ6d0RQR&ZnblwNx(6 z=k0_Ds5(2#_Vi>%J5uCBEX<+#10K>fITGkOoLK$?Ce@N_JNy$Be%%BFHi2c-Y5U;| z6^j7(V1yjID7HMrQ}P-}89iT$bw>M!I)O7L>H47%>#uJ5TFmSC^OQT`kam_KRjT>m zLbO){iHw$smBoB0UK8+sxzDT-VCJh!4FK(4oHC^NO4sMY8Bsk9#srjSWsDhXnErTs z8be5YBRAG*CecB6Fprc~Bd|Y~Gb?)TAm2+@$CKMYD;h%FmZi3f{@aI*$Cvh=yUX~c zGenNji(U-qC`N5vlccOx3GJmS2~VFdHAK_^j_L zQ~R!L7af|0ND(PHie|Ek&!rGuNPssc#gSx2<>a_Bk<$$6l;FrCw=xmZ(@4Za0z>oO zZ^woMxr~(5paw1dfIH@$FF8fS$43ZduV}VO(xBYygevAE7>l<^J;%kC_h$&q=7x~W z?H@%I)Xs~yY--K~8^2fw_r0j4qRq0x$6;@l@m=HXtFC$Z3sBZ;*9Ct>y;d{bxLfXH zk+fk}rK#&Q`C^-Ub6><0mx#iP;VDz>>AJ(?P(ZQ_dD<`;2y`={iYcbvX^3+vIlcbd zG2yY466cQj?BtK1D{6aW)$7|7HV}`2n8r&8d<(R^=9sCYxZuP5hg>(e`T4Fg2Q}8SKZ#Iiu7`tRnp^7j4%)Ir)$}vx z|Lg?-il+|G6as~GqTs4ruMa%#?^t-9JDt0puXLcM}PITn-zU09UPh86Zy{){^c*f zcjWic+KDj!7WRLpwOmghp$!psSw9k`ntHSJ2!GDeo#)LQj(n3#Dis1)5h+8xHhsaY zw-CWiGy5Kq#{e?HlaxOiA6+dK$bM4r$0xezxhi)g%=U)*RpS}26P!0Fc=W1XowuFr zKCkof$$2nfd)KN6(l2dRlRvc||JJhSnXda0_5o-b=Dbv`9fJUAh>y`2B_4|q9RO_Q z7X(mc1Yzxr0uXCMCm$;#T`#zx(VCLRpOAQd5lS=e3?p$=Gen)u0P?33_9KW@WPcwZ zi{$*?CObA7_eJn=Jt=7cL^#|G1&-FkK_n!*5S{&iA}c>Wa5wRL=OJ- zo@e=N$)!lV6De_1ii3g#HPc$;xNyVoJw)n4*pOa8&~7M}HI<(s)dGG8u_6Ia=iR6q z_u2F3!Qn*82Rh_zyDh}$M-KnQ1^LCoM3^9}pmEEcq=^T79t6eW$+dLb6(#zOt#co> z)3c$vQ1t^ulw_i#_212k27%ts6!w>!9DL*+Q zBGtH_f}zt97a!hOCGL8#2fqn=|1?2;)^n3y2!Jx`<1QUHX^a#dfniBkU4D>e0^s`y z!VF_1D|g5YEa&YcGfI4pCf!g0URN`8QpjL@f|>B&ak6@sYwLbynpUCcNg2YGw;+Zm29H*RF zmzzjU%Z9@H7$i{x$S^w#6vTMnGnmPStW_+;202T-`*!XH8SbAQ@8F!9=*m%xHTof- zc$f&#IzIfpLLi!|sPD-IlM;yAhSa|}0FOH3oi|cnW&K%BJX)*BL60XJf|cl7A09;m zZqB9&XkABfJ5k^JyF@@Y8th7-q;*=GYa=fR*}1?(T*gBJJ+8}cz!fywODs>S4X1Df z!S!RfeBOERben&$27r#|KyI@Pe>B3Q6a-y=k+9_aZo0!2xd z8S(_;8&Dk>4M{3K7L`;Qq`-XeWEq0U8USIKVP+ALe59CqV|}eUn{CpAu&@6BU)Zl) z#;yKNzK6)b3|>y(WH`_ttpQR)bQljWV+@lI%@@F@pj`2m0@-SvZShMg=f%FgXgcCQ z)Yrw2C_*tfS##<2P{4G4g?XI=oapnQjBkYZ3;mP6RUkkhxd5eSBp}TNkoG2e$|2_s zZUk}YijhG+NlDcl6juF%2gsx|uHQBapzXjweMLBIxC~G+f*l zt-Z0_Hey?{qUrJe3n=_1goPRg{9Gi7qK|_~-~F=`&(_WQ(<&t$`Nj=K%mTpFk|f)y zUIWl6NqlkX&4{Lp%>RN=sg=CVAt@;Jn8NZZk_nW2TmytNP0PyD27pR2#IVg%H;iL^VK% zyRS>!hvGsq6@q{Py1{mn^`;r_uMW(%uTwn7ctfc0BX6;*c_1Ee| z@Zp@iH1+GJZ?(w3H{fs9>h}ix^ZD(JTg`ZF5(!&SZc)D1OUAC5Kb*n>}+nQvxo zY_!20s-eGQ296!f-r{?O>Gg4RO zW2CMj^i-s77O4*dng!EXKuTtoi*tzVj-M(LVc;6?hN9kZ8SioQA{p%2_no@>LA59; zslwZQ9T86jBD|wX$~WG~HLvSsC5ag%iODwx#iY4pK22(jG>McZ!=hY7-gc^G9b=Q` zG9hG{Q#)W6^+fT?v%&1t`}mV(#dWhiGwG@C!vw9A?$qlanGXTrw8TEFp3}H}ZmcHQ ztQ<1>Bqk;%96${XgrFN?Vx`?}j8w@sR;>x*VtRCl+}}s%!=pn6*z)r7<2Cgbhy8rs zo%J0^rA&CJmz3gE_P3GJ`(7}6A4(ZTRt(z$|F59r-7F75%|7GD(SqDu5N(>_LiL{)&49aMwSa@ zb=i=WtpJ9cE+B7>P^Ne`+eM$t)5OTY;Au=ujLzZao!5fgKGOhb|M7FTuilv>P&hk7 z*awYT0m&yyvmR+WkGHr`vLOf+)znDKf)B2hA}`Eu>8m?BIzC~ym(HG_NBbI>nN5{U ztxr11TyhKMHW{sZ8(1cS5 zfp?1mdb0M&M6b*r2VnD*FSks#&Wf#C=BM&gMX&Lwb`Dm4$vAQ~-B>Wq&xyWlusWau z@Et;gG5`3pE>QLQ0krfGUhO+P4IRopg3L@@I-C0D%}E5!C79{T(3efrE)fC=R=R6; zn8+r}rSJWbUKEl4{o(67RZyS&M2720Sal9;tS^-4LVYu+2eWb8&qU%B&-d;FF!V%a z=ziG?gRe5{)Ag$7$5!ART(PpUdL!HPrhR>dq~#au_%nFnUpfI2AkF=@urmo_)c8Jo}IHjdV1nX3PEZV16j_p9>`x=oSXE zvA$Gc&jreJva{!(?Hlj91=N@{fSs>SvMz$WSCo5`N4GL zlUW6=q1=AD-;=Ezpes^POrBgutsfzcK!KoOQlWfYG19@UeZ-jgDf85=eT@G?dx%`5 z$y2@gVtU0|9e6%#hFamkfnO<;edvt8_uz3tgg*5~m%|Kc6Jq!Exp;qy%bL;S0-`YB z^-L$3p_vdWB*EYde&RE}B3dQR=uRzp`J+&NqNQO~%VqE__42T+Fmaa)b^v{5=UYT8 z1dTfoz&R=@$)qpWVt}>g)RlIEV7L8kN$SUHYil{6;ppYwj?}ee_iOE`P?#{GN3f`7 zys?`!{o!9scJoUL5$}snmW?tWT3(oIfbgx;j?XQ%DY_iR?JB5s6l61IBpzgD>lNMV{gYoWnEfwS6vKG@9z`PX~muDgyN z2+i5y*P92T&KviV)E9PMpKtU^On5q?Po+6O!;9l80M&3q23cNyJ9)GS8?j1TF6WSN zLH;;rFnf&BlhpAlnP)sWzMOSk9w*p#Y-c}R6?wu=cUA9aGyCb9iu6Kd;7qno`A=xS zzu1&>P^4Oex^ZHs3XKNlak8v1DR6o_rTgbcVwbBIjEeJt!p@ETS815I8PlD@9UnXz zIvAA~jo;2Kk^NDa`0wrb#hz_CqrbP~w{-li;`^&~-k-qi=u^1ygTy-a;B)awRv>dP<#wJZ!MN4dZ?Yno?K&;AHJv}3|69rXl4 z=^I_apW6%VtpIsFG}5#0<&a5NHU`ScNGt6D@guCDu#yTe9J#Xdp{$mOv}}}2^cEm3 z5G-v=k>>q`G*XNJ_+e&8nwuk;l#`i>WtR_=b0>aZx@pF3+I<{q#_ohbeYg9rTTL6# zaTvIxciui0f;)gl64-WZEANX~;GTYrYD-jo8D;?h;!3DJHuVdFl=0h)pLIjMtfzMR ze$g0eTuXFcJ~}Pwe??tg9cr=lO4g3j%Xp2Bj;a*eO*^lg|EpuSC760ldUna893glj z-IV3j)eEp4FT_vKt2@4D&%?i!e!BF1_{Qq6`?H%(HmlPW6;RE63~3%G6r=v*_`vOi zloVO*h4rQqmknNMd>)mSW-)bV>L0LXSP#6SprwF@UKGINTnRPpNY|TQ9Pq6Gl4L(1 zB{BZwDg&h)*%Z8R1B`I|9vBTFJ2 zXPc>BNm5+xf=SXfvZKYv!3wIJ9s^XU0O9WrJTik;PGy=0Y$rNN5d8O)EgM3g9f0=W zR}UlPS{-Qi%K;$RBHZ~9JSk~&3I?4JKf3y^dSXKLF$x`ZwFAAQh^~PSS6Z4y&@xQv zWEqu(%?o3^7v!shtXBfNOQ!%J_x;N12L0x|UUTZlxR%ryE3GGwQ!#S$(xzyEbmTiWmf(k0HuZ`uZCf6LS0Mb(RlYjiTjxe&Ku9pExke$4; zh>Z=QiZr@bkf8Ta;)j)a{ee@e*N?Z? zn3d3z`}df(r=BcY)44b2Qo;~oS;b&b^0n^KiYZdqcb0U5Cqz;d(~Os6JBV#9sDT6{vkWB~3@vp&TiDJ48I`xcv3` z2c*td3a*GjfNJ7J3TzFA&4Cy94ZoUXT1}JT@XxnZe zd_Kq9J2pDn4?QwGzG?t*-5ZRN&9|xTMWMeuNDW-BFD|+hPbTD@HOc>D4&!gIV&7f}{Jk!X?s4KXs?UZ#O1t*Y5?MC0lHf4kH41_*4 zr)-5Xw>MDu<<|5ZJ$vDb9cUM^AF&Bth|=N5F3D)Xl__baJ^AfLj<%6^aFocBPYQTK_Na81qDH`2%DnXafP-NO7r*@_755vb!0AG`B; z(|2s$tp6QC#BJDy7yC(ez70#rHVo&V;M%cAeO46+qqi^Z+4(jIAyScNDz?2H)UW0u zK}tx={QCNi$3*i?7AEeL94xnUsdxdSl02jEr(FGNDF2=2SGe1T{r7hK{X!A`-j3hW z@%OU(TUq_Ts;naTe$g{vlQl74{7;6m703HuhV6@no5r8L05n^z&dU5tM=FFtYIQIv z@5J8>TcjL!jLf!tRbN?OBTF}LzpkM>ivPvpzrlgqG+!*rPtY)4+_B>QahB#vV1*** z_7;-AI%CgVVWfMXNc^v9{L9C=3P69^A&VBT*S~T;wprBMG%zZh$gQrzuRl-Y3(W-V zHMn$lp58wJ*9G6XABXp=d2Un1pQ2#mRLZr1zi=tGk4|JSjOzcV+tFfAk@>%F^s`Wr z4?~*MH3^40NCZTppWox+SgVOToT~iSK5Dr8eZoHdH|F2W==MnWddbWDNOI=MZU%;cawUu>2nvb-rB(9(DZ-ni?JL-Di$ft1` zH*#*{Cr}t#;X%R_2?Uu(1e9bi&q#Y9Rg;EBM*Po`H4C_@pfQdebZ`M&f)e5J@B)N3 z0A-Kk4Jm7v)7H9tXC(eoTsECY3KaxY1S;78(As>5hK3NP86XcIp^TIY>F1@bUmHY% zbdC{!?sjM@e=PZ~v9T9)vlG?S)MVDoKM!?-l$DeRV}X@(EAyYSa^B-0;A$OI(j6j% zM(&(Q-^SV7j`QPBp&iOcsAC(bq!0IEO3tMyvk zDxUc=>8X)ux}~Kh&2GBCtoZ+g9_h;|H2v}D03ZWTJ3BkqWc&rHcsn|GKdRLAdYM#Q zEQIb^k%TJO0l1UJ8_1}atkTx-_c?66{rk@|zWvmrC~ic2byt~vYsqhq+q zR&df5{&E^E-O1Or_b?R&;O1%2B+7JdDlvw8$5FFLgcV>jS8~ldzFJ`Zizhz)H=fw_ z!BtT)RR364*Gp%>8(7@-cXcfYDg#Mz1HE87Y$YivNp^d@F71DKiOlc<(ndnZ7eEClN?deohWzAah>~{^9s8J0XdkOR8OKwO|@ zrR}~ke^GfQ$EL$Ol%@n%aC)HuTe_-6E%tAZ#&gN-^p^|&85~Uj?&a*;KXgk^4o_k`AD?P^^8lVX^`_(@ ziIeN=%{Q&hKDi%0<>=ON4Mq431LH@Z@0%%T(sCktt(LPVJ|F#cgR;G~l@xB9xwiX$ z8xZ|)cDr3$_#euoKby7|?>&Ip1pW&-Y8h{ox%U2r&}@C_G4a_Yf4T&vB+&UvZ|NPN zA7qnAU$MS4m7EjYOcPzJtc||c4d|~0ydrkwcw$0=CvvU(3thONjtq>zgt=YX8b8JO zKY$d?MgD{o-zBZXG+3CO=vugc+GE1y{Uw{N zEAtl?cMmdtcZnW5&j0^X838jnplR2CTCc8>eFp)~)& zogbF~!C|uQliq&m$FIB>6jA=(pID*)hrRcXr@H;)$M2*fsSplAiIlDET~>uC*_-Sw zd)}vHk0^T+DtpU1WoD0Kk0UEP*_+?>?!GIPy7kBJ`}mE|ANQkk7a29cTEfx%v4u{_n@~{%-^PZZ`h60lsb4 zea+tgHo)(V>HkiE-y75a51at8Mc2oTzdK+iSics_toSZW8)$BXiH0nn_4J(#JTiTU+eQ!4Og#K+++MfkzlVJ7^*jcwY`9z0^T zhf<4-@aQ;mk z0pBQ9$C22Na(3@Le0yw5QE=XK8|(>!f4{sBp_PJ?Z6+aI{NFHindDeP@Qsy+!Yuo$0DXJDgX9zQjuoU|lMcub%k=@^=?r3Pj1jbLrM2X*MgXD5N{B|eG-4P6r$Ese3P3+# zY)0}vDaO*u?D`Y76HvaXyv;0Z4>5|IWNWv*kn!$9Uv};RJs^FCfTz9?*R4R{8_wVbJCI z4mD8vGwJUyw?omTC;A?rfO?4s&^M?GrdQGg?Ri<&v%TAH0W)Ix0K!)ZvQ;a62%8EJ zzP9}}1nJ2X0&=+oAoDu;5fVX|2Ux#Kpuuwnh`@}X+->)ftopWjp!Gp>TfI3rFKb#& zGS~_wBk1pY;i6Ms>bO+oKZ|F=4lXX}dQ*T2b7R19v2pT(JEV%TtZv@n!srW(T8oyX9@!whO{zTjD4+!svt^z7ysqYEd{ z$H`twe*{({6jB430fS`L>FH_Y3LwC@DN_5+DDS(Sz7Bwy88ooR_&Z!y^W8DVo1#b{ zDTqkRwwtZ}{b!qb7R4cX=q#)0&O*?&itHNo0NvMCiq@arztwlp($Y%aEbi-WYYTKE zEOsav`F5{SiCF#V8s00Tql3LrgK)qDnckcBn%HS-b{RB=@~Ubb-=7Ksk{FI3_y=9= zpGNa(N9&~w&Ic*T+o1)~x8Mzm;%Ok}(r2dWOmien8rbi@dy;*sxJJ;m)+f*tblzJr z1MQ@8bSNo_=vJgs3IM%}wGiFc%6gZy!QYAO%B z4j)Rud8QBA$bDoDt_eE@V~Vu_NUd0PT_s%NlF>jhP0;n5WieLA0kR@U6M$NsNOj09 z=<3wjY>xdW*>=16Wx!WMDPVd=uRFTm(@d-K$ymLp#j)T3VvUty9sy*pXEacUgY+G@fnJl^vG|MxLAnHnk*T!wl&U=u*q5dejL6xDc=?&wDScBkJz&aL5eI(Cm%sN3LxJ|OX^k#P|*Nw zM%||V-W3zC&52Oi?AitwfTNy_s~GKmF4D$0&5dIXdv-|b*`Ox+^yIwYFi>UkUnxFy(HK}L z^LqM%%H6Hz-Abqd-xjcdKK%e=HMrLg6aoRjGRApjAE~yli}Z&NmsDeo&X-f3fm)kw zxZ+1YK7_E>5)o{PE&zZ%{35~c7P)rG+#x7&4kWNR6%`dVJ6_4?Bfwlc6|Wy0e}{4A zvQ+-*p}6XRer+bqlNVizvZ*x6A%taO3g4eB2=rG)Euw^fFtUUoyR=d+%u03O@3i0p zw*eZ~>>J@nKcaQzwi~P}y2v{*jKE3>V^z1pgZ!uKE zelgUjB*3D%80u91c8M-s`C5kNO8KV@O_R#;uXBGc28~QNvf1kDe!3F-Usfs`ET>x6 z#pm{>y5-+L^- zdK3;HbZho>-w#JlJ#!N3NT9~T30CRB^uqtdF``+u^n+Ki|D=8P&Ps(qAQnRRT>s!x zrwJa^8~+b@V3u%!c~Z@Ol5}urj~B>*)5(gv9C)@{!~mArq4s+C!KKG9K{~MV%-aVW zaIgpj7;#>Fq*`S9SI+ia&9}XHbicj$iy?)VKok$3E56WvSi1ggFXsHX7k}kzjNSn$ zjsF44541e~p9(1dAHM%Lc4_ag{?Fk1|2G|b$NYC^p1}W!c8mWZpp+SBvW6(h0rK#UJIoCuK8s_u7D&t=SEZBf%sO z&tH3Y;@sp?!;+=UGV@~lT!^@h6(oaeDhK{Cb?R6?pL-x1{dwuda>Dbe zanhAvXAgy+((ZM`Vm<%WVf?QTusF1YI>ouq@{^!lO&_#CPY!_gLN4#u0Pcf?WC$RN ze-@C$$ptyYWWkf1lnyG~^HND^QC$vy@Qh^wLpNPpmEHrn?HU$<-^q1cUtj@HNvmMT zw?_z{t8EPvf(G)N`T$XL+n;CQ#%CFONQRdWG)yO9;$2*7y82AtL3iQcU00`J_hMTp z%ONGGqZ9YPnNfMVz9MPqVr>i|c z`g&P^s{Q385Co6_EQ0k(M0L46Cw9JOzkXv${{0m(!6g>YPe3T$d1KXOcXer)5K%aI zydj#G+d+ALF?zLx54p6o^qhCK|0<+WJ_+EY&3xc}oQ^LZJW>DeAVShVcM>*St}lQ* zHxf|RWs@D7?x3@$@X-Jr&H9&+owW5Fe=tbu@SUqWN24=dN)j@vMmDlLZ><$An&!5O z^sGI`B20=CS1QQ7D4GuF1;i|kEUa_>4eul{=oGf@;XS}M4qo#G-LEz7Y)@Ze`kaQ# z8a=wE&rLUn)!_y6QnQ=)4(ubV$YbA1llw@M6y)qJMj*?xXDN0h(!CFbkax_kiQ`ft9j%zuBN#lCLQ+8ezv8NbvX^KHI zEMzLmrZ0TdO6{$P>vEUt46KzA++I3ZEad;(h3UncKq}g!Q5WaXPgXA;4|n0f%GD$} z!QZE0HyAHlDYaGSt@f<{Bg{{=kAp3%Uq+m^lP^j=%XKF(2!BsHUVt4dce6h;VwNlM zeH2w-36 zio4{-{3mvANYxQaTpAAI^D*J#@Nd=Xz~1bpuvnY z0E1b)cQCcb2CF#i+b@mxjXC>1+XE;R2*JwT?$4~>=8D@e_J^KvjravXzHy=}W|8|& zE%%`Y9&YQ=sVV7Sf)V3;DCf(bpr7?L155$>71-?JiysBB+DHIU*A2Nkjp3*&HDIwO^ZQK1fGW9UE1t#)!La;-bBy07QhycqLM8+5hOUUh z8-ifc0I(jgEMmJ*OPy@X5Avy&Kz=59exxQDRE2U2hQXC>F8<+_qlLPW$ftgsA3`lR zCXrj<>-0oZNTdo7U?NI)q2KvJpTuM2x-_&+Qtya+9k3a3*=vGLuFi~Snz zqz2WUF5J)br$f}~4*qyj^exic)rLP#_pj$ceH7yc#aYg*{G|VQ@vl`Nfmr{)Da-lJ z9vs3754dp7(`>Y|H~ip4HGF7+7@KCc2hW9y_<^Siz%w*FcyGf7c=efzp@a7Wz&OEE ziQZ04yL{jonE0{gZ|YZb{TH0;8EcO)1)5cD&W`x&rig<_`8AnZu7%6&Vl^^LvN)y z_*Zn$DRI~mbjS|$TF^i;odDlB`RJVDAK)f@z!uVJZ#ej-bAy9#Xq*;G`DPt{Gt-}c z6^;YzXb}H!FqW3fXp4k&k|8Smh@~l`F9+v8Yce(n>_w%Q#tH{nHIK0xl-4Y{#uqpr zhbSq%Uz(-9MUg5zAN?0Ro$!fJ_mA<7UdEd#;RnbC5=MqnT;8Yo;v_T9W|Q?6%EZ^wY!{gBno3F^;s_z8Bfg z`ba(bI~oJ);l2OR#4eIiUygsRzwS#IqPQJ>Qz;4Cn!m7J`_jJI>mPr2sC>Uq6zRbs+%X*dnaKJ(TcGlV;i;UAc4XbGN)2>baJe(@85RbX0Xc3_H!e?RMc7GBFUaDpUt zHyoT>u1HM)S7WuNg?P_Ze*aGJByD{Q2NT{?*j zg5S>rlO-5KLH4U4vW&(sEM=6)|}alAn-lSjpRdn+>W1kWac zbam3<)n23bo{sza-Eac1B8->|t9~wsTSmVrc`TXyfFVZOpeo6f{|>Ik%kty<)jVT~ z028{2NF&cL2L0z3REI%P&CzSQe*m#10)yQ6;t#geiNoL<73Vx;502VofueywbQN1< zZn^@pxwA({|KLWJZf{=5%8lsqZz>e6e~8#rnW0r%uge9B2FuQTAi_Sg*bgY&^e5XA z_1#IWZ*0!wNXF>fQ?L9%I#W+RcI>9B4a!c}{yqFyo2-(tOhI5rDfSNF<`aXs5ETo} zGgVIn7Rr)m>P#c zwSwpK!9`XrsF{8V0%6U9*7j0NSLWKB=;c&M?>gDkc`;)0RZ99TQ-{7jgAYr= zUw6>A5AcACOfGL1EPIw0#Z+eLlYAtOeausMSueFut_3cOl?zg+Kvn!Epp27vN-f#c zc@@xn zNo)Jsq<>7p>}frHT#cpcFhR;=&Oii1B1q^hte}yt@A(Bk4vJZzp*_#$;A6fO1(d9} zCa;3b%|%GMR)47cu~W6f!J`HY7KWBK%5tFNuFYBu(UF2QQUq$Fvmi-=t%q^|H1X;K z#?(nHwYBRTb#i5aR2Z^Jr2Ni~19_g&Sdf~yEXa9w0zsc5B+FF`u%z?>>b0@K00c)x z_ta~LlW%3%%_xz9glt>L)w_h}+y=u{YZYZPB4%R*a`F0Amrb`?8qi1Y) zwpTN^2cHkgJOV5c*a*sU73&_VD1d6Tx?uI@(6Q$bRq7<*B&g+!0s2@gHo!<(CA)y+ zAkvM*By_m7TKMEYsbfNg=h>h$+YG)gDI)2*+b3c$dIfHXRmR*B4k01{@@c zUo(S5aSe=qolW7!RkXD3qY0|pJ7oFlBtB!O#%D9}nP=FY;k0gXPYhw0%x9bW)X@>M z%ni%fEIF`~{&RoNW)_chd#ycg`MZW?K;p#;x&=s{bLV1zIQbOK?X^H(K?OnNAUAu% zE)|)X8jagJJgB7u+{OQB%inz(a#;C@1!I2=kid`Q>32ERw<|%c%KA0!9E-0A=px&a zB22Y4ohEp%T)3 z3b*YlMx!dGGYi-k&bw9Ek@!p$aJ#LS!9NgG49$~fl3LrGRl-4>CY}gQDn8N?E#M4N z@b39bQpJAM^qf16{})GWZ)Jl#e%&5X<|L@8I!jwU1bBzMxb!< z%zblCV-s=H0644Lx<}Zh?;c(hl6(Y_35Np^JeRbn=KH~&m4ejGsyl`bQ+A*WAx7c> zfSIOVogJ_li~`LBo!pIFc>CG3pO5^1esY2avMk`(>rse*zdjxl0}Kf=*#b0N=qm#9 z4ErH0wNT#}KyeukY}iHx`OcG%C5ufc{e+P6oMt|52L$LoU-4W0gJTdDF0kJGDfLw| zl*k5ROpy=^^io)}yPOg3{(r&`uhh1?$jY!eZx%dg}TD^7B!+Hn2R{h-5&=@nEfTLM~D z{H4<>bGJbGL<}$$+?o#^xp6X;eFj+eNkBfTV?UEslU+k@Up^?9BcB z<}|uuw2=JEJ|KVu*+w}#?ukL@sHo!q14;ua5Hc}WUHSu*@&p@j@CFE&m%n8<|I24N zf(gPmjJto(-qZ)ubk<6h?IGWMhP{sbK7Ps>O5jAEyHR!U_{Ri_qyIDwKw19q9R9%% z^&$xnmK6w#A6)HS;2Mah|2x&NX9OnL;CBXOhlXSa*Rg_=>(gV@rVNa~qy<0M)AUro z#N_O10*4>0i1%JG&fI*9Ev}Dy051?801a(pYjnYan5l{!?DP>< zB4yW)hB>t75(Q!$V_snXkJ$pVq)vIZG;{^H9247K_h;CU}#eu)Ndj@TIO8KgT7bb5L!a64L@BKUvtZ} zEDl&TWHw@74z3d<-_pg)VZTflr=g`4wjw2>2R3GpRn6s6W@J zlPK}JPdJ<4KWQnZD`0K;H54j+uB~5^dQ&R^5B2&18fc7b_EbIMJ^hb~0KHHgoRLf1@V5Ej1e#$@b`POBrqfrXYCa)xG^@K;+5a=(#v& zj8bm>EpQo%K18Z`cQT%Ry8M;+3Lq{R2qz{NHd6S0VZrt>jxEnYIRJvbz;6RE(giXe zRGf5t8rkQO{BFhV^Aj&nT$NAN?F#DjLeCWlaFs>LCE)wpezJ967*4*w-%I?oJZ8^; zb$omKq`WmPZi^->Rh*STrp!UpZLDa#taGJbw!m*^{GX{8@c`3=gR7Tv^S@s0>w_8q z4(~$Q%Ke~+^aS#ydM1j_BUOnEdH)&%1}mcdGn|`{DNt)@5B)aX|4x#6=vTkvr6KEY zw!MH2@G{J8swL9)H#hsdlYj{@%C9l;zpniM{7@tfj*9==@1cIrMpo%Judz#;Y2VtP zyH)KiHa_u7r~+ITxGYAA`M-7$9%FrR>4wHS2{=9vZr&x-zo-9W;1X10unV6dU^s$v zt_2S5s%nE%HBK)n%TI1yPr?1qZN!v~u4u018bKk_X;asW3)6=#;Wla3AF|?;%jo?F zaRvI0oPeZj8RkCQ?CV;@7{%4AN*Hk)(HgOci}*%$`;(t7u!+ucx1bpn_|rl;-{K`& zgI!Zoy}t5ozf7P*NB8L?R$uMIJF3_KF2b0wpT58M@nCP!crVxvr2WwC`}HC}e}J_D zuR_#vEp0D0a_=R7`ijqe@Msq}H#(z!+Bf?wcj#|Ka-ls=>P))_*7*DHUQmcl3B-^~ zSLO`V(jNHjEAM?i0RUw_PM`L?oS}57&9afT>p2i`;gbO4n3X(YRt`nSO5_DF5BqY!F@`%JOR;FA=@f^ zx%+FPXU&C7(KLK)x2M%&l5R(Zr$Uq2t` zowIZ5e3hI`4e>clLP>KV*z6F~Tyh|2$v>8k7;mDRvK-=Y1wv5wc~5pK1xCIa^#8e!3yh|7CYe`CjpQ=|7gtg zOz}0TA7DXo2sGW9IH6o_4;VXZoDIoO@fd=>eEkUHd?MQ^0%suh!7ORNmw`H_h2K13oQ6X77Wcz1r<5jEoh_$p|4w?0)gHkt4p$Yp>Q_Y!8r#tm?(_X0_z-f_@9J&)H$96wstA^82Uy&fY!3vb4krJVn?20(4q&37lJ(LJ=Dg*?I6bJ znTc9X#4kd??2CqU`LV5j_cTJ=4`I4(piAW`q?UMyaAyfAnGB9u_U+{i7YUHtCN@(f z&zFr2m^B@%3z{5qC%+9Tc`26Inj6yU=|4Kof?Lsh{b=7{=j}PC*JzKbn4EW~w()8# zM%wfyeRl{%PLKm3xh1eq5<_cR6wnSEyD#S>>Nxr|bStw0W|=gXfpT9m#A^bK3S(b} zwa9D-1v;^(hIoP5Q29)#B`Al@cLHxT%O#clL~cwPd#U*>zK4@e>0sebbwNRXuqu6v znon;1JzA@k=^f3&`?Y~H>`cA7@ln++L}=SpDMi-I63d?e>C$t+(3BnIC@0u6IWxVSIkuIKx;Cj-w10(dwG8ybyZD zm$;UeAGeG5cIVCr|4zcHt8s!a+bswkfCS|3NcWMK$?|cOfx-`yu$Gl}H_*6j!x23ja8WW^nP|+E1fAGd!>WVA zFJ`?#_wI}wVzqQm#%0tB{M=JxMt7tj0|j-pEP)3=phT8mVAYlEki6I=0_G=g)cb~x5Gp@$p7&N;QmPbFT?n*nzazT5`TA5H zzMdtwPlW2Q4P&4!t>D?G)z)h^E~_>{cXslx#kRskhQ|SASTyV3vag87$v+p*K=+UK zGlcP(iMbM|%xxo*6ri&F zj0YRBpf2XUIkgeRHiyAY6lz***(dLplW z^e^L~R0vXH%{vLIXVWSMxoW!={9A%vJU(@?X-O2VVJ_|#WNM}kvkiH~M0o}**1osd zue2Xk6}#zrc=JSxWP)f>hT|Bm0&bf--%{Ke&(n}HAJ>S_)w_|ojo|1K{xAzV8>3Wg z{l#qs$1xf}NnHg9)lFJuIH>f32EUx>pka#5&!xH7%}xwyh@A}u?F+A|h!szAO~mE` zA=&##;p-Zdk9VdKJBGbrnz?inMWiH%dv+dP_~^w42(XZBeMV1FpWN7=H?j}%41j(R zg}W04Wy@)FZWlb#5@hpQVX|^u3~e>}gw&@xYRfv1wgX&Xd%%1nnJbUD_iD^^XXPc7 zn8pbtARZ&u8%?vl1vftyn_4GS0>ZX%2$_)>${(1WB8gIxck))GZFy%6V}4bTMz5HA zX?vp!Vf{cX0&&yV$Q?91X|-dsb)PHUo;8)^o-27^Ep89F zLd1*5)?uAB@jX;4gcWtpEcO+h49HZI^~FZuuzU`eavIs{xweUT?JbLj+(@zwi{`o4 z=lxxHQe5-H7qkLwFnuFyRtri$sN}CNVzS^C%U@4Gj-@ny3ROzk{p6t~XlM=D2fMdes!#ZqUrHuIc18hrXr?Cz*`pg1rMg}xr=t?IyHbEKeqn?cM0=WA z+q}l8lFHuC=e#DNXKZVQSfz1B=}H|#gD9?^IH8Sb&1phC5YlYYwK6e?1IV_#;*J_% z#J@-LG1%|vlE2f8mi|)-w%RG8+F0{8!ei0%a;C2Ir_}P_h-k^H6c@IT4TSIm6jNh< z^Hg(*2{3-lKC^W8@~@WUTQ@I9Fb5&SYO|jJ0*j6$50t38oJXC!(TN96jton&SeNZ?&&|zR zF==j;6P7UP~qfX(qr}3XK@Q?&jmWjGiNoEqpa(0c|PzeG{0@Q3dSTF~%A)^aiKlg>bt36tFfKoJL3nC!&fF^QCyUt=8}z%`=4Vx%IicJlHa^ z2$;yids3VJ0IJi~S>oCrgB5y$AgDB?!jY9yTq{7>%E+B3>w-zX_hdstM2sEeb^~y< z!;sOMeDin5%QgXKA48(nN1Z@EBy!-%nVgzNZSw@H{?YaDt`d>C4_4xR%&)-OvNz%d z%>7+2Z!d8XN&0m|h4iHsLp_8*>6&mFco=>*-%ju_1|juz_7a zW0mVtEpa1@6oQjUL~<|}U(5p9p)68Hqhmu|B$~YjNjH;p@E(R#F zr71Z?QM0|!DKVg%p!yjOL4_kScaFK8kAi_=4QfIbKE>xYe8j7FCw}$qjRhaqmRo2V ziTOd5Zwsg2NtFl$FsHUN6y=ny4SVMl8;?*3osEUn;a$4omRDP3ZXq{0n^(dL>!BiC zSAx}WOY*JX?~E|a1AWHQIW_3^mNI#_vNF8;PZ}B$AY9BJh$&mzuH?k1Y-X(mQS+pAcop)G4xi#Ni;AcN z9>z1$8XzAYe^u53)pv6llxA#F+IZawDADZxtS_+oWZJK!9 zw6!RfXN*s%or8>17Ygnw&0fLx?A&U^6P&%0=GiJBMl@r?)XM?+h^ZH1pusNTs4fAX zQ1y1f>XF@~Z_T+o)`|@H8)n0OPg7ChDTkD2bQn;5kOfZDH8f!p zm(~||H$(sKHUZwmN@_s}u;uxyw}T(P$_ULv6Ru&BO2As2a~$VZcvM#p+@cxg8=3WZ zm$=;KBZvr3Hvw;ZW|YBR)2kA7W5yX7fqF)z9a6w{A!9Tin5ZO=iI?mQWWv{-x6-Rk z-#>_6jGQs&R5E!z_D;~`Eiqv!X{RhFv0^ZtLVR2iK~n6hu&Fy}N^Q0e@*oKszV8A& zfSW*c-Nj4Px5k&*yuvk39v-g+^QZCip2x}1JCk<8TS{e~Z|QNmv_Rw(z=JuMcIL0Ng_N<7s5u-KjxGu>C=!7j zo&+}N%@Jajs(9z}oGEHeKjbnan{RL=yIZd|aS(%Uk%;p#ty*`W3CzWt18LJcWGnOZ za&S4e$x7e$A-isC$grhw`oMeYK@@JpYN5 zrDwT)CdJw+!?LoB&as*vMytJXZL_x8;l0hJjFRVyKKe8G`hzZ86zcFQIOmoC_?LTH zB{O@}L!z|1YMhzRKGx^_N6QzbeM52XWOZbm1OrsXf^=FmGl=X01fC=h^K{%5do{27 zt+tg}U}COl!j7?-uR6?x(E5`z=6BvoA-HtG&Ai58CgOr8J8sE{U!$#1J)rFjWqs&8 zdV&09`ok{HBVIf7KF^yA^_C#QQwL=M%7toAhyEkv{--%6pFa&0Dv!LD5?1Ono2i2{ z2{H9{_T`hP8fbgjsV!N%-Hkt3{}JRTjo4tF{+|Q&p1eb}2qIkW}b}c%C=$K};q^ z+~w(HZn~@)Lo9jU3!H3E{o|D36e}4WtUmJ97hVmnmaWS8uM|NDD z*dju+e*5&~n2lMNip^}!QsGIQC}~JaM#24Pz=>CBNvH0J4YCxGa`~I;%M&Zgw`Spu zxOF>FPfa?F3CkT9>WTnB)onaDey_n7Z@$6C05r6)&L7~c>I#xjJ3DyD>Wi;Dg^fAe}K*WL@dZOz1 zGn@)YLfhe)0p+n8%!lBG!0NDhe%9>GAqB}S^RA_OwVegJ9`3+ain#c|LY0rXF_DA@7pTI#(oL>SdfM$k^_;L4^lDi*B}8gdzz=9?rczMjn}$0}kvQoyN9&S=mpo z>Kcn((r9QL?0`&=w&ae-${oRVa2sI0wqV%#t|iRv3|B+V>|)qAxnVb#S~HN6eUYtjZuWFk;|jRL!Z2N;~7)HMf0^`Nhq+!JW8vkRTAEOVNsy?>cF2KAz#S zZOFz&T%_qy`G@K@3FwxZKPyR1%hbdQ7K4(IhymrX1?8)k?ng z{+RxVFFQl}!mTL~-n;a23pRS!*~KmH5Qnmr7+Xdd33kv~oQB71CadV2p1|d2pmDoX zwN~mKQ8%nc`Q|L}I3@$d7@*AfJx@cXY_BG~3@8!qfGb2vRXQnQs+f#@kurfaX+z}s zW$C1)Lw?crBRX3(y0~*}4UGDbT7CXmjt@c6Odao$hDmBWc zgtEMhOl8#si0~;yCCEK5e2{Gqe4AtTgCbYNC|^O))Y1_e#(0|$*zi2ITjy~hWeOrH zGC8FIEd{HYcoCe43Es;|zGT;E1RDrzPtUJS)t&ZLHFqC`lI_FSjd{F~QhoxheQN!r z1pYhTp`}wY8B@}eAQqsl8c`T-(!1&Er1yDE;3Vk9`f+rU#2fJk8!g%|f=@xKK2)%X0>PUcx+E_OpWBLN|$oy%mzw=VUjQw4n zDy_vv_BmOe?=AzT@aoZPozqMAxx%|Nt7;^5x5ggLIUF!j>uG9d!Y8T*7TuxfJ zRg3QBr?Hh(o$v%YvQcD|wCxBG-sXhll~&2}trh^czy%AUh|*pmw6F9r6By>V;DV|V!v$c@5gKD`XN9;?pA1C%Qf7TQm!HdhfgG8#?H4I)dVi{VFV6MO2(LY}%WU z@phN0gK_X_n8IuG)luNEl+LtPnYS*}o7#y5|4rbpGFDLCS+3Lp$u=wPMYo$J@mf^odSQ_pHSbra=OT9c(ncst8J;=M1Ed7IHl}Xt z2q?+r}B@E8QysBNPzoNRTbY82n>iH zZ^r8n;i;ra3$|;JqQqxFLaSgesD`8)r}eyV4`Y-pi7!~_T?w%z_r!zRFRYPPJYHPA zG9zfFxs8?U+aO7vlmGF?xh>E;5Ys_pvgR|WA?OU*sd*3m*(>WKPm+Oy;|W`ij-oGs(Ar=0f}c4bId54}e*0gJ*Pf30su{WW zY9Bu?^0aCcTBH|;utJT&B8B7+y!@hu3oQZ3zEgT_&GLHTwaay2u?nb%Uy27&QP`rb zF{=dneT zLpG-Eh7fkuWv^LNHR9SwP;r9`NGLcsH24iRLZ;6;L4o((*^vb@kOi!TtF@%sX~9Zs z95;|LP3gi;Ag$hLfZzAw(6O^mjNOmONHZVCIFsk%dy+9e6|^Hc{hhe}LhqmO$x_Ho zRTTxR#!c83E^cU<^$tx406_uU7O3iJdwUxeExUmP2pe}(yR5Wi@|L-;iF;!t$k$wE zcNVTm4}*miyG5yN*c;9%4w@P#ET$I5f=qOurL}Q~i2xi_xGqjl(q#8sNZ}=CaPXfcW;$Tzcysf7Gu2PW{xLa z$q14+D-Ul_vgC@WUjjj~!DtDBKdjTw6!^NvX>@5ghq4TXUgjj%l2lA!)7N=K&&E@y z^{#X@scwEuQ9;?V7FC-&;+1OE%qDVK@{VGhykm9eU%*1*MTreHi?mMS|9qQXAYYWR@j(P$gmF68=Sp$?pGd|JZD@-`gEnj@J*T6tnagndtx;va$ z@s<{3`N+s+lK_|8mBOLADC=H*en2Cn+qV)pj86CQlfiutH%3si5(=sIJrPiO z#ke< zkJDXma6wXY$<E&Ns$pHUX?Ks>;Wd2SX zi@718YsYT;WIq2<;r2Jy@~)Z=b*GCh`sGsP13+RB_c6ci6sRlbp+Lmw;|XGOJ;y@@ph;5y#D|jZb<00d4*YU?(v0FHSso z*Fio+qu_vX@3*Lquka4&L#`;mggErm2YrDp{pVM6p*D~{!Po=CtOR0t0bsLl#N^-u z7&{AngA5IS@51{16aLuC@ZT^fW2k3l<@6s20O&!Kb}NNSzv}Jz`DuR(>Hqo9!A}Gn z0LDANX5ir6KLUNA=OCXKoo2t4xc{K_LIe}MAA|QLO`G2Jns9^HTf%@*B4J z%b$avw2%SN?W}phzI_P$f5RGl!}t{Ze_Le#S^I9#{@=Dm)TQ76KzowQ7ZVhd<|m$q z@&I;U{q=>hw8ImzR91J-a(N^sq~J0uc1e$Y_rj22x^PFm{f4ss^iNHJ z6GCuE_1v4G=n_N>F$t2K;t)9oPzAE<1--z?oz1zJB*_?*X9t-@6fpIH7sW$cxCbE& zAtV!T`RVPkf10TssY{^5QyW!?av;>BoeMOSQg20=6zTT@U?h@caIG3bB^0EELut40 zE84fXK<%Jc)?JpqJHJ(-uUbJjDhO)Qw<;e5)S||C5S{`;&$PX`vx`_?-yGa^&X0-# zp1(dIxt(q>{Ff$b&*lZd`yChB+nZDBKOMpF^fRG|_mhAp#Gmcn>KC361d?yz-G#|# z(zj~^kFodaC9nXegar`lE`cfr>o!QleU6zyF7r_vNPn#Gd2((ObjOiH8d0DoU~((q z#yuhm^M#MabGJ#1GOvkJ;h=yk`;eJUtWp57pksqSCKiYkr>AcC;8#wqPBF$oFmPVIjXw8r&a$`2%hnbyEVmf5&;yd3O8)j9k`PXf9G0OsH9(@$*zPbAe* z3UsJX{EwFb(Zyu!@}Y8!1KZBQu@RaA9Bp&200=x$Cm zTx}MQ*=SF;=U=2q?7$+wS+5h-%2f42|LTSWMXs?WD*p~?w^7Chp>r|J{B*)6CP%u z4aP%?^lW3_F%icr4G?~)T9RJkFM1e)e2GK2j=7nRWse$+t`t+s#j3RzYA&Ub*!g9p zSgCed09r_QvCkCY6oXzKY6P&_pBohHZ-aUp_g(p#(^h_}ShN`>NiZ=Ak3XW}>^i4z>Yh}*e<9WzCj4#L&TpAFEFg}+* zT~`*58tk2=3htO;8@$0Q@l$&_EFI0B;%s_Ap6U-inLroD7CaO);}~wMewa;`D>RFi`>=J_E3GR(-heg zu|zPE;9d%W&yBNdFZSHOqr($z1KJ%flU+TFX)zx<OjoM z&7HFl2UD<=!cDSB0Al9)SU7T*l1au*q_g0P@$8`dp#0p8hhEVguj+mEM-~CTi>oTZ znzr`oo$*gTT)X2gJG!RV-%qqAAffcGC7?%HQfP45?sCyCCF0wd*fpH{sFfbQ%l|hb zVg0qCb2qNAV-O{#w1xC@&6H6Fowc;Q&fdBfowW+%-O9Gq(`>z6mG3M<=&n8|@g38_ zczh@OChM8U@0zD)H#r9%L^WAisl1+;;__d$<{W?Ez&BU3S<*~CJGs0w7uel84p`Si z&Q6o~!)grZ(So3Ei<2IHVADm}eF*Eg-KV`0MnZ3D!t$X$2N1hBD$*TBN%f)5+Rj%I zE}9BDt-^S0&4__w+~g;J0W-aNyu z_m3a|7pD|nJ&7nSnq?57Odt8R5}-Ev9Amk1;cb?Qcb`3?O5XJ-H=JK4qA7&)oG0(723< zf2f{C2}t^*7I*5V<4>tMuWO&I@FnGDJGP(-kb_Ggk(~(49-kbhkAwA8`(=6n_knPr zRc+<1A$q<5COl5iS3FndaB{n9v6}bT$+Mxi`PN6dI&dXH!sm>Ou(z(?E%>}oh5IQ< zUi+nYPTm3<-8J*_>0G@KKk?#=UwU6 zLjANdSq#qkVZHgyd$|a%aqmsrpL!+Rk>J~@M=JKy05@Aoox+*Grw5FA1aT>hB2KJrYepXjx_|CN_~L8 zwz}>r7ED(wwkA?ljDhOtg7WQ-mmUh@B$04g>P@y|G(TfoFm9^YLwrw&?82@WNv@Ybrs7 zq8;=fizxz1{xCxg{Z~|rjmOwP#n~hmROTqWe-U!{wAy1&p#1R&%50gU+oD+eY`hc} z5aHi8J@kn~JdU2$$JWA58TvuxWOt0;4ag8=(G6tBMf6NHWP?5`Y#fpU>EsX%BJ z`|!o77|vuOu2a?M?qns-EU%J2ncAr(efk3U`rVkvUm>m~HA+~Q6g>4erzp~xBXy}h(;wT3AA+E_+86ZphTfrjt509fEETu4PLuFb=yt)+KPhKk z=v-TRdwJT?5&`OOPLpp(C6*n$7Pg?Q$gi;#`mk-48@y~J6dVZ2N%pG|a!u5Y)xA+48<|CK5Fyu^V}<{0_eDJ(9PJ@D79?d_q0>)9tk z0@<#leTjMzMeW1;9H2xJwzos z{;_3|OeOqQWDXw)qNIZ+fV%Q0w5?d$GiRG^i2#^L3hvO7rkhP()P`Vr>Rdn6ze(Zm zh4d#zoBF+ag~S+nSMxD6R`k5$uGfAa$el5C))56YboE9+2M|DF#(MX0CoS{Jd?!37WMMuU;1kQv8zk%D-YjroSz>>-UCQ*qT8q8 zIM>4f!94si3))q05s4Y+R-cADyGdq4+71fCl1Gy=+dv7Wx-+X`(}PmBoz0m@Z&I9d zfZ6;5G};+RB@v&UP%pBGMsG|6+Ch5vF4FHs`$qA(gpu6uy`y%-r&Z&^RrxJZ6FL5* zw+L?bno!zr?yc^7Qyq_c2+}=XQHQOE-q;o3lU9eJ7ApWs7@(5j@u*R~VP?{dXix6|}CNHHr zG7*}?AVuMU^Be91wwAnn6#$6Z-JbxNw z8kpYaU1{!kAHfC*PgVj}g=F@g zeHocLg~PETm1(;eF03!yuV0uA;fRLE0wg|GzH#;zxt!W1^mR7}fk+8!4ZmPcHtQU5 zx(pIY(m^~>*PP{w*^g5<;<8j6!&XJkPBj~3loOd^3G*}1q-#Fbk?&G54Imf!JlAS{ z77O63QWcp!VykK%JrO!>!l%yfWbYDa8)13WrXXtgxRbG(R{UAf1lYZi@F5N+^zmBs zDZ|c(uO)TY4xN>yVq@-4fOQSH@~3yolEq9;uV>%4!NlK3yk0-^V^F16aUIJy&Mi*t z`rB?W0&QSTQsTxF4Jc$`E;g_{6EoLpx;1ve9)Cri+7it-2&*JM-9e>9dtZgOY**4& zf7#Y(f%YjvD@ePBymADht)-jI>y9cGN{nwu^fv z-b$4{+Unkjz>;swjK5J^yCwxfFlR+JU}qX8iqHXJTv-)~eYGm#H#=aG=j~?siO!C0 z=)@|!5SuiR4El{0=15(@%ZdPGH>cq&-Z5we`oO5ozGz1BUCE-wQ%~+1<5WwEHG+g+ z{J}tgXmRz4xf|Zpv04F%IFMU?I=0z6VY%tETn~DpB`OUIzb`9@E-La%%`T-(Q#`el z(mMw)=}BI=`gc_@(?iFz@$M~S?P2p@Of5(HiOq+QY#G5;3+Dxn_o!a;M8;9Zbwn2) zdh$9+_ZvC+e~bgDl(lo1A?RFGDBgj{yUC{_z6rD_xgKGgzc z<8we#-ODn<6A#+UG+&TQ?{hIJU*w`_ksh&QKJ-5tn_3yPi(f{XBO1s>XNZfN6sfxU zpb5oxEmFoql$C;T%%^KsZ%of*Z&A!Xh){5q1*Q71Du^|EWo}$gd+J(%uWCbZlL+~` zr^HwhQyh)n^BbMBi+ZUF%m$muYH14koH{UTp#4e1%{Q&!?@yGCCqFwA)Al8|J*NZ^ zM5l7+26ju!j-0qq{jLH{R0(|&o2R`qw7oB3H4La#aFL!WEXF?kQwU|^6SK#aMCtqf zL?boP5J!~PRQ=LR^2pYu4^!TYu5A?yOZS}RFF#6I_EEzzvFmbs!7tyATOWf7LEQ;& zIc!rCi6J6#ScsMSNhX_u9WEH^ZwtZrM+DdZQX!>t=676GewJcr6wbvTxTrHq%Do-I zTo zS9S>`bPES%YN;Cf$vh%gE%yx-T+y8)L3Vw`9_G&gUC`c@=Y;t593|UlkCXj8DFlz} zr#uZT@-EWKaX#DgtC#H(oNZl6YLwymwpILr0yMzUbmTdesh@gVqw~ct-B~8U zlzQz%w~jz#beC$(d7ko4CU(^;@$R9_F*V*ht$d{;{nBuhPe4!Le;>&HpEA3zd|_W{ z#W;^2{;p>IT@em32;TcZG*-;y$X1?!oOg%+xnhWjb7OV|@jPvwQs%$EA;eKcL^k&d zuFG_Vom}}YmRMNTx(Jt*|GVrbAosm526DMRNAJwsT5jc5`0zjG&Ia_qb3@&KeE4{# zApCI0Eglztd|tCHtSaYz&yOn_{{L~T{4T6;fd5w?M@T81hJ4J$2EkIo dzyhm>ONq~DW8PdKxbeZC8TRo0d?UBee*v;OX=wle literal 0 HcmV?d00001 diff --git a/benchmark-results/spark333-vs-blaze300-query-time-20240701.png b/benchmark-results/spark333-vs-blaze300-query-time-20240701.png new file mode 100644 index 0000000000000000000000000000000000000000..71496e57b80be8b3a0cc3dd03b5e7f266b824fb1 GIT binary patch literal 92487 zcmeEvbySpH*S;d?C`gEeq@scn0@977L5R{FQqrA6C_RKlNDC6uJv0geN(u-_4JF;( z@H@Bf`#i4#`uco-d~1E{r)#+!@0q#hKIiPS$3_~}z5r;Z#sa{AUyk$Xpu zoWeVDkI;hGup{V4NseGZ9|3>) zkCOiVTI}eRBgej7M>}%F+wchbzds`d{)PU8fWOdp{`K#%*rWgcG`JRv_SdIR;l&>N z>-Dkg(ATaVQHOr_r1?!Ht0PB<&Om>U>c_N##}+(tOXP;E{n6>Z6M5viMFImsPoCUG z^LQdCbkf~z8TI-y?xWUJKg%0uF*1G7;*!4{!Na^FS_V_UarV7Ya_bH5N6Xk3mIcQd zPNLr&Z04zdxi<}8o!E+eZne6Up5N7`U#_pWszkTzT(QyW+8rOnUo6Bw6LI7y8afR3 zyrA2EeKB5wmYQDB&C*h`F(mMuTGDYeK2wG{9q)I(`>)TyH3oDjx4t+#_{>+8mK-~d z^??SC{|~+{q1ln9q!cvT6vb1#=eoH%E#4n57DN%{0o(f0*FpZf137>9jQ!?B(tV0c z(+bL_&;IMc|9(P)^G7dK9xv`HzWv{4>)&6hLlg4QJ>B-3Q+!@9)*amgMmnxHMf=<9 zas?BH7s4j-+Y{-2{0uIYaco3->hC`l{dp0^4kpRp9yZBML7yyl(}IjMe^1_jdEuVp zhO4Di#roUB{<9drE4Y6a<2SS$B%XhE#_w5z|1C8&-Y*Dmm-|tzf1*K0x$f;qt?3NB zb?;&Zwje^UF}!_fVKl~d_j847?cT}3bR_C_Gw+f_IJ=3m<5-kk>yrz@t*v($qu|F+ zlYAa3n##|~cM;o*i-|IvD_c`{90^uvP0cJ1S9jf6VbyPn%=`N6PU6#x;)w|%$|?7~ z7}OMcE5L|1R>;#NqtbHk$LSD^O~v5etvF)6 z63oTFPj`M2IbTUCu-Ot6PH#5%s7+dck0Iv+)kFhJv&7!^n1D(jU!@@)g|gF3ew!Gn z$%_s}odEf@qLmJnHUb6Jr_qj+Z}gA>!!u=TC__&Ic>~)B$25cV7(8c$rY!nyh?3sR zyzRyMBuS3GJjr*JFf4VKjgg7ssSG3bLLwJ@YKfb8g-XWR)srm=^xF;{O8N<;riB;a z>x14(u~R{=+d8(>(T-A;%#WUH6>W@0Brza+b7n7u6c02$KEbutYnA)NM75iHDy?n! zLM1Fo-L^}2VhF*Qv7uVavhz{e<+@YiF!J@q^vuZki|{#L_~e4WnpGp)RhOQnf+}pD z*Wi|nF)M0k#skNAq=?ooq1s{Dp*@c0R zZ8z^dkUJ64Hy%$gXa{bdn2c`Qz+C%4u2;FcJl{rUX8!easoT;5JJyYoPLo`|OJYwh z^W*R>-|ZMPgHNd_C1;I#DsxJo5^4WLxwl^1#naM@QNu!hw2IWMl$qRhdtnB{r-ioU^h*}~MAyBo{;s)&r{WYZ3DvAy!h7-yvUr+Wy4e(^ z_t|~xGPecYV$_#&KdKACKJ?LGKWQ1?NxOG2rB9@|MqaQwtxjL#Du3LeasE-P zMKhJGis{0vSn$V5KRxDBL*xGFfrlDg;$5|@?y9jo(Q}*%>E;^f3#oLE#lG-XPNjQJ zEKM=G+SPK^jj`)|iCP(ON#89!(eT6XI2%S;mO5fa1O6)PKD1#jr+hZ)9uY?qaZWU>0=9ro5mWl&A>F5Sa%!UN}CChRuptkpu&Wckwye{;)9HK}C2dmZOr5NbRY;7ycv9&Yj zs)1#-JsuyFI2Pl|PkOxcOSY$Pxn&?>v4PyNZzAZvydLoc2@7-`#MHP6No{Vd*|tgG z;LLEDKcQ=nY4Jp2UY~8xNJ-viANiQU3YGD`y$Z9A0DdDjtzsO1@c zf`nOHqgzdjMp`7}^87Oj#NG_Pdm0ZZL_xpv{&gCM@uo36V{F>Ua* zJ_XfIuCd7H9U87Xd2i7$AJkLX6Au=aGUPT~pCIjD7n|(NwEX0q3?tZ~a$FukUl#Hvjq883E zR^JTg6<<&lzrtpgZQLfwA<0<2t}PneWKTI%_VCUW-I&204U;x|w7v!Be)Fo+qCOlR z0vjKt6W2+jka7xYZ19y-4M~CZcQ}3(_zD_un{XyAp$NKm+gh&0sg(m_UkWC-x&N5L zzFBt+cpL3BtAs~IkK;fh2dl5@n9ZufmPGStC+{;%ipt4SSKpvdAub2@q09Vjdyj3W zvj=m<&Wu@-P7oivsAs+b-jh;bWLwppW`Y~VtII4&`JL^bAJKByAyhGbdaZnY=xo)N z7~-y)?A0_1+niGAJ8g97dH1--V;{FpB*jEW+YkAMu+KcTnaU_J74>1-=I)LYdLWzT zJd@wqnrGP7nx>R{9SMue$nQ{o!_7-NRHO$ zi)(~_GqozFy_2N9G_F#6d3Xld-{du`=DbjytMi^sL=!1br%?-hh#WUv*#*Q1WbKt5 zClcD$7lw^iUv(@DsfRMeQ{tHhXln2&msa2iY@1so%fySxp~Daal#(7AMWXqw1$jX7 zEj5PivQ*vKXhQ?ZxN=HwF_V`s%!(t%uFYzoKU(T|I>y(Jyx4EiPxYCs;vL@Yz=v5D zW*0pOn-#=W#8EGq==UTIXL<*b?~apxc6J?}yD2-2=aYXqoxn2Q(hL=67PGsYwA3}kvn!1rv;Ll@btF_HngP*Y?oHu%Wvh>#{h?kQQ1cCM z7^wHJ^3B-cy)&PMiKg>@RC{iM%_l_^GkU4+5-^~h;OKX@%BHQP;h8PeVeW4gK|fh9 zw@NkNqIxAYVfeFCU-{N-L?HKMvhjKcr!AIMq&F;7)$${&`M}$_GqULBRB2~k1&nrw zmMH(qXvLu8^r>o;Jy9^D;n-sh)wBxd16j zJm$~RbQ9TYdB*~$uGVJ+FBEg%WQsXQ*bL| z7a${2Cn%lGpV#?iU$2aVsdNRT{)L8kMhBxeIhwn(V?6~4IN2t^(4gkl z1GP}nd@G-|WqPNoa0E)jnrVjYu6dyaoj1<2*Bd_PYgfY3?Bue&>LMPt^5VYK)eM@| zpeIIouuclc@mE(Xb7l;q9Q3J@)hLF_8OHbqJ@CWP*%?cwbF5JgV*~{2DM7j24^tB| zU3aD$<@?f6<$OzE+gB{qkY24!bWqU=V9Tix-*DdGaiLtq)uIooki&B}O$T0&a>?dU;%-Hci8xW4Jec%XljoA`w`3gB85mMJs`b^4@L>Mt zVzQ=wgu&@lZwcBnspttWL?jOo%!IH_P=Rl`o~=oVMF2m$`P#Rv`ZZ0A$JNB z0nBKBI&4W2SW!6e9hx6`v~Rr_yNKnuyD_FO*OTHsd(DhU*wrqt&we;Ssr}>P4oWvv zt()(XtEtJfhS5jwEV9U<7*p2ab58yBr~u-OGKO^2oP$7Y2s`o#lmA8%!Hoy7H2-9N2ZZI{7ncRTQ53aMNOHFz6 zm6!_4)Q@y!Jn3m)j#RPN9+#QOEW1!yOW~BJUD%YoSvV4+l9=pIL~uL?1O;b1fyF*! zTCuarpj=z#ua$6dokTbtW!)y(HUGiFpO|(7Y-7p(?NN>;;Y4428hLb>&iDijE_#LL z(QBFNbTX{c@i;^d?NjDF1HPX8PW6ag;DpGNU8iM)9*3a5*peD84*-mHLYBhRMd2zhF9!vPO&_LokCeP^C^vd>b+9FMC-bd1K>&8h-tquJ^^v7 zeupq-lsEA`9-;pGi4wQ)6dS!&T&B{IIp)K5 zpP-h^b-xOldBDn}QOD!^RljvQM&m7;8_|63hddc>IF*AGvtnNw-v;&fz_>s6**-yS`| z+SBY@;^-r3BKd!5?uN zKT<$?#RY|Uo_9TaI;UQ*oxQ%|&Vu?jh*_JX8^7>9a>BFp?=J~a=0m-N)vyx<@+~#6 z&`DK%(MIUL>W7(&A#km*v?7r(Tnga0Ufm2s+RyDShE@QVqK0W~0VRIcTfREiU z8S?HTLMi30FX3w0aAp;emkg6NM)g?k53=%~2FR8Qi?dI_nL@wj4qsc(>n}I>a)eU4xw5HpnE_X%$S)Y_&S0 zzoEL+iFzWHzjd0U_X$-NNpS7rYEd)%8tFam7)Ec;Mcs`xqVjo9>d`819O;;i2L0AI zNzvhEA$_&Uti&@ECjILsTS4Prh;qb2yqz$QMXL-}0fJ3yr+$OExDa&VVR(35N8F|qKbLeGV zYEI3kYKxhoI}NV$?@nOc;pLHOxvpFmvo>@oUE;l&%!=8{y*muSp0?5;{L-FF=Iv7U zVPc>4v!f0q)fxnXLrtCAQQ^904Wj;}SHj+-h@86D3mKyfmG${9_(MAgH}mot;E$kY z`;)fGR@P@fEJd9_TUr{sQRiH~$&-RN%8+^eWi!VbnzN?VsyU#~ejDo&`=o6_Y=4d3pb9$xR8+9!3)k@`jL zB<$C{{QOTTEcoIoVJVTs=tnywmKxR3*@%u$I`wa7Q4|$|cdWknU8{kIL&%Hqw1n3DeH+(2n6H zRxC5Yl2+xHra<$5t6mhRDf<+$#bq=3+4$xZ>~VGe$c@+Ah3!ili|c{6~7gOwCLLL0jrA&q`Q{$N;U^jaiBDlu-=pw#th^yg9d4^0q%Sry0m%)8R0XwBaSD zU?+avl69@M#S7m8QLh^MNu7u+gtROULQ@TW2cQf|)b4SM3sm&&s)!?l!1uq`n`+Qr zWm#+G3}Oh~s?Q;hIUC6{6GmhR7&3}3GNAdyiv>Ki?aXjd7}^QEkg!Ck_3^nB_f^tS z_7G8wCi89(AudUF@AMHh$zbCh7eqfXTom-l(BaB3v3iN><;>`2CzU~!O$LOZCVJvh zf5qflT!Pi-r6}K}8mkcRXTzln<=$2{2sg7LHlp)DmAB!6HFLwGp02ClfK*x4^g=|> zJ-W;1nQ5Z%aj7ELW_{_Xe)Kl@sBvGVPsS>s$E2h- z$!QD#Jd>6%KfzM2To_#>&QU=&2SGR(VliB!+s3yOW65-xY|&f<%P3Mo#Xm)UMDVUl zqt-KpEB=G_0DEmX+x{_5l#?a}*-k>m(O-ddVa)p^WEiSu`;m5akR~M$EF-w063lXRefGOCpTvLaHsX-1SrlwNorh*5=RPFq65(Z1$W+ zy6I80OiLO?da<&;Q{XxHUVElXu*>{o$_D&*ZR-K+-JRtj@I@Ekhww$6!W*sqSm89x zMRkR+YL|cYi;fb9Hf$IcBPRFJ5@rKZnsp^g&cL^0#QstzB74{5%}5nfE&GU@;+-w9 z057?$^yI7ac(+Z7gT0S2YTQ|w6LwHDYLijREm6^rp4*-S<7V(&9Y!LzN#)DTT>K~J zCTEI=$lAwP_v2>EX;3)5$;0y_RuPtjPQ}@(Jf6~lAs$=_GanJg>37+hF^qw)-b%-i z8&bsMF9sfyU)3ydmb$F5N;tLO`wWFuA2$h0#um)e3Z;&a2xol*J62r>gN;;a?hfL( z-o(SL>jCIb51}_r;N1rvK6O=={L7z&;nzjNBGR&JNC>KY3zEW>T>O)r&k2XROx)49 z4MG*!Q@OB<{F?xRBsg_3jrc;Da`3z|CM@{z;{`Ht#4AZ@fS35qOM0l(29ihcJ;yuZ zNBhjIbU}=8wmuUX-QN-e`!YJ1OiZ!~1!mu3thUcnYDEA#YQ-Q@y-OzG^(ydHy-~h> z?GXO)mc09ssGd&5h9^c}BAn8Nwe;Xocg$+$WVtHc$16aqj0x(uJJ)5>T* z3NdL?x3uD{Pj#1~;?2h0DvRhW-+_m_q^kUO618(y9>hz8Rc6S`g`}>?p|YUej^Q~; z;BX6jV>gz(gxB_`z|uM&YCYRi3;eb*k~(bFECNTHOt^krj`N>bK+=|PcsA`vCoWd{ zq)0+#@^w299_sG+y;VgodFWYOM;?@Q6H8LXG{USLAZ=BjAZTJlu8*Dk6{2E$gppvZ z2S7M}8vqYXpb{az@nK=OmT1aI@;z?-NJqyfIfob&tD0I_8rcKGbV@vmIy&TOGESz{ z<-*q6bP2#?Z?Tx(+p5^R17S55$@Z%c`OGZRU{>uU+}vIx4BiQguYxX~5BDA(LI;wp zyg6%E(es*0pC8N80?#Jg$6Ot&mi$JhP9RqzaT7rH%_Qp?E1bI>#-iw&WHaq_i$+4| zgb)^wl0&*w2N-7ciz<~=s%qgQWCR?sthpeNRFEYv15kP!2-Z}M@r2FAVfnq28>OK^ zJ*~?xuHMhdGES$>UFfYYd5R42;%`E}8RNm9v)xVvCVpI;%t8clbPXg@MkDn57gVwV zY#~cuuOIi!SauviSb>Qb9VE@Si-NG|3LFb*b%3d);ozkN?!HJB*Q4X#lt2>Om zI8n5y+qbg=9G)b|8@ozdF~X3dIs*`_4P*8PhRjPX@B8<5*NoRui#d}MFt=b857y0x z2*G-Ry^UtS+5CJo`DyFfqA@+;wVL^|g^%9uO*q8DxZ=U3g^UhPgPufw7qXXh7sziQ zofStG7sCRL=)FlV#u72$j6GnX+|iFPDKI>9-v5ZP4dv1d&ldTO*TmR7DDP%~BocT7 z-M2)GmPSCw!^Hw}KWBp%7i|ID%C4d{4{|GXkD{~TT$e~+HGbhp`PgS!&p@!2$q{wY z&Y;S&bh!A2GtwRvJqz+F zDKsua_%T;9*iwR{t(v&1ouQCHT6X(1cD$ueR>E4I`q+oeYL0!&#WV+Epr)<04+Q&y zAoUBlRg$I!U3%5|HFwkWqip24Ctg3lhc$Mts!L4MsK)=?#Jdxe5_qV>#X35~%n2;w zC#T(2>BlR5iA`De-sg)DPXh?Vh#TM|t!ou~JAPWs!`FAGDt76rS4K>#2^7+jsR`X9 z0po?c`ff;O#jc|a*EQPF>V>L-oce3^cFpA+T@Rl7ZJ=23;r?2W>SGEkn635D#ewTV zDeu8_F|1&<+Xz{NklA1~!+Y!ZM9*Wjj-g48;u#yQ6&uIriTA$v5$JQKg@ayxT6x2} zQowNWh;FFYKH8eAsBoT4J2M1{nX&zdAAuTEE(jL%s;-QA2ilEuRBTB4eR@!)BCD%n zl%DTbB}TkFCa~uqdbzBSl6iZ*OFu@`zk%t=apYIwxb%*(O`r_C&%NQ|KM$_u<5yWu2* z%5W^EfFfUzOIr35o&z!2)kh9^_NgnrS})>3^3<{GN$p0{LQ{>E;%=YMBqAx=@u6rk zi&uE2{uEgL436k*%aM2Mq_!YSrB~srW!Vp3)16c2x}=6pQNyh4C_QBp$QdZMe_LW7 ztva_Bxb>Cx$0$lL6L-CLj6c+~wodL1l^}79flOaA*~2S#lXMyzxBavpmBdFmFMc%P zmc&{q>Qr}10C7^5_7<$+{d_opD2!!v8PIH?=ru=rndOm{!`lSyu^>Gva{{?hMd;zkJDmxr8Xs_D?;@vso#Zhtw6VSD1j~4 zS%1uy*~1K7tfW}<$`vnbh7-r*7l*GVMc$J!OsD9?n*%W|WhBPax>`hvhm3p9CV3!} zF}hnlLuL4gJRg~<8JBnw&dkfeSR)%vx>d6bAuXUxS5@_mN!OM(#$uXL>61qW7gKybH`XO1Y5S{}as^mYbbt6z%jMe)Ff6nWA1#MA6w zV%yymJ(}Yx`!0v7K9j3U#Ws79N$<4;<#1ZVjWU$fd*4hH&aM_@I7CJtBg1LZ*0yhzn`;7YdWp zh;BqG-)K38!^@OPZP&=wdA92Q7>y6Dm5iJzU|}>px+!xmTb@n@xym1cWRM=KC1*EQ z{FZssG(vep$qF_5K~F;v5;t(9*4xRpLUCJ^Ht{$qKglYWfX2qa9r|pMjy;gCEdA(o zhM=|b674fFfNVt$j#?ESQ${zvu&O`OC2eiz!8;KLj& zF5biUDX=*;G&1L)#Z`{K_rL=ILG)>r4>!~TCNr5EmO%90f9WjQR>270%ruv_Yrv0S zF77xvTGyVOW9#c%&Cb?E-x!TM$6V$Ltrc$VSE0w<&#~w|81za;tv2z@Jvf>R`z&&W zmezJ9nB8u+m!6YActk8FUPui+-3yyM9A11qt5pFjVoDqUH=;}_zSS)F7s_F@VuD%4 z#qwkqz3>C{8I-Ze;a{JM>$O>EP{5a4=+6@GVH0j4JBE5?-Y^JRSfIXS@(HsbZMPlZ zaCDvcNiF9})`p=S6U2=$QGyR0-K8#p9T^pOO$K` zcN{))K~J|qe>%>F!{0qH+2D>fcz{b9>6D&y=(8L)Td_hQrSYf=>&KxV_Q`)wrvVvY z-PRO!I_^XHe$`WvLR5;4{)J`l&xDt+dwQFD&|l8baQ2CuV^N)XAdw56G0MyGrerJ+i-n; zu+QT5*kLLt_8ibXdYbh6y)qc?xC$Cz|0r(O3hX|OcG;X*2H*ljHm(l z7$6vxP){)!#Jp{I$cU+#CAV&4D|i%*)SeE zQz+C9KaJZ2s2^;MG(sML0`>sPI~ZjkvkKk<-#8|aYc$LnxWx%vR->PL4M04VsmO2e zDb~+{i8`p@aG-whCBq^2FSW2bTVADLrtqiC#~% zB@u-d0(dUz7&d2YSSDLGxH{d{WlgUF zRvuwek1uo7Vzl99R~+#BuUBUq<%epxaFMcayws2WScyPfD>i_bsjI$lh*V8o5}c{^ z;mHNVU^)3)0D!2tPwxscwQUfg8c1@KF?YWM_^mroV4+V}uHO*$l)`0wP`eA{5ALCF zcehrq15W8jb~q3MI@=<^d26FW&_83qOqzEDQFt4mgoQ95rUC{=!4^U6)WPfFlz{#+ zy{QiT{j)pdfPgnj(fJbj@i|S>t>@GU0F9tcr9QV4pi^6ro*&rzxFgVz@4!?YaYs>s zXTol!QU(+W-!fqOm}~^HPE|eDZ#^^Bw$n@gOM(K`UNr;>m5Kf)TfY{cFNb zDNH6FTxEdw+31=G#L-NG{Ma7REA#8aLCuNvdqp5c-3p{i0qd2jI#Js<6c+G#)zOKU zRz{4z01jT;nLG4PGr-1owpk^=QM;G}V0<3<7*GG3FVrz7QvrfT6C()-(*{;POq=$A zJVLY;LbJQ(x`zzw1F>)$kg^!hqppA-d$zt}CEY9`St<-^+0Vb49i%HP1%}YLF*&^j z;awN&?_ub8P_2QO=g^&4oRK0K1!8)49^Q1UNj7IJ8oFUO#wD-}JX2 zBjI=_BC{`Hf1VZWlH{wn?Xz)I8YuQJnve4uuGfFfO$RZ`a3DdCtiqR6#V+@kfcM4} zW!-v9$=s|dlBwJdd9)FO@U#K_N4zCU}@HGgLuCv@ci24ycYEg z$QSx;K#Qs|_uprcQOe>L4AZCwJib<6n&_}Vkd&Z4TIn3IUJfM_K&Dbv{e@)b5(O() zA;uw_*zqY)pNz#y*Y+`IfM*+Pg4X`30%V=bO?YubPR!{gH`K! z`3wMoQ(}9Y3?r^$4^_ePC95B1u^#`lm|`mlDgFtom@UH06uKmngk6J39cRdUma1d(l|QV|x~m63!A3 z808aZa0&S^tIo&XczkbJK*$Eb&^L+HtY!QFiwMw%uFgOt9q(j{vUZ!nSORUdwb4xkEiESK@_0r;XRo4b9$nIFgwIBl}QR-u0+|^sxltf?}Djl?Q&yv zRhtFHORnr!=9WMuMxG5)9$5mma~m)WelyzRUoG93H1OnBAbz`Z0qiUx=52gDtX{xM z&v}ay!en?6hocu+GonwW#njO zVKh`95+|KaPkUC(wcZ3!wCS25s&jeRoS)NOvR?E9PkF>$ojUf67GsIlwa^3mFV@C= z@|6{o`jfYF+}l}$Tn5C{L{In5R&DH68B6qF4UjCTPXBn^b`VYVqm-NO@#R8W>BR(EA0KR$sY1wesYBLjmzco$RbCg!(4zqwDhYI03bF}p>ar7cNX8Y;qh?&pLT zi@5MK!U%%9<)6+2C0Wm21kmzUh%K|ZHm4!u$s`X!T8-q1;vu1Q%pr{DY0rjy%MM33 zCZ-~%GjH*(|c;_ir6XnpB3h^Xl8F{~UC_*R(?CxAPyz#BA{=H~yz)Yc5f&4BBnZdbN-V}8mq61mI+R438eUPFo1$SYQ zpRF|Sf(TR;g@D)Lx67S@1T+sM1z&JnO@z#tF>F{7w`vKSgf&^QGujt+wT26*t3YEU zaMm(8P3glx5~M9e#k4D0cy-dmX^3L^ArsO5Bs=c;Pe~Y9`n^eJnN0^&B#DA1#MuWz zdF=>9HnawH!1`4LM3@5MK7P&sWHeAVEgGpqASTpoyDp3BR+XM6=VpzOU@Tj{CrZpH zFQU0Ho!fk^;7xRLhL=4c63d%CYpk8G1F?kt<|s$SOSsG8^O(=Q8R?jSnA4-(R%A7n z2o*?#1-WSz1y%d;bGO`Iiw&}zBs`!S1Q{WWX@uj`$U*b_nb2p@Utr~pv|L*0pciQ? z4j?-ibM}u#J6&r8u|Q)u8(pf1A~`*RJXBHJHxDEU6ac(e+1LX`9x>Cy=R!9`DNzv)^i0xHvw%PkhX$u^-3#lHs zhByFKKhf|C0Y8CFrYKSS9{43ukSk4nquwvTQ8O`V_6Cl1`_&&4JKr=#9<3(lu)Obq zx(%>QDi~B`Yru<2QZX$`7#Bicmg1SoGj=K87_Fy0K}eTMP4$5IMmh0qPBW+`rWi=} zVwW`lz>)ybhYqTy!@O7yeBkP0Wqg^9dADQOH*3kPB(k}MJS-w%fHGH!`DT;YV)`UL z_1PY~2UMHfF5BzYOFWf-zlX~^W^1M&rHU_sm2T)CsMGpW$lS-zT|2UzUvMH$47g$e8? z-FZ~zVP;6kZ)eu2`I_6cwmi|6o25;$i(nc6=9aeLYIB;K(3{S`dI12sC&U8zSZTAd z_+<@bD~!*aCj-eS?jfIx_hjX#je8!ZrsoVs)YqN^6+@?z9P>bKj42hR2u}lX*X#9$ zJLyTY?@AvGDE|{1BQ zbg%KHd6`o!6;TFn-Huip%eEUCk@DJ`D+6jGqOs~#Zq+5C6I%vU85*{;1#GT00KpTJMTlpf$8107HmlePA9!zT;jf_X zaQQT--b}r#NU>S?5xbGz9+<0Euw`NrrJ};+B*?fdA1FV3lMpU@+E{OmozVqIXd6fQ zLI5j3ghLK+J!q*%@&`4cirtjxEhW@Z$W`(6ewjS*puf}_Q_i7IU167;OYs ztEZOR5R0_zU}$vy$;q_bChm#QifvuL6gT)QILMMgWu0`zIUvYRT{!&XM;K^ZuGi3z^Be71@R^2n7k#Q`id6n}6dgWC%F zd}28eIs$0pD14k4VP+$p;P&CH5kOMabmh99-0&Tp2$Gz5ySgo<#e}2M(_H zxDRkH#XnCAAN>8FT-kI3-=Hk?Ek1NI5B*_o2oCtOB)_@xf0pFWE;;D<9CGD*QF&ujD zI{^VzfOjVa75jpUy7X3n%Ri)uFYD>#_3`}89Rt*wpr@dkkOzx?lf=TCcBmbK@j@h* zRn8pXZnEtxCkaTJm2c=N1LO4#5uB~q+lAtfB&A%v1mL8~k|mb)C`=n40?ZvU@85Ks zfqd*W5RGDVJcMdexz`6CLq1I#2()t~A(p~2071T?SV}AJ6gh#}`{v|8q_A&^k2eV+ zwAh`f83g&Ep2AObD+4g|7A74IrBA zs@iTd1pAzJ*amqPLPSpqJUjm6fk^phcNdJ6bKgxjZFzS(?vz%Z39=8=+pCzB1xrFD z`qep$A|9ucBwjuPIQ;W(y-QFt^|g}?%K-7@&MkrezI-QPXkQx0FjYH4<$vlCsD@C0 zuzO{YjX>~=a55=Zxa4D=AFT%kPEJThsm~vMp<4WtXKc^3%>J%qf4b`45-*;VfqlmI zmJdWjy<;`jBng&1^N1o;B)tJBO-b4g{4Ahh31InSCJ=XXDmx*YgW9_0c6UlV5^_Uz zU4MJt?kcz!CyhXrI#eQ@AZ3(SO{gNPYD{s;d1EA$j`nPg7#_DX)UpLsW%$9n!J_EU z&;V4RNQ&a!Zx2aO6GfEFmtjtm!Q;gYl?AqfC62~8S?{fElHZ=w1@NobQqG8I07!;| z3V=GKujhuf3ewE-F6PhV^`IQM>tRe%P?c^@rsh6}&_PM5;>uHWuL1PT-v^x6iR@Ym zx(@M`N`N}?L{NeVF1xg>hlnTqpojF+E6N060tK9)KLPg6xX}XfZTsDgT+#GdMWe@8 z?}0HNix-vSvL2>?r|`1yH4up5*f4{$>nBqP`v0suWm zk^mRn7inF2>$e9Y$oQ=-TpQScaU_G|#qw0fneU$6x%P6Xt)l}q)8nYxb7+`wn2YKz zc6PkeC#pmLx20b2{N)Qm(nk*p(NC@z+=Y74l`Rw>5)vQwMBp|aFur5IyX6IAZ$J^) za?KR`Pafc)Na9j0V;ydT53yhF$3eZh@h8&Ep5K0?pTuK787+w{;2GU8D>Xfcrv2<8 zF9UO#EN> zf?@^GeyJ8F8JR|cUmDJOQ?hf-TUHTD>})o6J}CNuBr|>zIFHeH?WBHwCwG2E^DkzU z5@yX|?XP3-gHW6d1s~j~>lwcH`vj$7#7%`XpQE1tmr4JRrf-1Y;ib2d<|lvK?~A2@ zx^D>eas75o{Fe9pc@a*WUi8pko8a$VJ4c9@{!fPjg6>FAkq}YB4$}VYW~8JAc3ZaO zy*0l-Hh1*R&eTkZ0kH}CN`+H6&%av+K~loI$0w3NhLQyi3frXaFyzZt8$$UeYN^OD zs7zxSlz+SiNyOJs%bNgFjtuP*X8<3KZEYMRE&jb?>#oy%PX4O8I|B34cF2T0m4ZIP zqMJyy``Ri;Ro-|cpWO`J5y#iTio+X8yh7s3Uc*ZAJVqql&DHG^s{4sN*7=IF{ztJ$ z!k-Bp4gOr<#t>qJAn49l={A)wLH0*4Z8dpl>2*cTHZ&PGPhIMJxzu3A)jZ|!tf0-V zc|TFtizxY~57S9p6rkp28+B$V<{8M8&cWvpfQ!Q(4VY0NkR8)mnQXVHyPEV`!2&dU zT`o;cr6atiN}Qgao(OHlfVhbTlvK%kK0w^!WlEzc>|(&41|b*3Vx0Z^1mGBY!SakTRFM3TVHgnhFU z3f~)tQp#y^dfFe#?>B*lq0Vkh8|NLC7ocvV9k+zMi{g7Xi)__@dp{g;e~M8byMOqL{5M>XH{79ZgI|=HHd8Ukf$aC&MVjy1iH&n~xo{(M#i zY1&92TC}osx@`X{{%2bs2&u%*5-$z=ug&$xU|%;wmD+1v2nbX>A!0zs@fw3`tiOWZ+pm>O#M7oKH zI6r9d$?z9qH(!Cavvce+!|JpHTX<67hP&3~E&1>IpS<q@PdS5Xc)v+(EUfptJ6lUO&rTKL4jN1m*`)kzs~-?e7%s8AX2XR_kmjbF z*?#N()%6|$#pB11pT0om<-N8eQU$t5SqpTy@FehVvYkVb^YQULO?@u=p^VaGCr*ms zkQ@G`R`$PDb;=*CE8v{G2VE;ENI6XX0riLw3(yf0^NbgUstq@`HIIfZ9)B<(sF=pfYTOa(wtTDw2>f9R!jSJUDKh-yY+=8f%jxwuu zm8vI1o(>{+8SlHz}EW1v|u?EZa z3Z*=WQdj5tzxr{@d_M$tW*8Z_^_Antk86(`RtZGCSY86$fQswO*E7q?_@Mg0v%&D~ z88cS|M9>{D^?>>a19Gm$t;dheX3TpEOtM{eHosK3x+?R6R!W6{s4$e0n&Ic=H573% zL}f2s$KfDN>^~0>ONbV_6I1a%Lfi@Ag)$$;XlZqJ-lJCd{PG7^i_H5iNDGjam6hFk zl2xXCp|aP;+Quef@azfgsd*1}JUNrmMEpVM)y9#CyxBP%_yI-zlT-MIDkE zcMDuxDzXy~gufFUb5%66(|GLU3OIA0t~gw>Tr^^>1!*Lz4YqEe_Ty(1_+Y1aM2qyA?#4&Dg> ze-uC4VFzz=4giY{i$gs!8!Tv${*?fg?j_u`lEafD&+63{82uCl+O>|i-FL8 zl#3=Sn$z0a`?mv{H`CFBgZ6Kyf2if2c9O z45Kq^um&?O!NKyz%U^n_KtjVf(J1tJEjsFg4{aQAm^F3xI|aj&e{o;`I|2Xo(`pU@ z0;W`CzGcF1e)m1TZaq%0JBOQze(;mMzqu)z7v3xAmg-X!Hu^JQ`6F_axyveVuz3a|5 z8XkD}*a9jDv7?lz3nJq>lPaI?XEW7N4JuQJlA>&tGaO&22=sp<{XymYGLje|#~?v1 zw8^-v13pjuZf16_wyBSBr=#u2Sn1JIDp0(7iOn&B`*{m&xww_(B_0AE1N z&dxgfFmd(g>PwUHIp!J6oAeY^4-9D1s}!V-=U%=OB69!!edE4jwmqf)8UFUX1eaR; z4E8oSFzHOufW|$jOHd)`;H~}c5ynTuP2%CN`RVBl4p%u7C0?G z#Ue}HNKmUhQLPFZM1byk+lok2we3}8g`~ipO81W~pYCp6tt(no^~vhpG#n#XUjY2Z z(>pqa=e01d*!;WvAyN=R_e68Ma>v>YqLnwBZAF6*qC?lI?@n&L@OW!Sd#(qTc<1kJ zq=|)g>sn^_jY?*3STpG3z7Cq$%2|ekPVa`j0Bc#O#AJLO(;Ugg0-u?ky{5DWm`==~ zgy<%PNhBHALtPUVXW7{TUXE~zwg0u7?)IZSrzX(QxZn2ml|a@hJi_`b2OwLK7n}T- zEf%8u`o1gT5ac^4%f`TAIBq5eHT#2Lo)(VfrvpxSQOntLR63ZLw6v=3^ytUfhwJTc(fQy;?B}(s zA4}m$&pg{X#Q(ED@x?V8kO8-5S=R&1oG%ub_9?SYT_Uyz@IU9si~C-?;&S)tmGiEo z=YocJam6KebAbTIc8+>2RJ4NNmiw9M`7wh3rh2gGAaMm6CHOc7$iAQe7EVt*4hk3j zAp8cDH16g(?{06juv;U;8B_}1XQ-7d+B`2g3st>MY(Ur?BO}tK{MA1J{(aDNTp|}h zdkjhh5n~`!p}$Zz2WRA;cgJ}@-Ic4ewK}r^kTx#3$HEx@TG6(rDqwM1EH!e}f9L1LjUm~I#x zR>&6Y*Y+@kV+(I#)gI;2@>3^Uhi(MObr9hv_yT9R1^t0Ik`oKQK><#=ymh0j_g2x0)lBJcgUUzI zsxW0QixdJF3pyUQO$Ojv_n0=_kDbs4Y`B*8b^=VIYyKS_9RS;N0{9fQqmxs7e7ugt zKaK&D?EwMrX3>$R2OP!l|h9ZZo+)$S{{u>gT@ zHPl%)|L z#L#i7+7qn_m&K(hbRx-2OiV;$q+edY43*vj{Ikwg{+~`29!9uW8nfRi=tOXBuo&6q zgO!D#&yu%^#rUUJ%ef$W4oz$Ofsy79NG8+EoTr?eoPy$mssB9$cal?BeKSR$TY5T) z2)5!SeVJ*yUEiF%vj0>rn8XA3GyO@I+7_bfMAT(FrTz|FHJvd4@kVv!Xe}8PukEzr zHYi{Y_^NFNj=3-}Al@DIy2Z)F<(2xJuGtd?w$jyfP=04JPT7NTl5XS;<-g`McJmwk z?Zm$TGVbk|=G*KfiCLeCgQs1?vUH*@!&eO#B)t-SUI^DkHl_dL@q8emBp zvS)@L3*r)42TWP+GN%D>%y?fIn^6ak`V`dXsJW!1?dw|!J?w0f6r0C%b%i_WeO`E-fR;>bke!Kx8RfV7v)iCHx={r&%IyRMey6?0;L;@AqTM~doJCFAy**8FK3v5_U z4Lbn1$w>V}?Sh97j=bLP@?ID|(K~ zeb@Td`pW(5<{o*TXYbiFv#0!Krl*JT;=mlJwhcJQqxtss4w_v$VZQ~eeNSPd>oEJh z`}hq?PitoC9hvOR3Is!~1k18P4^UDvgVc!T+#wyCjc2q6ey^UszM5l4D#I~Q1Wt1t ze+_dz=jO#;FhSxyNHYN7Z-CpvC_wBPr=(hdn7{AzJF2gb{q~2vLHw;hUa#+*odZ}+ ziJ*1H>B}}7@0Zh9ww^kACa=K2W!?DOPkq(%@^|T;>VGjpervK&2!kYG#QS^V)>~G( zY-i2)Zke_)Kai3&bsAKS{<^^5{n)v$XX{n5YzZK@dPLu~^%p4nABgbh-R0jc;oJ7! zkrJ`z5B2B$Bg65t69G1$%zw%Mb8FFl&Cso!}cg_jPoj$A5W5-?RhhJ!T5@,)!7Xd_ z7a2gK{C68$xNiP?8voZbmFlUZ-Ic!3x0%el1Hc)kOS~ah>j;Z1%Vb=C({YzuS%}d#2!)%cto1gygM*d$ok$CPuUibI+Uv4VO z%Cc>PgggSElQhS$EnBAY_bH;dPjv!3%SloN+XVv;clT$Bi5#FBDy9$&P56-4s>XBJ z6kYE1k_1&Nax`;taj5}r+Fy9=h@nAikM<QF;%ZH)#E;Mt--}{35;>GWi$ifj{4@ zu=j6(^1v@1>-b4=sSXjersgShyiv5&dS-o-EdIps_-bFAKVW|NW%bU3AnHY5tG^>e zaX*&}TRrohJ$vvayFk0sGz)t(Gc$G)Vf2rQM}&j;m4YAR_i3 zV_InUsUc)#$_YzZHK1$k$g`?oIwJ@6KdOei@~w5rsYo z!RTY;e_Yj88Nq7i3A$6C$4+J_8X6i-^%jYcTIl70DXj#%)hOxe(-J*BJuRIDga~01 zr5Mt99Q{3xBO@J6uNMBs-%RCY)D&zoC1;i^8SEb`}fY1zCBsB~eUA)u(^wPGi$DfKSz-oQj$2)7B0nMXR>N zb=M}+WZ$RF(QG%0yh&xL(No!utqio zvcz)ri%GLmVf1G7p*qHs9v{-PBEOiZm^1MJZIjB(p6>c!&X{%W`YrL(ixoD6IbXTA zU=CxLY{v@d9``s>#4HH`adCk!%KU4v?x2}OKw5;WDyJ&9R|BbJRV;_@h6VE{&~&^o z+0gu&U5e1UtYPF*E6oqGY4_L1h*)IG>{YdxCH_~{9k{pJh9X}ge>cw=)ms#Sde%J~ zwkb09U|4fC7zi5d5-qyOH43Izh=TDv=)VmKSU9O5df{m$DZtk#yEX-IRt=Sv@>A#v zInbmzOaMeZ6*ptxa=J^O&Q~_SeLQ>qbXEqRPE;CHx4|kk`cpUL72?-L+fw}_7jXa3 zi6t+kD<~K4H_@tj>Pi_emtxJ+#@)ZF+Gnljx0mgCfPNspixjl9ZE?>1WM+t zgW2nIj~Nmn&Qo2Nue#u7`rfy;hGd0Y1)_X8dDmCn@Y^4-8>AX>J-|pkcrc}t4%9jB zAP0a{K2m;zhJp8RF`tJo$!*Xd-={=8DrXOioK@SQw1WsJlcYlkG|6-YoxneU zvggJ?u%Oby(J0GZw_&xhG$GkHuserI0l*9?P-O~gU>~Gq7IUM;a7n)_$4}+Sc7dwz z-c8G}TNu|ClJNA#l^;tIUKqk2-Q5W~Lhd(Qd41@E2m-HC3eqs(FzJ&K-tKvMd7&}* zU=HOEV8FxU#>U3epdy<6#+%`QdvBOWI#@GVM}D#6F#4}G5zID)^S8|{;`A#$Kp2U{@;O%7l z{=DnFWe1<`zfrPZGW36gl)to-{u?9y;mZ8M6#CyEq8BxB>i#3`%AYySVPx4p!lS?`SbIgW*RWq5-_L_Y@weOKEuv^qCwkqwx z?f;wuwmC?%`A}atg0gx^LpJ!o&1?Aghf+B}JeN;<;~)0gro;VC5yK?G1fTSU9S&}r zzx>^w{Kz%&I)I)2f87WyNwPxr+kspfBio40I=h?pfWFB`nI|+U?eqJt1;-ah{4rE#Ex6?;z)t?WyOhh2 z`Bho|{OzB26vT zOQ8Jo`hSmPizom8`H@_s-UrJ10^Ik5lG%jJ=&QU1-rBKgTW?|`%-(!8rHJLU9h)~8 zwEC7*W=is~>OuWoIKMt47?A4*T(ASXb8XS(V7#URDg0DiQeq4WDPMZxv4vpt(k{%1 z8tB$!Irh()qd$mXTUurRhK!Kl>FaiGUjUz>5?^HLss_msEAzlr;{e)ld}87uX+Rt3 z{>RuPy`VxWr1UDANQ(ZeC(mL+Iyid})1H>_)7E$qwm+>78z8=P5uXc=g?LpW%1WX+ zr|L`}?%Pfuyx)!I$}Z8c`@3{rT{)X}jrz6^W#|F=2bv#_8eKg=XLyO6Ea}Xy>ugu( zvOC21gm!FKeoJxkHizxqCx>&bR2WTiJxVI*8yo7&%tjlFTpG(9P0Pm?y30h&Oyd?} zW*y@k;})ceZr@G05aG7#zo!*;r(iF!1EIsQ6L5WJZ8y+3Pa|%%Em?CicLo4}_`v^PNbRP|#o7{= zZIPhp&M>&~I4l001DHr#;BHH>TqA_-b>F1Vf2aKX4qj(m!%zsYNV{fxf_{O#q#=A- zWIGpLCMG6g8_??w*LFddw4m2E%0P{4o#Fc5hM`&bs4Q)2wt5=FWfMcqxQ!_@ty%n# z9AOCaKUhFIPxb`c>!UV>PkY|oVOfknbre+rB{mladQBOdxVaTcgMq-zItZ>n_qK#O zP_I}AhD^W2>Y#90p3Of7@_t~*%{tgfAEru-nNLZM(=ou;X@!GUiy7V5z3WaZGaE%9 zgc=>d*jgt&-P}xN@ z^D~10Il>BNklCg8au(yN2N-P%_vo86;E=P9Nt|Bu-H&=3qF<6!m-8@YaWX;5S#->$ zGqX=|?{HG{?NOZW=W^$~LtT$1?OMegVp-o)oG1zKK5ptpQ+}}30Lk@(bpf-W_5&?! z`RxkqK5j!e=K?0hgld&z z!`jv95Fq+L0_NCpgUN$aPxP1%xVEZ_hh-5kSuF0;?}CnxfFL6O8ki}ojb7g1VeK8h z6QU+*5pt4uvSbEQFD0Os@TwS(k3WeIpi+Y@5ll75em3?Ho_RKat=Q9@+oo9IW zRMcO=_5{e_@}@%%{$Sb*LdgeCrWAw(QKxBxvIb zD5B>>041HeN8f9y{hItBCS#QAQN?6Qd792fy_sxo>4$?^Cn{FcDs=V>x+2zklRd{3 zSh?|2ozRU=c&nO}_4trAT5j=%`VVaL1MxSjb-?66Lf5CvROVx-s**99^9&ZZ}XAHYMZfIi5AJQz$f)kZ4j9w_(|#yQygZ*T>yQVQ2W)@iF0lKrNL;gOdu~qJokN1G-_u0N?_$~2Z zZuEtOQ7L55$jH+^$%(f^buBFcPa$!iML(C1?SWlBQgP;YFRE1872`5^ZDe5D_deFS zc1JSY%fn@F#=>W(GQxVhoS8IgZc#YoHSr1IB&7;#e2q8(IEA{9G^O~$B` zLvx0uI4q6QfBmjXGO$#=_}BSqpVnIW*REdY_u3}-0CG!bZz3KZ&g`+ArER<87n+Uq zPrLy>DP|1G>C;>9_H=l^ApIbn?xRE3v2%lY33gVGEsWB9Z`qS=>^33_c;lKzg`j zDEYL9-iw?qU_P%a8d=14mTld^+*zLjJ}1FtACWHZsAK$i)o!+SHWB9z+3lTFr=&lH+EFJyyV$IZbyZ(n z?7YU0ZSk)2G&|8Yl*A}@>S+H;Zj3HfHXYe}%XRBa&u~GWZjkI?zW&xb#eBEwU!Kb6 z^Q0m1Dr^@j^9!&3%K?5p`j0x<9Lu4c0AtfbbOT4%H~mLz~`_1 zlK=ha1e@k}9!}f3swkz|c`OFEKOg$b_ip~S(cQ{?s(t;CPW|IK`Oc7NT;@jJsVzV4 zQyE{GS;f`r=}C&Qh*#hnKr=SAlTEJGA|31-<+!q}k8V1tc)J^vx4HpHra`0hLOo_4 zBVOP(i)nHP78OiqU^RM{S1}wuncR*JQIpQ(<(odNGxi-b{!Dkn&h3kEr@LWFvpzs2 zcby+#&9FEq5jO&yb$w$zXxmp-I;lU)4eD+^C5HS&>Oj}J>--h#SZrjIXHx0P-vazN zuuLu4*3mK^RxC;Q_mGY0yIzxUU3>aei@#`KF~@dx=9{DR3(`F1YZkqH`K*d&URTZE zr865>@9fMfnJLC%;T5HxgEWRGNKtzvse#)mn~$-MZLg`M2eyxx2zY2qfS742u5_($ zmfzL`g{sr-$A&vBcbjQV_LRCb-2aWw@byZlq@;-A^qUEY?MH)?%R_Lr&Pb zb49_)5OIrc@D0myHa1nOEY)+m(|h{eZW1)Fz3HGFn5Dwv-qg}^#cj%%Rks|_JY3!L z1AV)tb-z?-#BUsc&5EJ2m`r0sVhGeJ2lfpB*yuGIFgq9fKvDYHyjSEyK_qB~r_)nf zAoNl-T^fXcIc3W#S@jKUdhrU1NH4Pm<&kE6-#XfUe;Z9_wq`rO(Uo-Cs{6)Hg8@yH zh|t%M-kUytKDj6!U7@|=dFOrGy4ell@Kasb{UNTSF&IeUjib6jgtn+%$Fb<>#S)p@ zs|a@mdd`C023W6Kt0an}DYtKyY{B8WXr?=8_j$4nON(=`Q5f+7*;Sxi1s`6wC$Dv?T?;hjami zQ#V7-Za^L!Q1x4}m)Kmy#?<@xW3ao{)O%TFTULU=*z5cl&tW@@sAX)iWfqTOz{a0L zW9jUI*03aF(VDS-V|bkxW5D}!QnJyWS14b-H$^6V*>cGHyf0f$8sgH88HE&DfGvAB zwzghBKT;tuq5j4wuGO3yJ(aaEE)6gC+QcA63=~l_H5d<}jm$(*{PW(V?qp$_nyO=C zPgDHbVdrFz^~|olFn%F0l$L;%%kb`8ekKdzRj>(a(b%D2cdraYfB5sWMx~Xc+(I!e zdv6QdYtU^`%OeKCZ=-`*QVd{{aMw76_lR!ksV~+q$qzLV(T+`Nn(2BU6ALiA@O=Vd z`if>*dIn~c$kD4A9L)yt6Nek-i#%{whJ=dEYs^B_LoE}QKfFPVOKuEiEG=l0cDr`+ zSf@2bQ~<)$JF0Uq$=d4Ns)V|_@sTiM`E$8FcihwxQap1j8WR@uFw6d5Wo#Qvo6fwL zHEs6qsA|f^l79VLty3D2_lAsu7MgLAb9i!>Q>u5hW=f!1+P6~QKGy4Eo(6k2KR`^6 zRa6|2+sIU=l5=r7-!b^mJYcE>dQvK>bqsYm=t4O70uPz$SYhe)0vE0R@$E1Px~>-U zqM<;!@?#}K0aA<;m?h8&lE=Cq7171183($QMz1V>@OH8oGi2?<51uy?WpQ0JOAVar z)n@d574L2hB7=_~n>H0({X`9*$QwpocJdU%Klizn(7>SCIbkaX&7H$@CRpH` zUK45+3dmGz)bk^Z&2|1KG+8VOZy648h&Dyk@iI|8j`2+rp3z`lc`5GwoT=1nS|Cr~ zp=zRirS=$nTJs}abT*3hXKP-pYYo|ExvFWG4^>`bSVll6{`9ZdGhGU_!=hUJ1QR^= zMb=Uo>W?Fwg6EBt71jfR!gR>;=u7hi!=i2LTS#g>1I&AeeiV)L&zSX}T^gJ1$?9EJ z7IZfpu+6@*d89E zE>af^6{=PN$+8ks1LcwCrwSvaEC5%W>tEdkw&)}@D#9o17uW3%|0JJ4db7W-?knM{ zyP(H&PwJ}%byR7A#16|X+fUnIXJFQQjxp6g*s|-C?FO5)Ezf1rHZTZj{v1Y0*S7up zFJOu3O<@bR2sf{ftaLkMaE-R#+q}VmBqAp0{yq@Zf7wN*{J3mwIqsYK`3P0A$ z;@W|Uw-DcN>=(yUNkL|MEM{A+?ysN7zVOu_DJfmJtD&SVtlmKP=!;ndE&<>JQ{6K^ z1YXnpN?t?pOnypkyrtX3PbdkS-d8yNEVmBJOm$zZoVc!x43Js#?Qr`NKxJ!&0v~xp zK!Iekz#4Z%N$$$I9J?d76$SR+a4>||C9>y_DS z6#19pPey37Eif?nsQ$8mHQO456bk(N?|+qWPVn(Lbj8@ zwF;O&dzRw_@iU*nBo2|@z)ypXUbtQ0@vq$XWn)$kD_g>?{yzWf1K!Mdut*TNJ?(hz zn8Gh#yqWYymVi(;{<#^O|KpG6@nuCH|K@-85ohZLQ>%L)&uP){i$mt?Jp7zKkVjcr z>i%aA%H^$N|IH6I`U8XA`JY<{e{`WlQG zV1c3dCDVA(_5*PT4c+rTYlhDBf*)Q_H-288?uK~g6;fyxwu9b$b+pvBL7?l5k;^AB zgWY2|J^MPXt_rnx&k?=Pca_4V^cR@gcZ@1e5mg=$vs50#v;(OLE`i=#_9&9$04q8G zUc4?^aZ2EVJ^$XLQk#+=KyymAami!wx>0pqJ{Oy4t}TBR@Q>`H4@(l@98)NdDdmq9 zcVYwNAisL?BROCeG)~OVPfxEa1#Tu<>;C4avV67Os>vySdm`#>+-iRun*vZ7w|96V22GXTLj6A9-7fuj&9_HhjAk zSzcDM`O5l2W|EqN$%26GlyOR}BNv$}Ia&9UGcdJ6j;&Fml9b?yf&KYp$JU6w`p0Kw zCmh=S3Ej~`+UMdqARvZHO0tg;N1eZQ>->)IH*+7wPOE}S>qD{E!e>tlhWc-glX~p~ zfChMQaMOpe`^*$SQ6@$&DO3crua8*Uexuv&U7~$^V*v3LxjcJ?R>T66ff(5BdJz2S zD)6FfuEZ#utUbP+>@jzI)RecVq2J!g8zi`(XC%e$}oIJUurqZ9XJ9$BVv}k^$-zlJxt9NIh%fgQl{bobhn*u$JI&9D> zscc}=)V-g$*Re^_#7hCD=<8OB{6;010QJsb#}l7v=ygCayRYU-$Y+E}yZ~8&zLO@& zIgFZ`kt4R`7&j44O=f_-R-Ok6W?<%!k=U^b5daM+0fvd2z>T$kfwLIh1g8COfQjnbN_ONL@-A&lH#RH1T-bqi4%j0yZZ%ojQrIMg)rbbRnq8t-$Xj_y>`*D)5PI%J@~!nWYfTk-Q(d`G zOOd}UY53wZ3Y-779s9yyM;Yc4U+ihcvu!6sk(l*pvvn|_;n98Om~<6M$^!9cq!R?K z6_v>(ub8HYGKdR&iNyZl^A$1B_e{3CG2gsZ;@F50L+Uvh#%bJnit6@3|L5Y$2>XDD z#Fi(A$P#u$H7ZQBVvx@=NYbVLnG}^))GJrotipVI!B-5xDY>#{fosFalp;!!Cq&{7 zN<`zV@SjHM@~u7>W!dOld7}>?%|1`=aQ?=$Z6zfh;Z6aC9FDdfc#SQvZ zI|rYS>Xh1RAOL?=Is<*ytLBB8@%N#;Wcx_zJfN2#c#xd4HeG%ol27=^r$C#=zHw!r z)~n8%$7p!sGfKSQiytXEbRFFwm2W6tF}@6kG5H-bzh${Jg5(^+rT3+g8TGiik>s6n zM_xPCsU8`dVp^_y=Smq0LKbXD$4o*wbn_l6iW1tsC%%N{tm0cg=7^l{^Sq9KG&@zq zGK*;R=^e<5Wox$FzVWP#UpCT^ZToh??-OnEir`j~x*f&~b34$yPw1Pk1k*kwxQ0xJ zKUoK+n9%~6Dh_qAT?G0D(^*Uw^(g@F}I-`zt56=fkNfO)Av1M11nL@P(UKdP$6%P@C z<~9Qyy|;*=l`u%T!|v@jUu_8%;N@%k!-#}+6@-(i3UPP8&$s6u$_!COnMZpg=63AM zKat@^yUnsnFvQjx^Wx%pvy5ER-dZ!x8@e}BQ+D^PrWx~9mVL@Gs(|&m*2U9A#2A}< z(3I86NeJgw_KgTPn&CSvm4Tg{v~8?N#Jr)gvFcHhu=9G;`6@9;KBpO`OJ=0?=G!qA zgsk)}Nh%I>qk`8788>!$qy4a?1wBNgy;l>F-DFti^Il-*=*=Hpes~6TI?Uci6%}E+ zI+S0=oJD=i>S=lGb|Xo8-2; zws#J;AB!tEc-Uq4saMw+GGbenAUU;iCWQw@XjBqBqdkL(yH(VTkBm7Ee{vx^tW4vi z?sQB|F{PVx4`rMtD2V$@3R0aDbT||xsk>kbnPSP|JT8k{Ssc9FDsckepSsX&>R((= z3xbLvp5+#-s|N-|Yh~T{!lnpcC@usbTdp&KOG4(PW$Z(llP9{_vsFQC^uf))H#ymRdjaN7gvg6D${2Z0RLs%!!l=a zNkIf@J>Tf%r?hsrgoOo5=Z$-~sGa^0%%rung#00EXd%Wd*ZpQyQqa)D(srj+nY;!< z&p{|dz}senr7!pIBufP$5qgfav8(C*eG7}?EN0vo2P(+{Rp46zlI<#!IMp7)gWnXwNZ zYTO5WH1|?77c+h4aj$*O<>abf4&5&Vyq(o-$tm3b5S;C-H8}A4fJk>5+iQ@NN}K9w zPvb4%PED$7PmVmLGo3EJms%8Zb7YTX?)?a0%Ev1JE_~v6$#)=_GkZI1+KLCG(kloHv%_F4(ku_D6>S}8PNNT|P8qU*CL(Z=Ch1&vX`at_Rt|=Pz?-<21iO-st5tEK*$6B9@eb{&68; zcfc;-`r9i7O$+Stv`rcojUDBpODHa1Q*5-2gtZx>*$H=ZQe$%Uur)^Dok61x8>R!P z?(!Jr#%#%LJR;SH4eMDY%OsyX>P7`C-v#(GKbKEt;mloI=o$)H(!Z4Hus&Y{7im15 z(Ca$nKLT1*2q5tFW_6DZ*FQJ_q_GEQz&%+}wkXEyRY)seRSm4|7)*lxUe6nqdPnkS zdm-t*zJ9#)UBWO6QJ?DAA;8PwH@ZIB1}gfM5F^a<{d)2+d+Q7fJJF?!R_$2=rxjHa zC=!Jx$@Dv4W;X51o#BCkutz*Cu8hJ=oT$37T$`~ix|=We95T|_Nmi*BVkKMm-oWX2 zd{e?vt(5&;+_0qlI_LaEGsy>Y8i>`a2jy3MsHEddr%VKRYM2~VuBpv}<|s&1|8hPy zyr48O#%sA-_nLDVT1|tTM=GqFHack--AJB`>k=s>XQV^vo>(vFM0XSz#!pgS0_lf} zcqY2T0t1pF{WWO|;C}_n{bG+-uem}jDWX={F@}cYtA`;0T~WgMVbcc>YtTUjVsnwOVQvPU_x@&JXNS(48QYBEP&AQLtB zq_|2(566bw&O1uO)9fqRd*EgynX=$Qe?`et66#`P?2yj=)6(5!uq4@s?w3*~B|9Li zx|L@16bw4@%m(8h3KkT1tkPXTImN0Ews)>JFbo$iuFYBAwMJp)TMW=smqiQ_wyET; zsfG(2Cbs#?@e|To&?5H|tmv&05l}z#_d%_za$2OUxa$d`va<5X%RnzHw=pTSw(R^v z_0SsWQ6Ysa4-gfEP1Bi6;<;Fhwp|C)PQ76E5LXa;&aEJK_>57j2bPcXgXJ5SOfzZF zjr+Qu`$$T&T$=kE7*96-KsCNir{%WxfAA?l1*@f4Z4S3AZZw-sTDli*KY`om+!0YHQcyB*{w!iOE4ki(&iAt>)Okd-4O_GRI8C9?WHy%y1X< zIqSN;J!XwtT9siV3x(FGhwj1cUUL81#&b{h9-6gTuGx)|OtaM|pOznh8 zjo{eRH@?V#eHME-u!&r#AaAi)uhT7;2{nm`&Yr`k@QWumhF@8*p2v@ESbJ6{qQ{J{ zGAq~imx(gVxtZwHTtc@7o@svB5F6OR%{XX^kK+|$XV2<;WXH)7=d+Sb*pKig@YJb+M;hwWct5uH{* z2fi;KlQR*+9KPkTE*duLoP@A0==ROW+eUl$qx1YP6IxZ|I$_7DWJ{7*(%!hR5;H?2 zdZGfG(h(g(Jx=SmPuooz5#isEWlfl$a3s_)Cs=0+CDgg-t-572h(!f7$tEI7#is8_ zGDMcZFUxvm8tWsD9}@?k)Tjtn3n3ORF6cyCJH>iqTfqlqtrsd{C(_8@v+9@Yuih+4 z2zV-1LwvCws4SQDs{4HFJUW>~3a`T>2wcJZ5x6(J<`Pz5V_Y0=+Y!h;WD)vh4@pB% zJpw8TffKhj4ke1VYEeCt=#i79l468Tt}=ygFhX{}k{~GJ{eJRdLVEjPUr7mZq?CC( z9$B%DvNpkq&Xjm(VwNQ`5n_QJBdH#TI%47zNFEbu}V zgn?_zu>o)tX=*5|bsLk6vGzAc{Y5x|^iP#y)J_LmwQAmhYRBQW!IXu33&m(L4{*0= zrOg%&1th_h%e>L$wPxFHsl2!`lBq9(YxTLJ}Aww_)hJnE7Y&l@~@KNpRCGQu~gR#pf4r`8l?i64c}*3_3O_T=KKd4 z&`Ktn`l+TPHS0owZYpkIM6P}&m`V5%#p6yWYVyj|^Tv;uG{wn`Sg3pSFdg#O-vosC z!Lk4G4EuJ~JJAi2pfFW52A2%LAc}BO3U?$mEi+Yg1^{#5q=}j_p4i$x%!OYf+`pqm z{mV)fs(sa703b2~250d3(lT?CrY=a#zokcl(Y~XF7p54$d*a81eRY_?k1j@7>Gs!4B)T7GD9=0tJ<@aMZ-z_0hf3C`y@)&71_|(| zGS{tup|I&smQD6Lb0&cL#zvYzf`m7bumMW1Ze26P%E9raAB%ShBZlE%wEac(VmV)phX;#WUA&^c4lIRsP?k(Lf31cdLM6IP!#O& znwl$5)wvH>u@F1Udr05uGG2>_G0a%&PZEE7oU3No)(#(Cfse*k5OG9)=xQ=lhp?JO zEC#bclUK!8;=}@TV?U}Ap7IYonzGI#VOrrNnHnLkBZwMBP*RnqaZ$Qw{9qLT_Gx)< zbVAEKM@0}Rp15gL=Uiq(mJ&guTsuH&rVwM>p$~9cBQmpY>iD)~*=x*NB&^(0&v^oJ z9y+tFb#<`EA znXrn`Ma7YgSt#@xoOPv=D$`;?wRj^UnU5J4!-~aKAGPJRsCYiDR{-Tw&6@D?lBF6s ztb&@pf7Is0N9`uJLPa!c;P)oM^niMXYR-6|*YeBd)=MhZN>a<#NSH#NvDN$scp_=sW8B;Ckp(9N%0{TF$XbR+t?ls?#m(e!GDeKc{!t^8cMg>;jqaaCI zDY*E_TBN@oax{^)fWK7LlWpvA-X9KeE2MMQ#JLx#>v-|omOO0Bty)X$RUa*N0eHOd zHL%lYwF+zRo>FLB(>||>5a6H?A50gk|H;gvPlt*Oh6)8I)YPWU2CxBJiDsj5 z_xSTxXbZ=kz1FwaNj2CXA((Q68shctx8OX8*wQhXtQhAW3!X_V8~Irhnj^AYt75giU49|De=#Yz@x4|V=Gry8w?>Sl(Z={7R(z0VRXFu({eT5wqky=K zuG0c#`*7R!YFl376k;PcWzRFt6(PL3uX{M@Fh|fp=ncIAx8yS(NW%?Qtiy>{8f9?3 zSasrrT!xnFqPWRAE8z|F1aT6&k;PpZKcTSKz4=a^i)W2XC<#-eDxjXnUvq&H>+rOR znx{sM5_N^up?9n+c7y|84to5h4GpE0W70bGo`m2iDV&hwaVjFYfk+(3`$)%|zfN!IiBZE5I&{)HH*gI^}C z(mOY<8V#HMSa;0qahCZ9)ehRWN8M!hJfIjNBtRDfHT&aWgA_N}_tp~S@La^_kkwlB z?0VyDD3b^gYe+zH$A;}unLmR?2sbRHl%F@z52&R;4HvGD0qvvJdEehgY?OMS79uhN z0Av&|q>o$dMrS*YEHu`PFR~>)K{L&I=gJuQ+kDaGcB0(=Ov(5w%48C`!?`3yjdX$H zT=u@oYDrr6N3Zx}`fn^jAXj^|5tbKc#DSL?fUz#(9F-k9mF1=i@lTP9o~(Lou=uQn zvn#8-vD3nDHG1K)SH*kIS>4ONjw~LXCH>dn-& zfDQ`sqN2_t=G(62r_QK`j>G$hnO>G_VxWx&6T-D;f!aBI%>kFLe#E7l>nSbTxNq?V za?&$DnN~anotrrcezxJf&p}q=h7u=&UcxxTJUW&(Bx7VQ`_7(hY7ZQJ%9lZ##osx*=n>(*==>=j^4V7E&^1Ct}-c3ORP4<5e0UjIvhO zL|kuzAF5dS`x@BjV<*PNrmC=%DM& z81f=qly1_f4!UMbv|f6N-snSTknL;&)4sT`bD!~J1zvbsWol>n@~e?#Z%nezo?g!P z6nlo*qqe#1K2nY@0x$YHvkS0{wutDY0x7(BYilkbe z!CWxqP`Ttt_RxE(#@k9zH-p8jI!CW5m)%vmD)Z!|-fRwgHs4w#*;n z&!3tZM6HMPemgpC3-xHfe=*vq#U!ogq=m2S$Vv5-%(2KLa}|V6qRx)wvB!H!4NudY z-(b9yK@VINN$RsryiFD~Ba$Kc=P;0-L<`;Ft~#M7%O9RA-wvl|gKt~2-RQ8*Aj8lf zi0tn39{2Tr5%pAJx{!#MBbps7CD6;w(`S%Tm&j5jT6Dj_KI^-5aQfZ zD?d9Qi^j)ZLrG6h6eecGvAu7r6;nn`&JB2zWS7ml=YWKPV_dDoiOvwuK^|PagmX)^ z>)ahy>4*#n>sTqmBir*SH>@j}Y2&qlX}HRoM6UX1)DsUvZ|evEi5!=y`b$>R9ZmmnfdJzMcuSUXT1RyMNR{c zm#wlHYun%nKZD}TH~LfpFWw{|#j>3AeRKU6br}-^-;SGPH}RST=Xd5+kC$Y=Su!}1 zkj2qKm?B{2UA{Kf@%3KU zO7hgo*j2|Pc8!&~lNZ2@ zH;8p|J`hAS$y=Lhu8i6eMo*F7rsjU9q8~ZUu53zOmvP z{wUrf?DbQ3sEd`^%ovl3=7tR=$we+8j>Pm(;7rzDHmRrzfLf2V?LFzrjFA)UBN0xy z!}iZML>7sG@L-`9&pIx3J!3iB9H;wwgU%M-b0MHly9c<<~SQ?Z6gCZr@P4nwiVyM^Ew%xE9{iGLr3e2be@^E7X{8RM{>-HLl?x;EY--O#B;39 z5pqG}nb!WgqCA8B@o-xBlp#hX0Ch1u<5n_zJkNCey0Vc?`Qj6wq=;mHE$IAHRUex& zC^ef($#Ls2UJDWSLV+__i;#}o1yjoWyvG_H*0*;G6pugSekYTArbA@Ov|}V|G;sqh_uenx+H|enO9gY~_KwLT9l{BA znKc!}wF<))=l=RvGG~Sh(B5m8R=bluYdLUgk!}LYGe?9=JZ>LqWr$sjI|<mQqVpyfHv2)XX_$e!RhNLz|+mAI;-oax}9ul=c66e@S_I85VopU zA(U^V6nC@y(GaMCTkGW-WO}ccxzv-n#@;rlQ;fIm?8IALW46g>)kBRy%4bQ&{6J94 zdRLcxu5wxaZ08L~A%E>?1xXZv5RK+!blRL_Zp<&nfQxi8*ouGx#*?Q1dvR2tj?}c! zY6i)yk5cx%Nv|3k%mn ztT@SELE^PG2)o^rF0VSu;T+rL6(~2vif_hCt_c?2e8P^7zo@8RHq}>B5cTj6n5lJm zJ!QnYsn4lH(e-Bw-0kGZ;gKfBjLKy6`}MLVpRkL_+~j=NE~oseUyObrLVF#9IHCG> z(Yl~7AL}`{5NUrBxktSBQV-?6j)|MPN4gp-b~Bh!R)}7q_kq(J)p2Wv5S(nKgH2-# zthnyHfY99l{gfK&51&k~?>!A*`)Y)V^m;$WJIa(Fm5)SoFo7rGj41ZUEUOysyTkzz?BaOdrRhM9K(eB1=0#<|dDD|=W zC2avISF8BQjaSe4Vx36NbruEP z=si7`mh-AoX$`>$PSsV&wi_GWp7=lHnqsEeEAa40y4EwO1LGP{n{&3uDn z5>O4ycVAaxw&A`vk)a!9aOH8(71KbZk-F(*#By-_gWZ}L2B6@>_iFBuVJx3z^6F+bAJ9_7D<~B;E?KJoh;)*jSW&Byu}@ z?ot1>PENsN_y<)OZKX;1fe5qepa$NPO;s@b^RVrWyJZo=AgfYUnn)PCU&#N4X%b>& zgVAK{#1|F#^M_E(u}_D#pNB6|5<_grTsif{dbUN0ZnV#)9hJvl#FE_Ob{{9~lYxiz z>*3p;x0PLl#}ovQ8qHnsA{fd{T9NtZvNsnt1aFI@2t2i8?q*VFaU(6|T;4@FSVmft zS%NG=vLMLO2?6dL24M2%X*^N{foJm2#AlIMwBr-;P|MdutGp=s^ zV(-#d=iU>G8=?6j);HxamiA@S>lvMi8A1~|vD2Y>+a3DqdE==+C|xq>ag98g-mpKg zc+V9mKLh?xwwqpXCawdQGM z18nfE>ehOdsDY(o&xKV{xJBpSFyE5vTxnRhu48$!~ z%lbzZtED1^8==Y=JSs%M~*s5f6KJ976bi-p_ADt@wYJJ>AU{Xyl)M@ICUIE7EiS*t$MLQ9hudr0}S#VffTUl^B?sx!R|!(+uUfBg(Hv+ zP3|4AL#@4K6jUah#p~-IF!CAe&#&CRC+wkj8yQ7;p%bZW%21oG^3kP$=L=^{2EwVjxnwL9r z;smoANSaiX3KznU9K&k3+{wN4d>_7Ay!_>^p1P(WRmWN*H*ZQ|IsI6pda*l7OT3H{ z-oo-R#xGcUiC`?PaA-q)e=rBv;zc->%=npW#j~DcQogdm+Z#jAY%H`L@q4UIwxfqh zLHB((+|M$8_d)bKt|B2N{nR^C!XaHom+Z)Pb6Thb^vK)1E8Z<%=bS9FlC@nlRdiMx z4cXR23X9x2UmRn6p07xDG+BcFZSbTiDzT{(U*$zoHo7FIJ`JFcm?j2-CvSQ$OLyK| z=N&sAuBWh{DYkpoK_<;iB)v%iV%KiksGvV0w_T^!QIcLn<1g^uI`D&SkFNeI$WNQHW0v)v6Gh|2-$9-peG(H7Wg}AHAe8J;Xr(Zoqr*o}X z+`OOJ)$@^w_(TCOi=`b&`4fKbP%X%56cR4j!r$cg(EdW?TO)|zO;7AXEA{h;Wnm=5 z)y}Y_JfI!7K!QB@&XhS48sRv?qB=2eR~$=yt9u~*fhb4fqyet@6vS#S*CGM<(zYq( z*{#cQrzZTQ6o+N)5|E`a$8(4hXxvNn*AjoNzehn3Z+Y7W=gEc)uld%jwPiVXXPw>) zIj!b3t##FRGQo4v(3m{I8v966y1xeETB_dJIcTL1BA}EgZQta5rOmkcvT{RTN2enR ztcEpMOld*lAvZs7aEBmj(0aZ#h~uU=)^7z(&aJdq5O1B@sC~aZ;SkHl_&jx?xzusR zHPvMzy?xz)DWw#MDiBKCRet zMM`OblBt|qltH^2J9E#e=9Kl0o5-^x)SEjCJdp+ei>>#JYBFlphAoJq0tzZ3%|b+^ zDV-=p1VuVXuK|%xK!Jz^2w(vbX`xB)y@V1-^D0e&oKh_nw*Sx@Pvw=^*jfZ+Dm4dOQ;?q!<(UJ--U)kP_~m1}E~*Vy>RJ!@tYB z#e*sI7Nd+#GJbbyc>j+z>-B?QoQ!Pm{_Mc5B?@kZQX10CvfTv$5#?^>iYqZY~Uv1Bt;#lU?qwps0tqIcj+DPB_T5 z=#Nh(;x~8J53;j31#|qMP(|c`TF&Y(arhB$U@uk{;s{<;-1#{YLjfnz^^i01_Y(I{ zyCI3U2+m2vuTM`dIDhdutqUG9(W#kIT5&)*$hTcog3O8>H#%M`e2^j_N-#qnk=_0H zv}6Oa!XM`#@;bMq8IILKWPZ(lxt?`b1h3T4aV0cYsR-oKWc#?^$7U~cbvM&=VZb`i zZMAK~>jQc7cM=deg`J^eJv%9ggzT05e78!rlr~vX@^FQ9j1ME=Ut`5S#?#oKYudC^-4kh`t=r(IK_t#AMwtQ`XGqeTz8ipVI>1UKT)z98d2v+i3U;eZWB%e(FBArB9 zf%e0tpWd-tE!dEpvm&*&?dz^9$`A|6edt;tK}aVsX~25V2dQ;ygPGybAAhk7J^BsDv} zF-pSKYblUI+9>^qy%v*&?DQ<3p4k?g1*sS#!sFwEg4$Nnj^CX$`EHO(hszWTf-W8) z_%vl;AFRc^Tl-OVV}YJ15NX+ZV|_P2%YBV}sN`Tj>%d1dKzU(3P_tC`jtgASdwe!E zE4u|FGU%GM_bLINc4z`TQ3+t#_`Za3jd|N^1lLhx*X{yqAI`^9lvzX)FT`QkX(>QJ zaGWGr>A*(RxdS3qnHT9M%*ZV)r%QQfC2Y_7lkbme!(L6t+eYv5ACx$k)M8olC$b2x zYK0uLVO-A%z4f?Cc;%Gp4s!4S27D+i-eZjdNm!lDD&>W1m6*iG>mClG5ZrBuo8fOLV`D{R1Qw| zVqL1Do&otk)8+5O)0^D0-3GO$;qk!^N8zIK2PWZZHsrb>JV%jDCe%+8VoEE$&YM-T z9fZQ$K*QsY!(Rq!0wNAY<#i=(#Rf$Tx=+n5SSV6|A3!UQ4I3kO^diGU7uPVqDi=Wyjv`01SmCa z?{CHltswT@P=JsnC}jyO=XR7z-%BT2W%QO&tK}Jc?T*t7r^2Ql^H>gC`j5dIUH7b5 ztEBB>IbM&y`*8BrrMfQTN)bW!Z<L`L9fyl#;&kMoPa2Jd^vA4DR7(>*0h}3OGcB)Rz8ws05Yv-nxN{ zB^IwF=`$0TwpOG4D85buErD?hvaU=63Z@z zEo6xXhJ3_JlAnPcxX#6W8lpjayluY9NmIN9z9Q$LU>BeuH$SxL^=|)6y3x>Nmz)G- zHMK@JL}Qd&DOse^wJHNc+bGNO%Q&!hk6*y3ksYqj#nx-R>WnAxmvw2pn!!=6@NoDA zQqZL19T1ep9{K9_pp7W`HDV3G?^_A_OjhB7_! zf*iUtJjBAOLb2#CjqctlQcn80=6&s_(-Kgi(;|FNDYUS}oh_*szWqU_?*@L24{sPW zAhON-vtjrxOT6>tB7m9{HMM?x0sG)%J)(UiX0TE(*3(wMxlvc3PuZaI7+c8K=Lv+@ z-F%aO!)}ILRo&Xn{p6H9+&n&&AGOX*Luq;DslEYnF7a)FDImw>89bQsu-+s9_2}#nmidATD68yj$sJrYKro zd!>y_HdFP(CU<_|kf~;{>Pp6-yr$hg(EB`{xG)MnBueNm4rVLXZPlZfp$A~=5U+)I ze*&gx!u5OnaR6^%w-Iu%fiBc$NbU)vkjZdGSjJ}E9KRu_n^Wv8Pw*N~ncWU8Fb9KU9($N?{=s#LFuD+MQQpy~5?2)TzaK8z{QvF$+Zo{luU4eVU!qdv7{_ zFDdjZ%Z=;HQGKW7y6U-FV%suZC3$~!e|pU7>T@`?6BRnb!Zq7@?Y#7gQj&M*WKX&7 z%zBb)Q4bBp{ql%`0{E7J*R_lZ(f<03QR;#QUH7}@E4?ARce<^@c-}w#AG~>Ab}MX; zN#g&W!W&@qY1n-o2iBcYakk$&8tSYkhc4@8Py9l8e`SDfiMFM)WxCYPzSOb#cKK=F z?bq5z9>^%B-uvaDFn4zuBr`8;ena&z?1*@LOpj?&U#fgJvz4_T|K+&0ZCQDr@p4A; zue_dbDTh<`#RCJm5~|}s3n}}XfxuVGErcVw{I4qPu_bv1#HWH2XRg*ZaQmYEJYhzDVR5*DUxSJ| zl%I|s%e+SJ_aaSUOuUzWJ`$L$yvymSK$RevS5je=A9mn8K`*CYflN>DQ|`nEbB^WH zaiGpfj*2Qhls@{jvN;7*AsQ>%=BZ=`vh%joCi_&%?6M`-lRl34zk_>>_B?F_M^Q8N z_xKa`B6x6r-zNHjC2F50VOTCb{d6s~h%Hu|PjsR1^o#sF-&$Cco0`=o^qX)S!bTh$ z=JZ!SG-l|os4Y*ysAk6~S{S{p16YeJ^aNR8H4rRlds5|Bi7w~lBc5IFphYK)vvkrC>DZ^;`DB1*i@zJlH z8K9`9R9}999Qa%b{}2^Wqarb+!Z^MiR7U=b**kAWnr;hn+f!VQwN^9T)-8;=4$HPa z7&1%R`x&*FQjwg6+Z5wiA;?mm(T-(qw=9^*$mI7S_ZomS+;*JQL4L3()I&2%gB8Nv z68qwZK7Cjpw$lV@6tbc!5Y6^C1&?`C!WvNBUeE`9Iozf<%asJ>54V)>jeb2GG*&!%$@ zLRfluypp^xOlPTvpJhpj4{&n%Cf(bI%TIo&`AT6UiV?UM+zL|7fF=9R&x_nNJFtG4 zp~os9t?e+Da*5LK{EHrd-wvP@sReVJ$I`#;=0-Z!zSdGIB;Bc0kXekU+j`Z1?5R9g z)XVaBmS4s^Y{>1<91ZoWRW^Uas})35Jo*eq_StG)+Ubh&bNZ0>F#GxXri=#u?8mT& z9;VQLE#}b;?iAdlNr=%AHSUi|Sv=**r{^7mZRupQZf`GIfrEqqbNKifZrFUe=vKSg#tu&ht-RBvg;-(c0Hy}P7>JKq|6*3f7k(}_HwOjAbG6rEn z-R|{5`i!hhJR0*f7P$DQ<)=>UZ#x>>sOzP_Q#!QQaPh-GExc<}y|g^9-xE7LspQ3M zjb;+!H`uErS9@mOyi32wx$Al23i@ zrtBBJV~xoHEJ`J-^LE0Fl&V8Yg|v2`($b^;+Z%()ILxK;a$7|jbX0oxRma1mkzM^u zUBOO$I)aZEf8COnBIRK`iOlO4v*hNPVuyVVmH1dcPb< zrhFbeV!XZK7{?-5&{OL%kAQ-z$aq(45i^j|rffzhAC*_Dcv`dXtSbTZsIp8--g66C za%F5J+w7Z4x6@GnA)Qf=yrki};TViuO}-?W!}JpMrzX+)vR$g`d!+@mt-Q~N&g&0f z!)k}8t&Ij_V%t8nWJ=kIe99{css=-5GDRv*Kc2G{m-$ZYHp*O2C`|_{5E@JtJ~i!A zNE&q5x8TV%X?d5ZVx1)QiV43;GCn4Ur$+k7A_R5paB{KZ5MAfQGZ zDQ|Dcd;vtA9>t_R`s@{Y^b(JURrrv8*5x5!3Qn-&!+lK9`h)M*&14WPy9c?+Oxc?J z@*P9&x9yW$uOIM!|Ea^EVAdNbm+K9e7Jl?7Lv5#YV#kX&jR@X-;~r-w?Pw+KN~VvA zX}S;s0{1Cx%W`Fz>s@M!n2c4xiYggY`@3M(GcSk`&#%T;(hg@_*9^V}7blefY5S=X z+}DpGxZ#l|4$@d==Tv&YAcX%QJgJMBNP-gvrkZ5BBTPaGUK< zt;L}y`mxS`BhP7G+YW}~hw@!Ha9-9lSM7$3)AqNP?+PX(>61~DgcvQzoT95$XM{rZ2ru#4Z5TLI!Ey<- z2!`T%cl44Hy!uL%7V=k|x@I;(`9TT4K3m6+NwR5m)q6v2`i)a%eIT*!>bM)Rn%twA zBIH014chK+@0IZQgY<10>Y#~(0O7XxAVOw%co#H-`cP~Z(iZBts+8NSh_0vHOIf{d zuE3H<;Fq)dn_Q2HauhwH;~M=+8=mY2*Q%N9&-At)}RfU!7J<=4Bq{3S`) z9!7?A|DP#XG6vg86)Oy?PH}AAREU3}a_1%;z~WHf+V0{_pVst;v@WO_!GI>a?(Us( z>qbmkQhJ`xIEJ2B)`sbv!!Oe(?3$=E&zJnpvsicq4C0%w2jy)iy3MEg644y*%wh4o zR*&O+Ucs8JeKr?RK3oDV<`sI^6l8XlCen?^BnO40lDM6dRgt_O^$*$WS*IKpkNQ4- z(XpBLwALYg))g5ev&<}@G-pPpg*;NZG37{G64%Vbxpv+2KXR2dIf}(Vd2O*=Wy!Z& zpLgCW{PpXjuU)-1pI@rUd!?2J(7yYW41HDz2~NZQDpd0x9;9YJu5V6J&3aBS7*eTI z4kKzqWUif-gbnroabtVzRf|cnacbRnZeDWvc01ku!Px%7hAv@|+?2LGqCE!w4i7P6 z%_DP!gJS^39vGMMPUb{x8X-#3W+7PYm4`{IRe`MvLO z+LF4~9bs56#@{GdX2!;;$3mIUc?%8XHwA#6dX*G&@Y&k@qh+@1ii833Nrt5_;I{%T%Y=lXxv06^Tv#~Om zwWSdhGnIJ87nW!F{Sa-k7cG>$F!D``ZbqU6VhegN8zt}R69>^_j%U&~kH@^Kiy2}K zCfu1cWbd2TezNph%SP8=%n!NXX9{aR{ZeF61-28IBfdQkOR9{?ksQ(w(Z+eC1gA{P z9L6r?b<0)j;=A%~OZwTYzvBOx)u-6aVCsyn^J+Z8p5B`KpdBE#p0$BIQo?1P9@6qb z3iMH8CT_?)A7WcgkzNU)4N#=-i9s*wwN{|3f7;^inH@hn@86ma5ndR;M2@nYogZb; zNO&CcB817~rC>#%V7`>%j6~Mit|M^#R8(@yQ_$BcBy1eI6f=ZvhK&>icM=ALvSw~L ztomoJ(R%nZ>K$-z;&BNG`EGu-B|leg@li*CcebRZg-l%cp^7iKr_%A@kSdj{>t)i| z$~oMfc=#NY_Qj{yolpITBObagMsFHUJ$Mmxx*%(|rdzU%hBJct*!J$2T>tb4???6; z_1%3N6&0w7{dYWkN<2I&E`Pq@xy0YozH%UoCrxykM2D^U%yMt+&l;l@H^sXVzTl)kd4D%fcwk2c-0RqakARW;9qAN&KPjF zE*13j$foxW_bB!9H*xT~M4u>BS~zDbclg-v_npU%Ifp(vcIEsz<2T!x2Q}~?&Uok1 zg^7NpRpitZE%4We)mjo`rFCo4dDR(h;_qsHeRt_;3df^_M+P26MYbT{-}*cq3B|dm z+K=-;3xP5c-5Ea>#3^V|^Uemn?2}=m5sP{-cDD2voaexeg940^q zV2lzTv?P!sj@wiXa)zqDX5D4c&M+8cSQDOU-K!wxehh^N>JBw2=uzbD1g4sz-r9;K zeie|su;j#@QT0A+a~NEJdJ3_d2#Z+d(foN(PP#&Un(M&EzMYdhS>dZu7cG(ShYR(@ z_sN_HWH$ouE~Ckn1E-HrvL#2Cp5}(F5>*<8tFDvTZ!A?^C$3d&%UBj_T_1e=e#{F$ zv3eAK#G+N#G~l_>ZLRYZniYW*XmOC)g$RTl9WKP+c@>}8)y{BHZ&V1Ud=F?rd7#d@ zXCcbfQm0%{NqwPH68`!TSJy2Xa@dcO`t4(ICIOnsD2dlQ5D0X%}nM5 zFJ6X}jY8eVOg$Q>4f7s?E_s@Fg_JgXW*UEv9xaMCNf}ElN!iTHV9l7|{ zX58;jyW98@pEp)66))~%JY$VllXAjc1-jjX^U|c@IMBYN_y^5I<<=gp5Puhy3|%0vSVDjM8U5%_b5-_K*-#U`MK<{U#V0?@JGjCb!i9+X}AGTdIx zMXK{0NewB5cY|?-IbSe4^&-8KiWOra@IZ=uAuL8Odhy8vU=GwqYqS`fxFv>e7!wCx zL*)|=Q{;8_7a!kbj3WdFyTq6Axdp)o3`Q6x_Fi$nEK3v8Ve#rq6!WLq>jJJ})%kPH z;EVgMew6FSskV7{u<#twF7~i1rkevz4C0!c6cO)SBM4Pabj4TBSkUE%Y<{j~Zu*Mn z+4{MO5>=??;kg?X^a=!B$Gvg-jFmEivYm-^zae^mA!qot%SEL1(8%uPI_i!t%aw<6 zF{V_`q55d_k~ygBt^Zd|8YPkV ztknK8Tu(ySnn)W9*|FD5LSk$?Euy?Kvd<&Th`YDg+0cD2Wz!4ZhozVGv9V{onS4W zAzf3|hCSL7r#>v6+Rj-VjnHy&AnWRpexqKnJPn$R*|Co}AB?dL_Z2UlIY<4%F9Eh4 zQaT@XzeM={gh9@RC9M?=e*0y3aq3)vC-a1M%$I;x*n4w;rAbueWgX&2LISbk4KCrE52XR9rEp|r1T~g`kB(_&tkbCU{ zvWYmJjH67)2t<_kmP_~4k25SUXlfPImqgYlwS$1qam7!A1_%g&nrJlK&~ISr&?CRF z)h0?MV^$vp$Bg??41@p20{DitLnV*gSkq>+YkGvveQjiuZN@3qWR;)9Jb$xs&v=SZ z5^4iJKh?ug-<|7D*SRU{+wgVSeU%6tbOj68Vz^v&v{!3ZgcKipYvC!;rsl04nSNhpJA-|(2wN-C z6d313q;2aEXSyGWP=E_(lzl|rG&UhN^`S&LV_D>_OUIVymG2~dQw212-}2ZlBDZR@c7>%fpvLG5@?k^GFfGPl#}&{?cb84!?oUk& zOa1crAMc>$a+OH^E^(Rw&-+S=rCe;EA%*rBv)BMbWr|Z9{+Yt!J_Wm#=*pqM9|K*g z8}-%JQYNDvuCaHOY@9GE(xVDwyU3oV8%bjywwG>w_OJE_A#W#+tXfFTly{J-!^!;0 zFZ6sG?df3yk4`qNnJTlmjg>e=09P9W80NHZV(86wKFSYEfy1J0kkot(qAL+u2ty;K z-saMqMp0qr>|MGvosAOp$whhPeO;`^@8qG~Kl3lNgDI2Z|M6`s5)yPZPUYe4+{$R< zg#5CNkG0PITL;4q>Fdyy)@|DLg-RLSmg+t7(4&27sOw0E@Xj=6@S=i8Q%KP%e=XU@1IFG2aOZkP|99E( z)cn>n?{#^em+HC=K4=24Ef5{%PV)?uTCg^S6rzN?^hH=P)sMZ-4zOFan1Wpcfnj{R z@g166?Bx_M0VoH}{t{O)a{_RYCbe!9jj%1=f;R3IKL*M6#1V6zEvOf)iqOZwIF3O% z*`-&D8q>-!zxZ<$_*XD)-o%e|!(Z6d zY+^)lB#)6qaVcOC$+%~L3a)rS8Iua$)rN>?>(z8{K5Tz*G$Z{xBHfpcsSGZeJ)2P# z!9!m>-J0PPJ)1yegqxSx_+ABp@K_IWh;gz3<% z8gBdFylv3lA4DyKQYl8`=6-Wz^^$tErZ!Q{YR~pB=oVQ3#er^8^lStcZmvCeKro@v zFG*;9Bt=jcHUBOy0A@Fi)3&FvRB63CNZ&^L-q(F4i+;_-#FuQVAmxV2$>UP^ zeeWri5p8_f#L)A^Ej&25FpYrGH_e>}ricn=dHf_%i-W6$<$E>Ei*nBoV=4J6Ut&(m z2UosPq$a`pmA7}MhO~Z@I@WjmIq8C=M=S&C@8<_n@&ACkmx@N&)BxiiY0Nis@$0pi zO_Hv!?R|<@-2>R;+vmzAK~c2ivRdTA4PLct>7t<)YWuEf=Mb)9+YKXu5BI-m5xW&n zt>v*DLkk2UBiHa;tcm6YFZew_*x&-Ot9B0*ra;P>^(w0xff5~q z9ROUZw}F{abTNy5k$rF7@!7e{b*Qd7T}|(mF(wwtyY70oNaXXI_m-c?FKiDmx#WAQ zeEK#9nkWNPj~XHn&hQ^;w9O?(ZfTOf>{{Wd z2fTY4W#R>#nTL(V`ESf@j0XjxXwQ=#0ygZ4r@sjPt-;eoy(abN?$AfTEdkJwty$~H z_YB`Pp&xUB7-0R{N16Z=E~Y+fUmtm~4KzGYIxy?88aqNS9>C{!0B+pty3WU6oF7N- zysyt2%_KuHZ)W52mo0hh&JDR0j~Hm^wml?TlHL(~;x=H?<8WP+uJWaF5N5 zYc&a=uJ;20D^N=J{wBII9WJ!h3ZmSv4Fl`tneSPl)d0*zX|TjNCp{s8!_2%+HQh6I zuPzNPj)CvT3Kz||(WBW~Z@JGV8Qe!%53=ylgNQR?)r;@0MgLtuC%h@jJvFcOaCsbC z+YD0pu;LJ-Jm1eQOmAIyJ;$iE?KE9;%$~9V9&ua<2FY7sxzCk};L!CJRQPf#hFXkA z{Q$05Lw1DaCk~ZPeY_H(+nN*R;{*KPbxHH&TDNQ~gJ97~*u;$iuvA;Y%&<)#?$Csl{6@?j?BAok1peId?XFw?jG9eShVWI#xy_ z!(zNn~!`7&6%A`yO!Nd^JX7wjho5vw#nI z;_+TOz^iwKGhYVXY-Q3A%F}!l0WOk_(wz|*reZz4zd!$gU<^})JD6Uq!PDV+_hTM8 zEdi);Q?WguM+iY*^+JK84eqMA1BXjLQ1kc_sTWeV9S9mHU*`ydFe4O|DLTv~5%;Qh zT3TZTgoO@%Lp({l8y_gQKjIX56PL>^A^2Wk_w_D?c^v#z%&g4XzIDfG^luNja45 zOgFqAdDh-0a`)kOd(+RV!FM+kN1rK^y)wQ$+^>3(fHWQ()?L=R8n}X8*xz0KZuqvI zPEOHKUQ_2T($)~$|H6dxP(_7AsAx$wsDs?%UI%w`rHkom5T1Y`?k4Y`7DNPZL;a3c zbd{i|VyWN+XZUuzoQZ%)hjszI3Wpx3?o-~j4!+jNm633-ctb?%K6ZCLjb3-DVsA8z zqNuXOF7YKsD)BRksFIrM8QZwG(oAP>Bu^V%n-NtqCU`e3$!|U?-b%#$=imde zI>gyQ2=!bJDd&cTAy0OqW7(YnH*EZ?9o5>+ym2)Ybkj~ocy|d1Rhr0-!r~7>x@vyr z6wgMtfM&4^VlB>0B4)1ww9rtDXDK}cJ^mg*({N^Qh?ix*3^gxnU47hdSa+@rrd}Kg z5K;l66Yl4ksMtUJ0@KI?Aon=H%slD8;|4fG393t;y*E03|ijN8Z^{D()6jQ880_$f@uLZmhtp0SKx3^@Sw0DWoMwt z@@|7?J8_LN(D|x)6o>F0LoBcXT*DSvZ4La4ziWK9vw3&TL|97$( z!1*jqM?s`^nn-&UHdW-4RZeo$LeMY}?A= z+Qdo)7ey*C4Qel_uyUKS9a70HBJo8^%4ylilWf^1vHU-9?0H0`gs7_aUO-}OcEMIQ zO9d3j%j5DQ^4Tn(2b58~m=c>16!jKO!E(4}^qLh2QvdjociAc#fcpEBA*!YdJK|?v!&uBnS8F^qv)D%<>fGMx}9Rl}e0^i*m;} zS_JEMOCv`ivorDlYc7LWyyyZYe}Sx{6NSrJVxR{!(gO&3vydwP8H@~Mh)dlpfDee+ zI{#xoVJgtZXE}E$<+UeK$KbI!)D2*ym-i)N4e^Q10Ro*@2Tq_hmy_^60!u3DWc~Ba z#}@;%BnQr zbKx~dtEGn33xJag)zgxa5Dn893e2(X53_(H<|C-D!+Vxe?8ZA&%B{}XFQ+D#JWY6St?vI zD$w{PzqiqbH3-|=6e_8P`PF00Xz;T$ce!G8LHMkxn8|#=edE>JK--a+lVSHqvfktv zz(v$6{Ej`3wPl5D$JNCYdbX}rHyK_3YQdC0sGPfNGlV_$a-daZJC#C`s@f$yh`gZ0 z?)n{=vjo|E+7@nts{$713rS$KJ6Q_t&5^OHEY^Fp3L`}WRx4=@9#ZU?5~IBJBiBY! z%C_alHp+i@F8L0;*;c!{98Y$rj()f>gn-1((cJk8ZRNpm@@)OMcXw{NHS?~S)B|Gm z7f$zpH4O?}wi{+HGFK;M)DNBZd|aVVeqilM4zg%vCs~%ygd;2(4%eSmZQ99$^K~Qk z#l5b5<^tkBam}^eUh_h54}y=0nfQp0t38ie#AH$BJzAr=@H_PqE!un>Tm?cB=#1f+ zmNG|rfm18TS^Q=buL|H!EMfpnsbP|{R_K&FVvSI~=Tccd_7Tqa6?g(Pc!(BHmAA7A z=CwfQVeCbVtcE4w>E)RxNID9>dl7||tqK#qLr!dOxV|6rXICh<*K^aonts>6rt4Xk zJ=hP8a@xu;TOTX-OzPoOGvcR{6$ik8G1he zAq}+8%6bzg8*vNFpYhwpX50%;-@g@q>D!fa?8mmprWiy{YOHX?Ey`N^K4rigViB_g zh%gyK4_&?`h$6?zNuwfShvAI>XdBFS1S#dXhOZEgETHH;c2cZx;Wsb@^wiU)aRoG>oM zj{)7Q6HVasho23gEfwvlUy_KlsCAk%1A|6f7`|0!Lgiw{8jC1F%Om?7|^o}w4~ANsvVH_99KMeY_mXm3~Vac zqC9WFziyKAX*J_u19C9KeIoH(#a8*`AtB_Bz2Ed(o+hAs-PrcxIXNAgb;PHpZlm_I zqFt7hseH}&aX{mn9j5QIiKH7C4SE%80H}EPL`>42rfxeJ*R)59**$yh;kj{HpC9(& za{&{AU|*MBg&y;U>2d;^@M08LP2`VD=Kz^pqxMG_{B;c1i4pC_=wJ-{B^jp=kPE|W zyNtcnRq%^Hi(V`=pqT8BR5}Rj+7AR7O{;pq)SoZUqXC9YoN<26cNY3B38pht^Fg04 zTB>&*T|3zch|7l;Wur3nS~pD$P?gmKMU$lf zc?H-&qJQg`j}pDPeVF#9Gh0jjmfx8<)Vap}!t7*{~V(Z#;`@M=50#lWf? zb~nwtb?+R#cKol6E;f@r%{!tP2)^^W-86R^am{DqJMegD>a2hi&}=nryNeyqcTikN z(oU9yA%H6-sooK9bZhkU%)PFa|6V{`^FRev7A47EJo#icOa%s}4ja+yWs>8Z}nu1V>JUi{WFN~)I)@Im7VGk^Lpj=3JvDDV&r8pHj8vg(BL2jTuwjJf1Ve= z(y?s=fam65AXOj7nYZc!NumrW9M~3!Si}frK`8Wm_&&4A^-tQl3<59{v$6y%!T9Z52C@ z6Z05E%9*uOq^_O+1o-uu$$qQV@SiaqB-&?^#dfu)1J&gy5!M4nUO?j?Sg8_$%pbjC z><~<`NxpE~n)240)#uua7prz3(|#1Hx!)MBv}va07>u;{w{A~ThWS3vB|}#N6ow2m zsTWVyna;X3N5If)?N#S4Wi}2!YeM5_r9;n}>Jr8~E66!o?h#&00+YI6-yWvKE)n=m zw5ZWK_f>FQv z#m8Irub8bwGZ2@p0mR35_hpq(ZaqpB+y9?4;a~IbOpw`cO-8yTOWKS_h-5B$cLlBh z4f{?6$|>K(AuJg{4dLUz+Xp~d^wLx}h#O2*v~=%6>@Nmdt%_94xZ9SmvSK{jEn?7) zcGwI65xql$%yH)EJPVKZNFJ#dKy^USVQZkp;{?zh(E7VtWAwy`mHkRs?yFT(guMU+ z97K`NcCC(j_g*2`bE30G5E41|Zt*d24wzX|%=97`jB%F&>jBg;*pDXmr^-m;SQXie z2^0R;bqwrdsde1C1M4mB02X87gk1!05COsKtc;@~ZM#3On_I5^jM2>T?P`hv{09?i zpyk9ONmIEJS1B+c(ep1VG|7+NmEso~aJmGOzNwaz1TD8Xh>LqV6Yu-TtjoeL<=<9H z4|48WuVA_-sPhT;3W9HH-7`8e(so?v543@JP|n;dg`ZEkAs=3ZbI2>rGuso61#M5z zy-ltzvhsP@)t{&!U7_r};=N@Tvi*v`6iNwX2Z-J8=X1K+zu&Veon`t;mh0Xj5d zg;sWeyU3@6K!_e-k9sWsvq!Sl)VvrKvQuCBQan6&Bbic1;#4Y~&9sb!^YtMDv6h}i zzE}zk$0BH$XiTbmKYk*b%uamd= z#xy6~{9Ac`o7~YOalCI?v|$E z5d*mg!2Mt8r61J}a!{`UKOo%#n6tkU47WSafJWYWzh4cw529i3dn)qY&{s?PNI*QI z%=q)$B$zDkOQ3`GN(T?r*P-*nb8yJP=8(QzWW;;a*~Cu>V32ns5Q!B*$HH^fqsL>r zR&paH*AIRxUkzJL3}EhEi^YHT4wKQxZnzvUL9AICDvDXnuxMpjXp-4(&9BcV!Txz$ z`=FZ9|3Gr>s?9tqIJHB*eEEYXe<}cK$AFq5`t8_xwS!zoVz~ul8|5kf-@W{d`SFk? zVm{`-SWME9BJTv$vz&q{K9c<}IylFq7|I+YUsx3skcn0H9BC zik9aHZvSxLQ~=QDe)-~twj@@a5X4Ys7h&Y7o4vhx=+F&uO?6dcA^MF2e>wf@dg+VE zgT1Lev}gTKrxgHONIZI$A?^5{PtMI)MRUy$zn4)a-)62_D+qCOY0bh!qx}sziya8j za6sLP>Pt1M!4fNgEM)hwRS|#t0oL51e&&HNS7sUv6xjt3n^XjG^e>?EEC5qtJ)mC8 z4$SV!N51%P%wzXf@l(nhJu)gKyFjB(s+s^#U`fv@%Fdc5V3RwWgGd-NCjHyq$tu5)4e zQMTaTVQsxgT3L7-(0+HkB>6h1Eg0A-sgK7%D{k*b@xT>-9O1JI%D#=KyJZLHqO%fk z*LW4HAplRHjMllgHB1F===yC>_%#Dqu~$USvt_OGSe6+gj=D3jTg%ilI5(aQQ0kvp zVL}ZS>d!#HQlHrnZ#P)1nRI zF96#_Gynf@Q#%b+04SV3FM;Y6fK=~RAuvoCw!Gmsg%(cjEM*7qvUDF}zv^_kU_}ss zg@2nhIDHT6zGGo~M-(#Ws{*W|1nG+C8T@{e9Z+G$KZO8jQWs|5Xv<-ay>^9rzON?A zg6Xh1yFZa{Qz$M(3VP11HQMbGK%n!Dfc+sBbpQ zNyv+H}=y7U(yNn;mZN`M@i) z|E%n|;LB`%>(jTX8G!l(ZkatB+7d!VUWQ!B6R|+!&uXW!xid`z8gtBkg0)6)3c=Kd zfRr~2fG5Anx%^Zhn;e`MGu`1S*vhDq)34JQ)3#Mhgex7;(nT_LSuoySorm z^+I<0iXHqr-tHU~q#S_mv|Paa;rkbPtO~7m(G@|)ugifoAmP_7zt?yH4j8-kmegr%#P%d6PLep z92<`O7=~{7Q`7yvuQg1DzB3bJqaC)111?GjN~Hsc%~Z7}8}WF^;M=7>`m0ltH^4!Y z>W4ztkGN%ZCrOAM6#OXC3=<$`S#oM-Qg?3K+FlJgbmWBanX1Vc3xfWwvcU6)SYv@Vg;*94A>md znCMB7vB4whC$HmO2Cilu`hgEVV3Mny*?9f^XTAko=M(dbbJZW9PQ-s*w_#D?{sCD3 zI556~T&R=sTYk(#rijGkA89Hy8>cZ2 z^wJ>5M*+F)8SYrym(#}-0=7J|4k-d6Pik2cP|AeW%K-nE=i|29T&kYWbnYf=IpqU8 z;vEVI&YzU`{?vMYd3V+mpoZPY75`MVk5kb272p!bRR!Ngg9gBaW+B=%b+t?{>|9di zaO^CQsc;#HodrnHaQx~u=C3Hf@b6bZl|}BV{hz1d-_r9PDIk>gJfxR^m{aO%0Z4QMu4V5q`)#`8|zN+8fmq^?)lx2L1H^p)n$0WHEw zSfHEQ=LrO*Vg)m#=fkPI8N2%b+Pm^_sMoijwnNS&R794_93fFUh%Bd+ED=K4rbQ*o z6k;^SRw7A4RAkS-R*YpVA(d=lkag^8WC>#*-uv78UiF?coU5+u{p-Dc{&3B3eV6CC zx6l2#pXa%co=`~C0r}>+b1)<9+Re44o8q83Iu;JrcGOm)Uw0DwdVxET0Aw24TgVOi zFob&vWyAp%5TB@`e3Qk`j$%^fv0uN6IJ@#Hk9^j|gjaBud0kRM2vXz`NlJjEy6Cr( z1EMbnBb@W&&^~RgOU*U0Y0|88K1P0x?2{BQ) zW4U|0CS0C`l*!F6(6BQL6Sbt-iNX0D6$tMTy6z=^@xZo0HV_BX+Na3-Df({uW;Vz%B+Lg>_ z7jP@N76=3`bl@n}OL<;G0pvNGE%SCjE@>hMF23K>EUc#xA7&DakHxZnA&f*2P%R0g|M1SnC z;F$aF@x|{G`j3Hn++%*A@;G4K(LcY<$qEko9~Z&byDXkDIy+m6Fw1?yNXb%l$)HdhPL@Rd4KKLROgFOK1RPN?tBnHQV_?={%SeF z{KwoF4fX%r+_H{%0BXqMt8o|#zbNkCl%=US+s8Q#1b2>^WLao-eE(V5FdNaVke2cO zN>e>Oy|7S;OY0fMWx=}Iig^3US$;-i@*oOko-f}p{b~(cE*FMSR~;_j>@w5sjeS`y zZQqgaVDCJf{iw}#ez>jFIvg#;5X{Rx2oxG?J%KBSo*UtR-m{E5fJy8PBR z;yLX-X7H>_zQxe+rs%}v=Z3c^Zpl|^c>~mjTA5C#*sFuF$NcPgZL)M;7Wbo;F+`6y zEUAjV4Y_iX%S0u?pw$yxdvRih`jnKW5Yr|HKKWFg>t+=EkT?!-*nD3JtrCWgS1pCX-?O>rlXyQPi`w+(I%Fb?S}vip;Fi`N4n2vD3P;{b?Xdqp}BamQTTV{ z7=hVgeJeAJZ@N>#GR`b(7x<92y3$Jq4DMLa)FUYUNq6T`=G+eNQMsidY%i_Z(trh} zkz#%Toru~l`vC=sLZnlygjm2B^y3*<=t(P_YQ=ucJXd`lns(B@a0%2*%2Q_VJ43sV zj+`P|nt>E=axa|e<|a=|bwp}&(8z9JD1OHoz*Yv4LJlx7Bv=bCwX{uJk@K`&8GHJb z7H55XYr-5S){mx0`7)p7*VHT>!}=LvZyTgVj2<3O91g~dK9#kNX#;{MR$yW~qhc#Z zZP{?$;NnOnGQZ1=T`n6 z7?3ZD^sE~t-_hax@5nHjt7a`%uE@=$pMNBJH)S+Y zW}F)G}))p+Bg#?qX2rCANNLwvC#@m4|U z_?<&CW@%kvb`+P`f~qt-A=0>>oL)wZ`FNqA(84I2IBQ9* zz%6I!qRQHyHVzg0F${C{V4DYYHL6wIH|)M<5SC~md>*ZkM^t3rX4(eFiE5d_pw7u6{73oE&w@-~=NP?tvtHSR zD=gV};JI{~#g6BaQ>>q~9g+hyo7nrg@t>#KLIF{$>=6{)lxS{T^gV}``X$^xJO9C& z0X;DV<`P0X5oT{tY;GElDLZJDO9oLLT6ltZBYHsx)Dsk=_Y1LMvl|UYAVYRkKf56= zbQVSNQ?hjZlcIi8x7M;QcP?E~soRtDsaGqiA}IOgA^kzPr%dFgTqKW2HkM#1F4N+V ztKe>f&QeewI=ARjJrSVF{{*)u`%u<)-!14<$oAtYmKcQn;*~tB{5<30fI{oBAuHa- z)I*SI(J&2?UH$5D(qo@u{pW0sByl6Xq&nDDw+AfcqHPofJpUV&|83fpe(DsD_l+#gOaOggZ2y8IdWT^@F+^ji5=3NN{tRo3zCM=S0vxOI`OB+0vw~Kzj&I z?V4*~@;ubghd37xu))4FQG*v>_=Ja)p0!ak_ZkJQwKaR^dK6 zNXQm)?`zN-BYzZqa$ionorVn1chjY;60#`q)EhcUj;_58s0`><_9T; zJl5ocanDBiFI|%on;Xw4tV_NRe%=A8M@+#AF z^7LRzMo}wUTsuUMz{*{kQgbuV!ADJ0lr)ev>CM|JC; zs2?fQCYfu6BtP=A&x;yHVD zFGnzV&ZtJ_L}B@0Cv9B0XkzE;;03P5_>Z`|g$XK3=vCCK!J85~M@~7mT;!s(b@Aca zyUD$e0$Go=KvnuF{srq%7E@pq{CSmJ34d{PKNQhh*i=OQs^x==dyjMynD>7u`^-F_ zKGi0%q>ru*7Nx4`&`y1(6uT8oE1`r_RTP3@aZ?$a_aUpMJ&|LH5llff8El*9wJS7U z=pKHEzerX>PYnYXp*7I}D_%Zg2vfQ&fj}>wU6i6lB=WE8Psn$L{<9M-o>WuaLJ3ii zlSd~K@|B>Wm13S=Vd`xT41YI$Y*u)A07i#hHl`_^M$(I4Lt*)6X_ zkG@&+k};#gV9Pt_(0q~5eRq~$t8B1N6&0m>>Knb=2WgJC*bihG zPv;bEv^gWvtq@2yOtH{J7~82npFqAwsV-#W`Fa#jA3gZmjhcglGoG606XlgiC!L!) z4^L)SOSj3Qz9#eY>!;^aMRC1;G6S5<+5Cy#ZzKgq3nsJ_y+WJ`H~N&S6mOHuE_)ED&XK9p10CCtcDU9ekh{A$Xe49aG^1T34*Kp>BRonhG> z%hUY`yASWiBB+fzYLAmM@VdTRUHH|L z`lD+hY7CL3jP>+_Fqax9Te*6OQ*SdP8l*!dlA7wJkKN<0OZ5|-oj_nl$Gd=CF7?AO zrgp-5yHTMlYc@o6N;g_1!^#+Zop0)6{`ySo6r5GhRVEK!SSLbOfI1q9dEHa z$9b}SGq@b8Hy_+_N z_)<-^6gJ)GlP3xuOX_DYdPX}30As4M7b&U<>^$S+e?}t9&s;+?)wq9AGpNZNZE#>}GtX2|OzE8pv1oFmT~SO!??FcW z`_Q!3ATk(hW&dX3hvt3%v~APL@k*JZuc*k(DOjqLy95nz-ZkR^AmCI5T(U;%hLG?i z5O`X^E1D3T8f(@HC9X3c#2y8<#-JxZ#dnTxJlOZ(ld7WawfbjjS`}A!uqSk}25_ls zwjqH8v~hrwH&{8roF8T19;(hWkrKfjyeh7*NVjYatmvCk^f{@p%PXcox^ri*!tNqy z__rvQ+_o?~;j%iKV+q+s5Xrq|(TpWj?Ji#N)II1}*(@xWoKpdS`YnoAdXL&;gIUwo zV{^V}KcbFJ8d)z2k0t)=L6IoabLhL!&;E z2(xz6u=TZXX`n${$pLuglZ)9A){vel`dl8Udg6M#?~`}$8$Wxh&Q(aj>T)y`DX^W1 zQ0s$~+eZthBN~P*Mt#srV;F(m`!s!yHco~bh!`Z*Haw;PMLTEJ#@ z{Z-^$cD^B_nXn~HdUOY)E3^5_J%YKdRM(s_XBBK%t-ips@4P2MWz!#o_rCR?_HhG) zDBMsJp&+m1R%e!bz%!kLIRm_Oq)BWyrZKD=9PK!znG^klk1$q(o%~c+)SMrE#9Iy( zFrQOQuNDqV$|;1y7QkcF%M@)%& z!IL)&?^O$x&0J`WuDh?$wkAdE20xnxXd;hR6F1E#59bX=Xen@b#ax`~y2UnEi)6u^ z>z4rP&UWrZ4&xY2?|u+Gnc%5|=6J87raMh>D_%0Y6Fg~)pk(3j>w%=Mo2&&H_!s1B z6=(Kl`CWfx#$&U~C>F}UFQd3l**KG>4=iNfZGC{(^m}C4=I*r+0r~2ZeRhRYU>Z>s zI-_bj@N#8{WiX(aRUT7%5$V=qP;@R2Tf|EcaeZH1W+6^NgJ zTz*_lwr+<{@NOydcbjBvdsL?1DlOItr#TF_HVW5Pg~`;iC8e1224g$&^MbS$TTB58 zt!t>4tWYoAChwR^>h|ItkEFpe9zNUG?A2bBYPaZjoV5FiLZwK|6VVY@Yk@8H;nW3y zf%8i912V;DY`Oa_2x!Wc)BU>W-z%QJSr~!icU+CAE!sz9hA_K%Mx2)D)6+|(&iF6& zalZ2-xMC=PjmRaK@#~W-p*Y3=+50AJ`r=0TjhX0ZT0{?26(}yynBErJa3*&qCnc&9 zkz-1(DAaGtd}Lzh6X|-Q#Ar;FPacO$2`R?7cy8p=aD7D735@T$g(v!bs3~<6KBZ!# zH@h`Yjih()Fh# zcNUW;3}>*5w{~hDwnjalzb!#1Nt(`ocGTw{)pQi8Y91`GsDErg(Z1lSFS89Pt+18t zd6}shLsO(C#xCs7g^)&Xn;5K}rWlr9KAwNq6;n&OG03MRl#Ty=&!g?zmPk``a(lE< zaePg<)?+qSM>4G|h*3llu0{P;o2cw6!)}xD4a>Z{%W9r|m&)&M!8RuA!^#G0mr8G* zW$YZi?f~IAKFsP=omyP)eR(b6kah^|c{~1mA=snl8Q||^^cz#_F}?>+)-nbncjYkl zJAbe{T1eTK-}@ebeuVsC<){D_&mLsSiE;k*Z!6H$1g(D?G7h;EKn#eBJ8H^k8%%4B zf?kEDCG97#V;q!nlmqJmoP&A|`VBim$x?eL-C_-NA|o1kd1ICJpFHzeWmC2)_`#1? z`0>es{j6+nzN8d8W1wKvkss;1k@u>yZ8`V*mBoMj?Poq&Br1D*X@&cID~&EQH$PtN zZ=aOQpsr!ft-!gA;{MBHqn-mlF5MKF^lOj%APK1!@3C6X<>avP<1A8vvG$`~rvB~K z8Iz2E4Eum4aQ|L)Qs;h60z6{iEHBWX$StFQ|ME-}C&0{elbYPisL{#-_jt46fy&3z zZt=_HO#c|p|Ni-p{T$*%@-SHX|kQ7x79 I!)I^)7ln=EW&i*H literal 0 HcmV?d00001 diff --git a/native-engine/blaze-jni-bridge/src/conf.rs b/native-engine/blaze-jni-bridge/src/conf.rs index dd476ed64..9eccc0e5d 100644 --- a/native-engine/blaze-jni-bridge/src/conf.rs +++ b/native-engine/blaze-jni-bridge/src/conf.rs @@ -41,6 +41,8 @@ define_conf!(BooleanConf, IGNORE_CORRUPTED_FILES); define_conf!(BooleanConf, PARTIAL_AGG_SKIPPING_ENABLE); define_conf!(DoubleConf, PARTIAL_AGG_SKIPPING_RATIO); define_conf!(IntConf, PARTIAL_AGG_SKIPPING_MIN_ROWS); +define_conf!(BooleanConf, PARQUET_ENABLE_PAGE_FILTERING); +define_conf!(BooleanConf, PARQUET_ENABLE_BLOOM_FILTER); pub trait BooleanConf { fn key(&self) -> &'static str; diff --git a/native-engine/blaze-serde/proto/blaze.proto b/native-engine/blaze-serde/proto/blaze.proto index 9818a136d..b424f1fcf 100644 --- a/native-engine/blaze-serde/proto/blaze.proto +++ b/native-engine/blaze-serde/proto/blaze.proto @@ -35,19 +35,19 @@ message PhysicalPlanNode { FilterExecNode filter = 8; UnionExecNode union = 9; SortMergeJoinExecNode sort_merge_join = 10; - BroadcastJoinExecNode broadcast_join = 11; - RenameColumnsExecNode rename_columns = 12; - EmptyPartitionsExecNode empty_partitions = 13; - AggExecNode agg = 14; - LimitExecNode limit = 15; - FFIReaderExecNode ffi_reader = 16; - CoalesceBatchesExecNode coalesce_batches = 17; - ExpandExecNode expand = 18; - RssShuffleWriterExecNode rss_shuffle_writer= 19; - WindowExecNode window = 20; - GenerateExecNode generate = 21; - ParquetSinkExecNode parquet_sink = 22; - BroadcastNestedLoopJoinExecNode broadcast_nested_loop_join = 23; + BroadcastJoinBuildHashMapExecNode broadcast_join_build_hash_map = 11; + BroadcastJoinExecNode broadcast_join = 12; + RenameColumnsExecNode rename_columns = 13; + EmptyPartitionsExecNode empty_partitions = 14; + AggExecNode agg = 15; + LimitExecNode limit = 16; + FFIReaderExecNode ffi_reader = 17; + CoalesceBatchesExecNode coalesce_batches = 18; + ExpandExecNode expand = 19; + RssShuffleWriterExecNode rss_shuffle_writer= 20; + WindowExecNode window = 21; + GenerateExecNode generate = 22; + ParquetSinkExecNode parquet_sink = 23; } } @@ -398,27 +398,28 @@ enum PartitionMode { } message SortMergeJoinExecNode { - PhysicalPlanNode left = 1; - PhysicalPlanNode right = 2; - repeated JoinOn on = 3; - repeated SortOptions sort_options = 4; - JoinType join_type = 5; - JoinFilter join_filter = 6; + Schema schema = 1; + PhysicalPlanNode left = 2; + PhysicalPlanNode right = 3; + repeated JoinOn on = 4; + repeated SortOptions sort_options = 5; + JoinType join_type = 6; + JoinFilter join_filter = 7; } -message BroadcastJoinExecNode { - PhysicalPlanNode left = 1; - PhysicalPlanNode right = 2; - repeated JoinOn on = 3; - JoinType join_type = 4; - JoinFilter join_filter = 5; +message BroadcastJoinBuildHashMapExecNode { + PhysicalPlanNode input = 1; + repeated PhysicalExprNode keys =2; } -message BroadcastNestedLoopJoinExecNode { - PhysicalPlanNode left = 1; - PhysicalPlanNode right = 2; - JoinType join_type = 3; - JoinFilter join_filter = 4; +message BroadcastJoinExecNode { + Schema schema = 1; + PhysicalPlanNode left = 2; + PhysicalPlanNode right = 3; + repeated JoinOn on = 4; + JoinType join_type = 5; + JoinSide broadcast_side = 6; + string cached_build_hash_map_id = 7; } message RenameColumnsExecNode { @@ -438,6 +439,7 @@ enum JoinType { FULL = 3; SEMI = 4; ANTI = 5; + EXISTENCE = 6; } message SortOptions { @@ -456,8 +458,8 @@ message BoundReference { } message JoinOn { - PhysicalColumn left = 1; - PhysicalColumn right = 2; + PhysicalExprNode left = 1; + PhysicalExprNode right = 2; } message ProjectionExecNode { diff --git a/native-engine/blaze-serde/src/from_proto.rs b/native-engine/blaze-serde/src/from_proto.rs index 1f4e82425..cc89de0a5 100644 --- a/native-engine/blaze-serde/src/from_proto.rs +++ b/native-engine/blaze-serde/src/from_proto.rs @@ -45,7 +45,6 @@ use datafusion::{ BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, Literal, NegativeExpr, NotExpr, PhysicalSortExpr, }, - joins::utils::{ColumnIndex, JoinFilter}, union::UnionExec, ColumnStatistics, ExecutionPlan, Partitioning, PhysicalExpr, Statistics, }, @@ -61,8 +60,8 @@ use datafusion_ext_exprs::{ use datafusion_ext_plans::{ agg::{create_agg, AggExecMode, AggExpr, AggFunction, AggMode, GroupingExpr}, agg_exec::AggExec, + broadcast_join_build_hash_map_exec::BroadcastJoinBuildHashMapExec, broadcast_join_exec::BroadcastJoinExec, - broadcast_nested_loop_join_exec::BroadcastNestedLoopJoinExec, debug_exec::DebugExec, empty_partitions_exec::EmptyPartitionsExec, expand_exec::ExpandExec, @@ -89,7 +88,7 @@ use object_store::{path::Path, ObjectMeta}; use crate::{ convert_box_required, convert_required, error::PlanSerDeError, - from_proto_binary_op, into_required, proto_error, protobuf, + from_proto_binary_op, proto_error, protobuf, protobuf::{ physical_expr_node::ExprType, physical_plan_node::PhysicalPlanType, GenerateFunction, }, @@ -182,19 +181,20 @@ impl TryInto> for &protobuf::PhysicalPlanNode { ))) } PhysicalPlanType::SortMergeJoin(sort_merge_join) => { + let schema = Arc::new(convert_required!(sort_merge_join.schema)?); let left: Arc = convert_box_required!(sort_merge_join.left)?; let right: Arc = convert_box_required!(sort_merge_join.right)?; let on: Vec<(Arc, Arc)> = sort_merge_join .on .iter() .map(|col| { - let left_col: Column = into_required!(col.left)?; - let left_col_binded: Arc = - Arc::new(Column::new_with_schema(left_col.name(), &left.schema())?); - let right_col: Column = into_required!(col.right)?; - let right_col_binded: Arc = - Arc::new(Column::new_with_schema(right_col.name(), &right.schema())?); - Ok((left_col_binded, right_col_binded)) + let left_key = + try_parse_physical_expr(&col.left.as_ref().unwrap(), &left.schema())?; + let left_key_binded = bind(left_key, &left.schema())?; + let right_key = + try_parse_physical_expr(&col.right.as_ref().unwrap(), &right.schema())?; + let right_key_binded = bind(right_key, &right.schema())?; + Ok((left_key_binded, right_key_binded)) }) .collect::>()?; @@ -210,38 +210,14 @@ impl TryInto> for &protobuf::PhysicalPlanNode { let join_type = protobuf::JoinType::try_from(sort_merge_join.join_type) .expect("invalid JoinType"); - let join_filter = sort_merge_join - .join_filter - .as_ref() - .map(|f| { - let schema = Arc::new(convert_required!(f.schema)?); - let expression = try_parse_physical_expr_required(&f.expression, &schema)?; - let column_indices = f - .column_indices - .iter() - .map(|i| { - let side = - protobuf::JoinSide::try_from(i.side).expect("invalid JoinSide"); - Ok(ColumnIndex { - index: i.index as usize, - side: side.into(), - }) - }) - .collect::, PlanSerDeError>>()?; - - Ok(JoinFilter::new( - bind(expression, &schema)?, - column_indices, - schema.as_ref().clone(), - )) - }) - .map_or(Ok(None), |v: Result<_, PlanSerDeError>| v.map(Some))?; Ok(Arc::new(SortMergeJoinExec::try_new( + schema, left, right, on, - join_type.into(), - join_filter, + join_type + .try_into() + .map_err(|_| proto_error("invalid JoinType"))?, sort_options, )?)) } @@ -306,7 +282,7 @@ impl TryInto> for &protobuf::PhysicalPlanNode { self )) })?; - if let protobuf::physical_expr_node::ExprType::Sort(sort_expr) = expr { + if let ExprType::Sort(sort_expr) = expr { let expr = sort_expr .expr .as_ref() @@ -342,97 +318,58 @@ impl TryInto> for &protobuf::PhysicalPlanNode { sort.fetch_limit.as_ref().map(|limit| limit.limit as usize), ))) } + PhysicalPlanType::BroadcastJoinBuildHashMap(bhm) => { + let input: Arc = convert_box_required!(bhm.input)?; + let keys = bhm + .keys + .iter() + .map(|expr| { + Ok(bind( + try_parse_physical_expr(expr, &input.schema())?, + &input.schema(), + )?) + }) + .collect::>, Self::Error>>()?; + Ok(Arc::new(BroadcastJoinBuildHashMapExec::new(input, keys))) + } PhysicalPlanType::BroadcastJoin(broadcast_join) => { + let schema = Arc::new(convert_required!(broadcast_join.schema)?); let left: Arc = convert_box_required!(broadcast_join.left)?; let right: Arc = convert_box_required!(broadcast_join.right)?; let on: Vec<(Arc, Arc)> = broadcast_join .on .iter() .map(|col| { - let left_col: Column = into_required!(col.left)?; - let left_col_binded: Arc = - Arc::new(Column::new_with_schema(left_col.name(), &left.schema())?); - let right_col: Column = into_required!(col.right)?; - let right_col_binded: Arc = - Arc::new(Column::new_with_schema(right_col.name(), &right.schema())?); - Ok((left_col_binded, right_col_binded)) + let left_key = + try_parse_physical_expr(&col.left.as_ref().unwrap(), &left.schema())?; + let left_key_binded = bind(left_key, &left.schema())?; + let right_key = + try_parse_physical_expr(&col.right.as_ref().unwrap(), &right.schema())?; + let right_key_binded = bind(right_key, &right.schema())?; + Ok((left_key_binded, right_key_binded)) }) .collect::>()?; let join_type = protobuf::JoinType::try_from(broadcast_join.join_type) .expect("invalid JoinType"); - let join_filter = broadcast_join - .join_filter - .as_ref() - .map(|f| { - let schema = Arc::new(convert_required!(f.schema)?); - let expression = try_parse_physical_expr_required(&f.expression, &schema)?; - let column_indices = f - .column_indices - .iter() - .map(|i| { - let side = - protobuf::JoinSide::try_from(i.side).expect("invalid JoinSide"); - Ok(ColumnIndex { - index: i.index as usize, - side: side.into(), - }) - }) - .collect::, PlanSerDeError>>()?; - Ok(JoinFilter::new( - bind(expression, &schema)?, - column_indices, - schema.as_ref().clone(), - )) - }) - .map_or(Ok(None), |v: Result<_, PlanSerDeError>| v.map(Some))?; + let broadcast_side = protobuf::JoinSide::try_from(broadcast_join.broadcast_side) + .expect("invalid BroadcastSide"); + + let cached_build_hash_map_id = broadcast_join.cached_build_hash_map_id.clone(); Ok(Arc::new(BroadcastJoinExec::try_new( + schema, left, right, on, - join_type.into(), - join_filter, - )?)) - } - PhysicalPlanType::BroadcastNestedLoopJoin(bnlj) => { - let left: Arc = convert_box_required!(bnlj.left)?; - let right: Arc = convert_box_required!(bnlj.right)?; - let join_type = - protobuf::JoinType::try_from(bnlj.join_type).expect("invalid JoinType"); - let join_filter = bnlj - .join_filter - .as_ref() - .map(|f| { - let schema = Arc::new(convert_required!(f.schema)?); - let expression = try_parse_physical_expr_required(&f.expression, &schema)?; - let column_indices = f - .column_indices - .iter() - .map(|i| { - let side = - protobuf::JoinSide::try_from(i.side).expect("invalid JoinSide"); - Ok(ColumnIndex { - index: i.index as usize, - side: side.into(), - }) - }) - .collect::, PlanSerDeError>>()?; - - Ok(JoinFilter::new( - bind(expression, &schema)?, - column_indices, - schema.as_ref().clone(), - )) - }) - .map_or(Ok(None), |v: Result<_, PlanSerDeError>| v.map(Some))?; - - Ok(Arc::new(BroadcastNestedLoopJoinExec::try_new( - left, - right, - join_type.into(), - join_filter, + join_type + .try_into() + .map_err(|_| proto_error("invalid JoinType"))?, + broadcast_side + .try_into() + .map_err(|_| proto_error("invalid BroadcastSide"))?, + Some(cached_build_hash_map_id), )?)) } PhysicalPlanType::Union(union) => { diff --git a/native-engine/blaze-serde/src/lib.rs b/native-engine/blaze-serde/src/lib.rs index 30bd4c282..56cd4a6bf 100644 --- a/native-engine/blaze-serde/src/lib.rs +++ b/native-engine/blaze-serde/src/lib.rs @@ -15,10 +15,8 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Fields, IntervalUnit, Schema, TimeUnit}; -use datafusion::{ - common::JoinSide, logical_expr::Operator, prelude::JoinType, scalar::ScalarValue, -}; -use datafusion_ext_plans::agg::AggFunction; +use datafusion::{common::JoinSide, logical_expr::Operator, scalar::ScalarValue}; +use datafusion_ext_plans::{agg::AggFunction, joins::join_utils::JoinType}; use crate::error::PlanSerDeError; @@ -111,6 +109,7 @@ impl From for JoinType { protobuf::JoinType::Full => JoinType::Full, protobuf::JoinType::Semi => JoinType::LeftSemi, protobuf::JoinType::Anti => JoinType::LeftAnti, + protobuf::JoinType::Existence => JoinType::Existence, } } } diff --git a/native-engine/datafusion-ext-commons/src/lib.rs b/native-engine/datafusion-ext-commons/src/lib.rs index 72f622354..ece6438af 100644 --- a/native-engine/datafusion-ext-commons/src/lib.rs +++ b/native-engine/datafusion-ext-commons/src/lib.rs @@ -13,7 +13,6 @@ // limitations under the License. #![feature(new_uninit)] -#![feature(io_error_other)] #![feature(slice_swap_unchecked)] #![feature(vec_into_raw_parts)] @@ -85,9 +84,9 @@ pub fn batch_size() -> usize { batch_size } -// for better cache usage +// bigger for better radix sort performance pub const fn staging_mem_size_for_partial_sort() -> usize { - 4194304 * 8 / 10 + 8388608 } // use bigger batch memory size writing shuffling data diff --git a/native-engine/datafusion-ext-commons/src/spark_hash.rs b/native-engine/datafusion-ext-commons/src/spark_hash.rs index 6a76bb953..85dac3077 100644 --- a/native-engine/datafusion-ext-commons/src/spark_hash.rs +++ b/native-engine/datafusion-ext-commons/src/spark_hash.rs @@ -77,10 +77,8 @@ fn spark_compatible_murmur3_hash>(data: T, seed: u32) -> u32 { // avoid boundary checking in performance critical codes. // all operations are garenteed to be safe unsafe { - let mut h1 = hash_bytes_by_int( - std::slice::from_raw_parts(data.get_unchecked(0), len_aligned), - seed, - ); + let mut h1 = + hash_bytes_by_int(std::slice::from_raw_parts(data.as_ptr(), len_aligned), seed); for i in len_aligned..len { let half_word = *data.get_unchecked(i) as i8 as i32; diff --git a/native-engine/datafusion-ext-functions/src/spark_get_json_object.rs b/native-engine/datafusion-ext-functions/src/spark_get_json_object.rs index 966b2f668..ede47a407 100644 --- a/native-engine/datafusion-ext-functions/src/spark_get_json_object.rs +++ b/native-engine/datafusion-ext-functions/src/spark_get_json_object.rs @@ -194,8 +194,8 @@ enum ParsedJsonValue { #[derive(Debug)] enum HiveGetJsonObjectError { - InvalidJsonPath(String), - InvalidInput(String), + InvalidJsonPath, + InvalidInput, } struct HiveGetJsonObjectEvaluator { @@ -212,15 +212,11 @@ impl HiveGetJsonObjectEvaluator { evaluator.matchers.push(matcher); } if evaluator.matchers.first() != Some(&HiveGetJsonObjectMatcher::Root) { - return Err(HiveGetJsonObjectError::InvalidJsonPath( - "json path missing root".to_string(), - )); + return Err(HiveGetJsonObjectError::InvalidJsonPath); } evaluator.matchers.remove(0); // remove root matcher if evaluator.matchers.contains(&HiveGetJsonObjectMatcher::Root) { - return Err(HiveGetJsonObjectError::InvalidJsonPath( - "json path has more than one root".to_string(), - )); + return Err(HiveGetJsonObjectError::InvalidJsonPath); } Ok(evaluator) } @@ -240,9 +236,7 @@ impl HiveGetJsonObjectEvaluator { return Ok(v); } } - Err(HiveGetJsonObjectError::InvalidInput( - "invalid json string".to_string(), - )) + Err(HiveGetJsonObjectError::InvalidInput) } fn evaluate_with_value_serde_json( @@ -296,7 +290,7 @@ fn serde_json_value_to_string( serde_json::Value::Bool(b) => Ok(Some(b.to_string())), serde_json::Value::Array(_) | serde_json::Value::Object(_) => serde_json::to_string(value) .map(Some) - .map_err(|_| HiveGetJsonObjectError::InvalidInput("array to json error".to_string())), + .map_err(|_| HiveGetJsonObjectError::InvalidInput), } } @@ -310,7 +304,7 @@ fn sonic_value_to_string( sonic_rs::JsonType::Boolean => Ok(value.as_bool().map(|v| v.to_string())), _ => sonic_rs::to_string(value) .map(Some) - .map_err(|_| HiveGetJsonObjectError::InvalidInput("array to json error".to_string())), + .map_err(|_| HiveGetJsonObjectError::InvalidInput), } } @@ -352,9 +346,7 @@ impl HiveGetJsonObjectMatcher { } } if child_name.is_empty() { - return Err(HiveGetJsonObjectError::InvalidJsonPath( - "empty child name".to_string(), - )); + return Err(HiveGetJsonObjectError::InvalidJsonPath); } Ok(Some(Self::Child(child_name))) } @@ -372,24 +364,18 @@ impl HiveGetJsonObjectMatcher { chars.next(); } None => { - return Err(HiveGetJsonObjectError::InvalidJsonPath( - "unterminated subscript".to_string(), - )); + return Err(HiveGetJsonObjectError::InvalidJsonPath); } } } if index_str.is_empty() || index_str == "*" { return Ok(Some(Self::SubscriptAll)); } - let index = str::parse::(&index_str).map_err(|_| { - HiveGetJsonObjectError::InvalidJsonPath("invalid subscript index".to_string()) - })?; + let index = str::parse::(&index_str) + .map_err(|_| HiveGetJsonObjectError::InvalidJsonPath)?; Ok(Some(Self::Subscript(index))) } - Some(c) => Err(HiveGetJsonObjectError::InvalidJsonPath(format!( - "unexpected char in json path: {}", - c - ))), + Some(_) => Err(HiveGetJsonObjectError::InvalidJsonPath), } } diff --git a/native-engine/datafusion-ext-functions/src/spark_null_if.rs b/native-engine/datafusion-ext-functions/src/spark_null_if.rs index af753d5fa..4845a93b8 100644 --- a/native-engine/datafusion-ext-functions/src/spark_null_if.rs +++ b/native-engine/datafusion-ext-functions/src/spark_null_if.rs @@ -16,10 +16,7 @@ use std::sync::Arc; use arrow::{ array::*, - compute::{ - kernels::{cmp::eq, nullif::nullif}, - *, - }, + compute::kernels::{cmp::eq, nullif::nullif}, datatypes::*, }; use datafusion::{ @@ -87,7 +84,8 @@ pub fn spark_null_if_zero(args: &[ColumnarValue]) -> Result { ($dt:ident) => {{ type T = paste::paste! {arrow::datatypes::[<$dt Type>]}; let array = as_primitive_array::(array); - let eq_zeros = eq_scalar(array, T::default_value())?; + let _0 = PrimitiveArray::::new_scalar(Default::default()); + let eq_zeros = eq(array, &_0)?; Arc::new(nullif(array, &eq_zeros)?) as ArrayRef }}; } diff --git a/native-engine/datafusion-ext-functions/src/spark_strings.rs b/native-engine/datafusion-ext-functions/src/spark_strings.rs index 6eb5d5e94..6deaa7c8d 100644 --- a/native-engine/datafusion-ext-functions/src/spark_strings.rs +++ b/native-engine/datafusion-ext-functions/src/spark_strings.rs @@ -223,7 +223,9 @@ pub fn string_concat_ws(args: &[ColumnarValue]) -> Result { None => return Ok(Arg::Ignore), } } - if let ScalarValue::List(l) = scalar && l.data_type() == &DataType::Utf8 { + if let ScalarValue::List(l) = scalar + && l.data_type() == &DataType::Utf8 + { if l.is_null(0) { return Ok(Arg::Ignore); } diff --git a/native-engine/datafusion-ext-plans/Cargo.toml b/native-engine/datafusion-ext-plans/Cargo.toml index c8412fd41..ce9be4a3c 100644 --- a/native-engine/datafusion-ext-plans/Cargo.toml +++ b/native-engine/datafusion-ext-plans/Cargo.toml @@ -11,6 +11,7 @@ default = ["tokio/rt-multi-thread"] arrow = { workspace = true } async-trait = "0.1.80" base64 = "0.22.1" +bitvec = "1.0.1" byteorder = "1.5.0" bytes = "1.6.0" blaze-jni-bridge = { workspace = true } diff --git a/native-engine/datafusion-ext-plans/src/broadcast_join_build_hash_map_exec.rs b/native-engine/datafusion-ext-plans/src/broadcast_join_build_hash_map_exec.rs new file mode 100644 index 000000000..3f1ca6d65 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/broadcast_join_build_hash_map_exec.rs @@ -0,0 +1,150 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + any::Any, + fmt::{Debug, Formatter}, + sync::Arc, +}; + +use arrow::{compute::concat_batches, datatypes::SchemaRef}; +use datafusion::{ + common::Result, + execution::{SendableRecordBatchStream, TaskContext}, + physical_expr::{Partitioning, PhysicalExpr, PhysicalSortExpr}, + physical_plan::{ + metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, + stream::RecordBatchStreamAdapter, + DisplayAs, DisplayFormatType, ExecutionPlan, + }, +}; +use futures::{stream::once, TryStreamExt}; + +use crate::{ + common::output::{NextBatchWithTimer, TaskOutputter}, + joins::join_hash_map::{join_hash_map_schema, JoinHashMap}, +}; + +pub struct BroadcastJoinBuildHashMapExec { + input: Arc, + keys: Vec>, + metrics: ExecutionPlanMetricsSet, +} + +impl BroadcastJoinBuildHashMapExec { + pub fn new(input: Arc, keys: Vec>) -> Self { + Self { + input, + keys, + metrics: ExecutionPlanMetricsSet::new(), + } + } +} + +impl Debug for BroadcastJoinBuildHashMapExec { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "BroadcastJoinBuildHashMap [{:?}]", self.keys) + } +} + +impl DisplayAs for BroadcastJoinBuildHashMapExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!(f, "BroadcastJoinBuildHashMapExec [{:?}]", self.keys) + } +} + +impl ExecutionPlan for BroadcastJoinBuildHashMapExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + join_hash_map_schema(&self.input.schema()) + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.input.output_partitioning().partition_count()) + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + None + } + + fn children(&self) -> Vec> { + vec![self.input.clone()] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + Ok(Arc::new(Self::new(children[0].clone(), self.keys.clone()))) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + let input = self.input.execute(partition, context.clone())?; + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + once(execute_build_hash_map( + context, + input, + self.keys.clone(), + baseline_metrics, + )) + .try_flatten(), + ))) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } +} + +async fn execute_build_hash_map( + context: Arc, + mut input: SendableRecordBatchStream, + keys: Vec>, + metrics: BaselineMetrics, +) -> Result { + let elapsed_compute = metrics.elapsed_compute().clone(); + let mut timer = elapsed_compute.timer(); + + let mut data_batches = vec![]; + let data_schema = input.schema(); + + // collect all input batches + while let Some(batch) = input.next_batch(Some(&mut timer)).await? { + data_batches.push(batch); + } + let data_batch = concat_batches(&data_schema, data_batches.iter())?; + + // build hash map + let hash_map_schema = join_hash_map_schema(&data_schema); + let hash_map = JoinHashMap::try_from_data_batch(data_batch, &keys)?; + drop(timer); + + // output hash map batches as stream + context.output_with_sender("BuildHashMap", hash_map_schema, move |sender| async move { + let mut timer = elapsed_compute.timer(); + sender + .send(Ok(hash_map.into_hash_map_batch()?), Some(&mut timer)) + .await; + Ok(()) + }) +} diff --git a/native-engine/datafusion-ext-plans/src/broadcast_join_exec.rs b/native-engine/datafusion-ext-plans/src/broadcast_join_exec.rs index 201173c4d..de160af8f 100644 --- a/native-engine/datafusion-ext-plans/src/broadcast_join_exec.rs +++ b/native-engine/datafusion-ext-plans/src/broadcast_join_exec.rs @@ -15,90 +15,203 @@ use std::{ any::Any, fmt::{Debug, Formatter}, - sync::Arc, - task::Poll, - time::Duration, + future::Future, + pin::Pin, + sync::{Arc, Weak}, + time::{Duration, Instant}, }; -use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; -use blaze_jni_bridge::{ - conf, - conf::{BooleanConf, IntConf}, +use arrow::{ + array::RecordBatch, + compute::SortOptions, + datatypes::{DataType, SchemaRef}, }; +use async_trait::async_trait; use datafusion::{ - common::{Result, Statistics}, + common::{JoinSide, Result, Statistics}, execution::context::TaskContext, - logical_expr::JoinType, - physical_expr::PhysicalSortExpr, + physical_expr::{PhysicalExprRef, PhysicalSortExpr}, physical_plan::{ - expressions::Column, - joins::{ - utils::{build_join_schema, check_join_is_valid, JoinFilter, JoinOn}, - HashJoinExec, PartitionMode, - }, - memory::MemoryStream, - metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, + joins::utils::JoinOn, + metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, Time}, stream::RecordBatchStreamAdapter, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, }, }; -use datafusion_ext_commons::{df_execution_err, downcast_any}; -use futures::{stream::once, StreamExt, TryStreamExt}; +use datafusion_ext_commons::{ + batch_size, df_execution_err, streams::coalesce_stream::CoalesceInput, +}; +use futures::{StreamExt, TryStreamExt}; +use hashbrown::HashMap; +use once_cell::sync::OnceCell; use parking_lot::Mutex; -use crate::{sort_exec::SortExec, sort_merge_join_exec::SortMergeJoinExec}; +use crate::{ + common::{ + batch_statisitcs::{stat_input, InputBatchStatistics}, + column_pruning::ExecuteWithColumnPruning, + output::{TaskOutputter, WrappedRecordBatchSender}, + }, + joins::{ + bhj::{ + full_join::{ + LProbedFullOuterJoiner, LProbedInnerJoiner, LProbedLeftJoiner, LProbedRightJoiner, + RProbedFullOuterJoiner, RProbedInnerJoiner, RProbedLeftJoiner, RProbedRightJoiner, + }, + semi_join::{ + LProbedExistenceJoiner, LProbedLeftAntiJoiner, LProbedLeftSemiJoiner, + LProbedRightAntiJoiner, LProbedRightSemiJoiner, RProbedExistenceJoiner, + RProbedLeftAntiJoiner, RProbedLeftSemiJoiner, RProbedRightAntiJoiner, + RProbedRightSemiJoiner, + }, + }, + join_hash_map::{join_data_schema, JoinHashMap}, + join_utils::{JoinType, JoinType::*}, + JoinParams, JoinProjection, + }, +}; #[derive(Debug)] pub struct BroadcastJoinExec { - /// Left sorted joining execution plan left: Arc, - /// Right sorting joining execution plan right: Arc, - /// Set of common columns used to join on on: JoinOn, - /// How the join is performed join_type: JoinType, - /// Optional filter before outputting - join_filter: Option, - /// The schema once the join is applied + broadcast_side: JoinSide, schema: SchemaRef, - /// Execution metrics + cached_build_hash_map_id: Option, metrics: ExecutionPlanMetricsSet, } impl BroadcastJoinExec { pub fn try_new( + schema: SchemaRef, left: Arc, right: Arc, on: JoinOn, join_type: JoinType, - join_filter: Option, + broadcast_side: JoinSide, + cached_build_hash_map_id: Option, ) -> Result { - if matches!( - join_type, - JoinType::LeftSemi | JoinType::LeftAnti | JoinType::RightSemi | JoinType::RightAnti, - ) { - if join_filter.is_some() { - df_execution_err!("Semi/Anti join with filter is not supported yet")?; - } - } - - let left_schema = left.schema(); - let right_schema = right.schema(); - - check_join_is_valid(&left_schema, &right_schema, &on)?; - let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); - Ok(Self { left, right, on, join_type, - join_filter, + broadcast_side, schema, + cached_build_hash_map_id, metrics: ExecutionPlanMetricsSet::new(), }) } + + fn create_join_params(&self, projection: &[usize]) -> Result { + let left_schema = self.left.schema(); + let right_schema = self.right.schema(); + let (left_keys, right_keys): (Vec, Vec) = + self.on.iter().cloned().unzip(); + let key_data_types: Vec = self + .on + .iter() + .map(|(left_key, right_key)| { + Ok({ + let left_dt = left_key.data_type(&left_schema)?; + let right_dt = right_key.data_type(&right_schema)?; + if left_dt != right_dt { + df_execution_err!( + "join key data type differs {left_dt:?} <-> {right_dt:?}" + )?; + } + left_dt + }) + }) + .collect::>()?; + + let projection = JoinProjection::try_new( + self.join_type, + &self.schema, + &match self.broadcast_side { + JoinSide::Left => join_data_schema(&left_schema), + JoinSide::Right => left_schema.clone(), + }, + &match self.broadcast_side { + JoinSide::Left => right_schema.clone(), + JoinSide::Right => join_data_schema(&right_schema), + }, + projection, + )?; + + Ok(JoinParams { + join_type: self.join_type, + left_schema, + right_schema, + output_schema: self.schema(), + left_keys, + right_keys, + batch_size: batch_size(), + sort_options: vec![SortOptions::default(); self.on.len()], + projection, + key_data_types, + }) + } + + fn execute_with_projection( + &self, + partition: usize, + context: Arc, + projection: Vec, + ) -> Result { + let metrics = Arc::new(BaselineMetrics::new(&self.metrics, partition)); + let join_params = self.create_join_params(&projection)?; + let left = self.left.execute(partition, context.clone())?; + let right = self.right.execute(partition, context.clone())?; + let broadcast_side = self.broadcast_side; + let cached_build_hash_map_id = self.cached_build_hash_map_id.clone(); + + // stat probed side + let input_batch_stat = + InputBatchStatistics::from_metrics_set_and_blaze_conf(&self.metrics, partition)?; + let (left, right) = match broadcast_side { + JoinSide::Left => (left, stat_input(input_batch_stat, right)?), + JoinSide::Right => (stat_input(input_batch_stat, left)?, right), + }; + + let metrics_cloned = metrics.clone(); + let context_cloned = context.clone(); + let output_stream = Box::pin(RecordBatchStreamAdapter::new( + join_params.projection.schema.clone(), + futures::stream::once(async move { + context_cloned.output_with_sender( + "BroadcastJoin", + join_params.projection.schema.clone(), + move |sender| { + execute_join( + left, + right, + join_params, + broadcast_side, + cached_build_hash_map_id, + metrics_cloned, + sender, + ) + }, + ) + }) + .try_flatten(), + )); + Ok(context.coalesce_with_default_batch_size(output_stream, &metrics)?) + } +} + +impl ExecuteWithColumnPruning for BroadcastJoinExec { + fn execute_projected( + &self, + partition: usize, + context: Arc, + projection: &[usize], + ) -> Result { + self.execute_with_projection(partition, context, projection.to_vec()) + } } impl ExecutionPlan for BroadcastJoinExec { @@ -111,7 +224,10 @@ impl ExecutionPlan for BroadcastJoinExec { } fn output_partitioning(&self) -> Partitioning { - self.right.output_partitioning() + match self.broadcast_side { + JoinSide::Left => self.right.output_partitioning(), + JoinSide::Right => self.left.output_partitioning(), + } } fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { @@ -127,11 +243,13 @@ impl ExecutionPlan for BroadcastJoinExec { children: Vec>, ) -> Result> { Ok(Arc::new(Self::try_new( + self.schema.clone(), children[0].clone(), children[1].clone(), self.on.iter().cloned().collect(), self.join_type, - self.join_filter.clone(), + self.broadcast_side, + None, )?)) } @@ -140,21 +258,8 @@ impl ExecutionPlan for BroadcastJoinExec { partition: usize, context: Arc, ) -> Result { - let stream = execute_broadcast_join( - self.left.clone(), - self.right.clone(), - partition, - context, - self.on.clone(), - self.join_type, - self.join_filter.clone(), - BaselineMetrics::new(&self.metrics, partition), - ); - - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - once(stream).try_flatten(), - ))) + let projection = (0..self.schema.fields().len()).collect(); + self.execute_with_projection(partition, context, projection) } fn metrics(&self) -> Option { @@ -172,221 +277,188 @@ impl DisplayAs for BroadcastJoinExec { } } -async fn execute_broadcast_join( - left: Arc, - right: Arc, - partition: usize, - context: Arc, - on: JoinOn, - join_type: JoinType, - join_filter: Option, - metrics: BaselineMetrics, -) -> Result { - let enabled_fallback_to_smj = conf::BHJ_FALLBACKS_TO_SMJ_ENABLE.value()?; - let bhj_num_rows_limit = conf::BHJ_FALLBACKS_TO_SMJ_ROWS_THRESHOLD.value()? as usize; - let bhj_mem_size_limit = conf::BHJ_FALLBACKS_TO_SMJ_MEM_THRESHOLD.value()? as usize; - - // if broadcasted size is small enough, use hash join - // otherwise use sort-merge join - #[derive(Debug)] - enum JoinMode { - Hash, - SortMerge, - } - let mut join_mode = JoinMode::Hash; - - let left_schema = left.schema(); - let mut left = left; - - if enabled_fallback_to_smj { - let mut left_stream = left.execute(0, context.clone())?.fuse(); - let mut left_cached: Vec = vec![]; - let mut left_num_rows = 0; - let mut left_mem_size = 0; - - // read and cache batches from broadcasted side until reached limits - while let Some(batch) = left_stream.next().await.transpose()? { - left_num_rows += batch.num_rows(); - left_mem_size += batch.get_array_memory_size(); - left_cached.push(batch); - if left_num_rows > bhj_num_rows_limit || left_mem_size > bhj_mem_size_limit { - join_mode = JoinMode::SortMerge; - break; - } - } - - // convert left cached and rest batches into execution plan - let left_cached_stream: SendableRecordBatchStream = Box::pin(MemoryStream::try_new( - left_cached, - left_schema.clone(), - None, - )?); - let left_rest_stream: SendableRecordBatchStream = Box::pin(RecordBatchStreamAdapter::new( - left_schema.clone(), - left_stream, - )); - let left_stream: SendableRecordBatchStream = Box::pin(RecordBatchStreamAdapter::new( - left_schema.clone(), - left_cached_stream.chain(left_rest_stream), - )); - left = Arc::new(RecordBatchStreamsWrapperExec { - schema: left_schema.clone(), - stream: Mutex::new(Some(left_stream)), - output_partitioning: right.output_partitioning(), - }); - } - - match join_mode { - JoinMode::Hash => { - let join = Arc::new(HashJoinExec::try_new( - left.clone(), - right.clone(), - on, - join_filter, - &join_type, - PartitionMode::CollectLeft, - false, - )?); - log::info!("BroadcastJoin is using hash join mode: {:?}", &join); - - let join_schema = join.schema(); - let completed = join - .execute(partition, context)? - .chain(futures::stream::poll_fn(move |_| { - // update metrics - let join_metrics = join.metrics().unwrap(); - metrics.record_output(join_metrics.output_rows().unwrap_or(0)); - metrics.elapsed_compute().add_duration(Duration::from_nanos( - [ - join_metrics - .sum_by_name("build_time") - .map(|v| v.as_usize() as u64), - join_metrics - .sum_by_name("join_time") - .map(|v| v.as_usize() as u64), - ] - .into_iter() - .flatten() - .sum(), - )); - Poll::Ready(None) - })); - Ok(Box::pin(RecordBatchStreamAdapter::new( - join_schema, - completed, - ))) +async fn execute_join( + left: SendableRecordBatchStream, + right: SendableRecordBatchStream, + join_params: JoinParams, + broadcast_side: JoinSide, + cached_build_hash_map_id: Option, + metrics: Arc, + sender: Arc, +) -> Result<()> { + let start_time = Instant::now(); + let mut excluded_time_ns = 0; + let poll_time = Time::new(); + + let (mut probed, _keys, mut joiner): (_, _, Pin>) = match broadcast_side + { + JoinSide::Left => { + let right_schema = right.schema(); + let mut right_peeked = Box::pin(right.peekable()); + let (_, lmap_result) = futures::join!( + // fetch two sides asynchronously + async { + let timer = poll_time.timer(); + right_peeked.as_mut().peek().await; + drop(timer); + }, + collect_join_hash_map( + cached_build_hash_map_id, + left, + &join_params.left_keys, + poll_time.clone(), + ), + ); + let lmap = lmap_result?; + ( + Box::pin(RecordBatchStreamAdapter::new(right_schema, right_peeked)), + join_params.right_keys.clone(), + match join_params.join_type { + Inner => Box::pin(RProbedInnerJoiner::new(join_params, lmap, sender)), + Left => Box::pin(RProbedLeftJoiner::new(join_params, lmap, sender)), + Right => Box::pin(RProbedRightJoiner::new(join_params, lmap, sender)), + Full => Box::pin(RProbedFullOuterJoiner::new(join_params, lmap, sender)), + LeftSemi => Box::pin(RProbedLeftSemiJoiner::new(join_params, lmap, sender)), + LeftAnti => Box::pin(RProbedLeftAntiJoiner::new(join_params, lmap, sender)), + RightSemi => Box::pin(RProbedRightSemiJoiner::new(join_params, lmap, sender)), + RightAnti => Box::pin(RProbedRightAntiJoiner::new(join_params, lmap, sender)), + Existence => Box::pin(RProbedExistenceJoiner::new(join_params, lmap, sender)), + }, + ) } - JoinMode::SortMerge => { - let sort_exprs: Vec = on - .iter() - .map(|(_col_left, col_right)| PhysicalSortExpr { - expr: Arc::new(Column::new( - "", - downcast_any!(col_right, Column) - .expect("requires column") - .index(), - )), - options: Default::default(), - }) - .collect(); - - let right_sorted = Arc::new(SortExec::new(right, sort_exprs.clone(), None)); - let join = Arc::new(SortMergeJoinExec::try_new( - left.clone(), - right_sorted.clone(), - on, - join_type, - join_filter, - sort_exprs.into_iter().map(|se| se.options).collect(), - )?); - log::info!("BroadcastJoin is using sort-merge join mode: {:?}", &join); - - let join_schema = join.schema(); - let completed = join - .execute(partition, context)? - .chain(futures::stream::poll_fn(move |_| { - // update metrics - let right_sorted_metrics = right_sorted.metrics().unwrap(); - let join_metrics = join.metrics().unwrap(); - metrics.record_output(join_metrics.output_rows().unwrap_or(0)); - metrics.elapsed_compute().add_duration(Duration::from_nanos( - [ - right_sorted_metrics.elapsed_compute(), - join_metrics.elapsed_compute(), - ] - .into_iter() - .flatten() - .sum::() as u64, - )); - Poll::Ready(None) - })); - Ok(Box::pin(RecordBatchStreamAdapter::new( - join_schema, - completed, - ))) + JoinSide::Right => { + let left_schema = left.schema(); + let mut left_peeked = Box::pin(left.peekable()); + let (_, rmap_result) = futures::join!( + // fetch two sides asynchronizely + async { + let timer = poll_time.timer(); + left_peeked.as_mut().peek().await; + drop(timer); + }, + collect_join_hash_map( + cached_build_hash_map_id, + right, + &join_params.right_keys, + poll_time.clone(), + ), + ); + let rmap = rmap_result?; + ( + Box::pin(RecordBatchStreamAdapter::new(left_schema, left_peeked)), + join_params.left_keys.clone(), + match join_params.join_type { + Inner => Box::pin(LProbedInnerJoiner::new(join_params, rmap, sender)), + Left => Box::pin(LProbedLeftJoiner::new(join_params, rmap, sender)), + Right => Box::pin(LProbedRightJoiner::new(join_params, rmap, sender)), + Full => Box::pin(LProbedFullOuterJoiner::new(join_params, rmap, sender)), + LeftSemi => Box::pin(LProbedLeftSemiJoiner::new(join_params, rmap, sender)), + LeftAnti => Box::pin(LProbedLeftAntiJoiner::new(join_params, rmap, sender)), + RightSemi => Box::pin(LProbedRightSemiJoiner::new(join_params, rmap, sender)), + RightAnti => Box::pin(LProbedRightAntiJoiner::new(join_params, rmap, sender)), + Existence => Box::pin(LProbedExistenceJoiner::new(join_params, rmap, sender)), + }, + ) } + }; + + while let Some(batch) = { + let timer = poll_time.timer(); + let batch = probed.next().await.transpose()?; + drop(timer); + batch + } { + joiner.as_mut().join(batch).await?; } + joiner.as_mut().finish().await?; + metrics.record_output(joiner.num_output_rows()); + + excluded_time_ns += poll_time.value(); + excluded_time_ns += joiner.total_send_output_time(); + + // discount poll input and send output batch time + let mut join_time_ns = (Instant::now() - start_time).as_nanos() as u64; + join_time_ns -= excluded_time_ns as u64; + metrics + .elapsed_compute() + .add_duration(Duration::from_nanos(join_time_ns)); + Ok(()) } -pub struct RecordBatchStreamsWrapperExec { - pub schema: SchemaRef, - pub stream: Mutex>, - pub output_partitioning: Partitioning, +async fn collect_join_hash_map( + cached_build_hash_map_id: Option, + input: SendableRecordBatchStream, + key_exprs: &[PhysicalExprRef], + poll_time: Time, +) -> Result> { + Ok(match cached_build_hash_map_id { + Some(cached_id) => { + get_cached_join_hash_map(&cached_id, || async { + collect_join_hash_map_without_caching(input, key_exprs, poll_time).await + }) + .await? + } + None => { + let map = collect_join_hash_map_without_caching(input, key_exprs, poll_time).await?; + Arc::new(map) + } + }) } -impl Debug for RecordBatchStreamsWrapperExec { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "RecordBatchStreamsWrapper") +async fn collect_join_hash_map_without_caching( + mut input: SendableRecordBatchStream, + key_exprs: &[PhysicalExprRef], + poll_time: Time, +) -> Result { + let mut hash_map_batches = vec![]; + while let Some(batch) = { + let timer = poll_time.timer(); + let batch = input.next().await.transpose()?; + drop(timer); + batch + } { + hash_map_batches.push(batch); } -} - -impl DisplayAs for RecordBatchStreamsWrapperExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "RecordBatchStreamsWrapper") + match hash_map_batches.len() { + 0 => Ok(JoinHashMap::try_new_empty(input.schema(), key_exprs)?), + 1 => Ok(JoinHashMap::try_from_hash_map_batch( + hash_map_batches[0].clone(), + key_exprs, + )?), + n => df_execution_err!("expect zero or one hash map batch, got {n}"), } } -impl ExecutionPlan for RecordBatchStreamsWrapperExec { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - self.output_partitioning.clone() - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } +#[async_trait] +pub trait Joiner { + async fn join(self: Pin<&mut Self>, probed_batch: RecordBatch) -> Result<()>; + async fn finish(self: Pin<&mut Self>) -> Result<()>; - fn children(&self) -> Vec> { - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - unimplemented!() - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - let stream = std::mem::take(&mut *self.stream.lock()); - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema.clone(), - Box::pin(futures::stream::iter(stream).flatten()), - ))) - } + fn total_send_output_time(&self) -> usize; + fn num_output_rows(&self) -> usize; +} - fn statistics(&self) -> Result { - unimplemented!() +async fn get_cached_join_hash_map> + Send>( + cached_id: &str, + init: impl FnOnce() -> Fut, +) -> Result> { + type Slot = Arc>>; + static CACHED_JOIN_HASH_MAP: OnceCell>>> = OnceCell::new(); + + // TODO: remove expired keys from cached join hash map + let cached_join_hash_map = CACHED_JOIN_HASH_MAP.get_or_init(|| Arc::default()); + let slot = cached_join_hash_map + .lock() + .entry(cached_id.to_string()) + .or_default() + .clone(); + + let mut slot = slot.lock().await; + if let Some(cached) = slot.upgrade() { + Ok(cached) + } else { + let new = Arc::new(init().await?); + *slot = Arc::downgrade(&new); + Ok(new) } } diff --git a/native-engine/datafusion-ext-plans/src/broadcast_nested_loop_join_exec.rs b/native-engine/datafusion-ext-plans/src/broadcast_nested_loop_join_exec.rs deleted file mode 100644 index b52e77f00..000000000 --- a/native-engine/datafusion-ext-plans/src/broadcast_nested_loop_join_exec.rs +++ /dev/null @@ -1,252 +0,0 @@ -// Copyright 2022 The Blaze Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::{any::Any, fmt::Formatter, sync::Arc}; - -use arrow::datatypes::SchemaRef; -use datafusion::{ - common::{JoinType, Result, Statistics}, - execution::{SendableRecordBatchStream, TaskContext}, - physical_expr::{Partitioning, PhysicalSortExpr}, - physical_plan::{ - joins::{ - utils::{build_join_schema, check_join_is_valid, JoinFilter}, - NestedLoopJoinExec, - }, - memory::MemoryExec, - metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, - stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionPlan, - }, -}; -use datafusion_ext_commons::batch_size; -use futures::{stream::once, StreamExt, TryStreamExt}; -use parking_lot::Mutex; - -use crate::broadcast_join_exec::RecordBatchStreamsWrapperExec; - -#[derive(Debug)] -pub struct BroadcastNestedLoopJoinExec { - left: Arc, - right: Arc, - join_type: JoinType, - filter: Option, - schema: SchemaRef, - metrics: ExecutionPlanMetricsSet, -} - -impl BroadcastNestedLoopJoinExec { - pub fn try_new( - left: Arc, - right: Arc, - join_type: JoinType, - filter: Option, - ) -> Result { - let left_schema = left.schema(); - let right_schema = right.schema(); - check_join_is_valid(&left_schema, &right_schema, &[])?; - let (schema, _column_indices) = build_join_schema(&left_schema, &right_schema, &join_type); - - Ok(Self { - left, - right, - filter, - join_type, - schema: Arc::new(schema), - metrics: ExecutionPlanMetricsSet::new(), - }) - } -} - -impl DisplayAs for BroadcastNestedLoopJoinExec { - fn fmt_as(&self, _: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "BroadcastNestedLoopJoin") - } -} - -impl ExecutionPlan for BroadcastNestedLoopJoinExec { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - if left_is_build_side(self.join_type) { - self.right.output_partitioning() - } else { - self.left.output_partitioning() - } - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - fn children(&self) -> Vec> { - vec![self.left.clone(), self.right.clone()] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> Result> { - Ok(Arc::new(Self::try_new( - children[0].clone(), - children[1].clone(), - self.join_type, - self.filter.clone(), - )?)) - } - - fn execute( - &self, - partition: usize, - context: Arc, - ) -> Result { - let joined = Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - once(execute_join( - partition, - context, - self.left.clone(), - self.right.clone(), - self.join_type, - self.filter.clone(), - self.metrics.clone(), - )) - .try_flatten(), - )); - Ok(joined) - } - - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) - } - - fn statistics(&self) -> Result { - todo!() - } -} - -async fn execute_join( - partition: usize, - context: Arc, - left: Arc, - right: Arc, - join_type: JoinType, - filter: Option, - metrics: ExecutionPlanMetricsSet, -) -> Result { - // inner side - let mut inner_stream = if left_is_build_side(join_type) { - left.execute(partition, context.clone())? - } else { - right.execute(partition, context.clone())? - }; - let inner_schema = inner_stream.schema(); - let mut inner_batches = vec![]; - while let Some(batch) = inner_stream.next().await.transpose()? { - inner_batches.push(batch); - } - - let inner_batch_max_num_rows = inner_batches - .iter() - .map(|batch| batch.num_rows()) - .max() - .unwrap_or(0); - let inner_batch_max_mem_size = inner_batches - .iter() - .map(|batch| batch.get_array_memory_size()) - .max() - .unwrap_or(0); - - let target_output_num_rows = batch_size(); - let target_output_mem_size = 1 << 26; // 64MB - let inner_exec: Arc = - Arc::new(MemoryExec::try_new(&[inner_batches], inner_schema, None)?); - - // outer side - let (outer_schema, outer_partitioning, outer_stream) = if left_is_build_side(join_type) { - ( - right.schema(), - right.output_partitioning(), - right.execute(partition, context.clone())?, - ) - } else { - ( - left.schema(), - left.output_partitioning(), - left.execute(partition, context.clone())?, - ) - }; - let chunked_outer_stream = Box::pin(RecordBatchStreamAdapter::new( - outer_schema.clone(), - outer_stream.flat_map(move |batch_result| match batch_result { - Ok(batch) => { - let batch_num_rows = batch.num_rows(); - let batch_mem_size = batch.get_array_memory_size(); - let output_num_rows = batch_num_rows * inner_batch_max_num_rows; - let output_mem_size = batch_num_rows * inner_batch_max_mem_size - + batch_mem_size * inner_batch_max_num_rows; - let chunk_count = std::cmp::min( - (output_num_rows / target_output_num_rows).max(1), - (output_mem_size / target_output_mem_size).max(1), - ); - let chunk_len = (batch_num_rows / chunk_count).max(1); - - let mut chunks = vec![]; - for beg in (0..batch.num_rows()).step_by(chunk_len) { - chunks.push(Ok(batch.slice(beg, chunk_len.min(batch.num_rows() - beg)))); - } - futures::stream::iter(chunks) - } - Err(err) => futures::stream::iter(vec![Err(err)]), - }), - )); - let outer_exec: Arc = Arc::new(RecordBatchStreamsWrapperExec { - schema: outer_schema, - stream: Mutex::new(Some(chunked_outer_stream)), - output_partitioning: outer_partitioning, - }); - - // join with datafusion's builtin NestedLoopJoinExec - let nlj = if left_is_build_side(join_type) { - NestedLoopJoinExec::try_new(inner_exec, outer_exec, filter, &join_type)? - } else { - NestedLoopJoinExec::try_new(outer_exec, inner_exec, filter, &join_type)? - }; - let joined = nlj.execute(partition, context)?; - - let baseline_metrics = BaselineMetrics::new(&metrics, partition); - let output_stream = Box::pin(RecordBatchStreamAdapter::new( - joined.schema(), - joined.map(move |batch_result| { - if let Ok(batch) = &batch_result { - baseline_metrics.record_output(batch.num_rows()); - } - batch_result - }), - )); - Ok(output_stream) -} - -fn left_is_build_side(join_type: JoinType) -> bool { - matches!( - join_type, - JoinType::Right | JoinType::RightSemi | JoinType::RightAnti | JoinType::Full - ) -} diff --git a/native-engine/datafusion-ext-plans/src/common/batch_selection.rs b/native-engine/datafusion-ext-plans/src/common/batch_selection.rs index 6aa839551..a5e789cd5 100644 --- a/native-engine/datafusion-ext-plans/src/common/batch_selection.rs +++ b/native-engine/datafusion-ext-plans/src/common/batch_selection.rs @@ -41,16 +41,33 @@ pub fn take_batch_opt( take_batch_internal(batch, indices) } +pub fn take_cols( + cols: &[ArrayRef], + indices: impl IntoIterator, +) -> Result> { + let indices: UInt32Array = + PrimitiveArray::from_iter(indices.into_iter().map(|idx| idx.to_u32().unwrap())); + take_cols_internal(cols, &indices) +} + +pub fn take_cols_opt( + cols: &[ArrayRef], + indices: impl IntoIterator>, +) -> Result> { + let indices: UInt32Array = PrimitiveArray::from_iter( + indices + .into_iter() + .map(|opt| opt.map(|idx| idx.to_u32().unwrap())), + ); + take_cols_internal(cols, &indices) +} + fn take_batch_internal(batch: RecordBatch, indices: UInt32Array) -> Result { let taken_num_batch_rows = indices.len(); let schema = batch.schema(); - let cols = batch.columns().to_vec(); - drop(batch); // we would like to release batch as soon as possible + let cols = batch.columns(); - let cols = cols - .into_iter() - .map(|c| Ok(arrow::compute::take(&c, &indices, None)?)) - .collect::>()?; + let cols = take_cols_internal(cols, &indices)?; drop(indices); let taken = RecordBatch::try_new_with_options( @@ -61,6 +78,14 @@ fn take_batch_internal(batch: RecordBatch, indices: UInt32Array) -> Result Result> { + let cols = cols + .into_iter() + .map(|c| Ok(arrow::compute::take(&c, indices, None)?)) + .collect::>()?; + Ok(cols) +} + pub fn interleave_batches( schema: SchemaRef, batches: &[RecordBatch], diff --git a/native-engine/datafusion-ext-plans/src/common/output.rs b/native-engine/datafusion-ext-plans/src/common/output.rs index b1a0a2807..d888026ef 100644 --- a/native-engine/datafusion-ext-plans/src/common/output.rs +++ b/native-engine/datafusion-ext-plans/src/common/output.rs @@ -20,6 +20,7 @@ use std::{ }; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; +use async_trait::async_trait; use blaze_jni_bridge::is_task_running; use datafusion::{ common::Result, @@ -221,3 +222,34 @@ impl TaskOutputter for Arc { WrappedRecordBatchSender::cancel_task(self); } } + +#[async_trait] +pub trait NextBatchWithTimer { + async fn next_batch( + &mut self, + stop_timer: Option<&mut ScopedTimerGuard<'_>>, + ) -> Result>; +} + +#[async_trait] +impl NextBatchWithTimer for SendableRecordBatchStream { + async fn next_batch( + &mut self, + stop_timer: Option<&mut ScopedTimerGuard<'_>>, + ) -> Result> { + struct StopScopedTimerGuard<'a, 'z>(&'a mut ScopedTimerGuard<'z>); + impl<'a, 'z> StopScopedTimerGuard<'a, 'z> { + fn new(timer: &'a mut ScopedTimerGuard<'z>) -> Self { + timer.stop(); + Self(timer) + } + } + impl Drop for StopScopedTimerGuard<'_, '_> { + fn drop(&mut self) { + self.0.restart(); + } + } + let _stop_timer = stop_timer.map(|timer| StopScopedTimerGuard::new(timer)); + self.next().await.transpose() + } +} diff --git a/native-engine/datafusion-ext-plans/src/ipc_reader_exec.rs b/native-engine/datafusion-ext-plans/src/ipc_reader_exec.rs index fb8651ce1..9688e5af2 100644 --- a/native-engine/datafusion-ext-plans/src/ipc_reader_exec.rs +++ b/native-engine/datafusion-ext-plans/src/ipc_reader_exec.rs @@ -192,8 +192,8 @@ pub async fn read_ipc( })); while let Some(batch) = { - let reader_cloned = reader.clone(); - tokio::task::spawn_blocking(move || reader_cloned.clone().lock().read_batch()) + let reader = reader.clone(); + tokio::task::spawn_blocking(move || reader.lock().read_batch()) .await .or_else(|err| df_execution_err!("{err}"))?? } { diff --git a/native-engine/datafusion-ext-plans/src/joins/bhj/full_join.rs b/native-engine/datafusion-ext-plans/src/joins/bhj/full_join.rs new file mode 100644 index 000000000..ca51b5629 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/bhj/full_join.rs @@ -0,0 +1,324 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + pin::Pin, + sync::{ + atomic::{AtomicUsize, Ordering::Relaxed}, + Arc, + }, +}; + +use arrow::array::{new_null_array, ArrayRef, RecordBatch}; +use async_trait::async_trait; +use bitvec::{bitvec, prelude::BitVec}; +use datafusion::{common::Result, physical_plan::metrics::Time}; + +use crate::{ + broadcast_join_exec::Joiner, + common::{batch_selection::take_cols, output::WrappedRecordBatchSender}, + joins::{ + bhj::{ + filter_joined_indices, + full_join::ProbeSide::{L, R}, + ProbeSide, + }, + join_hash_map::{join_create_hashes, JoinHashMap}, + JoinParams, + }, +}; + +#[derive(std::marker::ConstParamTy, Clone, Copy, PartialEq, Eq)] +pub struct JoinerParams { + probe_side: ProbeSide, + probe_side_outer: bool, + build_side_outer: bool, +} + +impl JoinerParams { + const fn new(probe_side: ProbeSide, probe_side_outer: bool, build_side_outer: bool) -> Self { + Self { + probe_side, + probe_side_outer, + build_side_outer, + } + } +} + +const LEFT_PROBED_INNER: JoinerParams = JoinerParams::new(L, false, false); +const LEFT_PROBED_LEFT: JoinerParams = JoinerParams::new(L, true, false); +const LEFT_PROBED_RIGHT: JoinerParams = JoinerParams::new(L, false, true); +const LEFT_PROBED_OUTER: JoinerParams = JoinerParams::new(L, true, true); + +const RIGHT_PROBED_INNER: JoinerParams = JoinerParams::new(R, false, false); +const RIGHT_PROBED_LEFT: JoinerParams = JoinerParams::new(R, false, true); +const RIGHT_PROBED_RIGHT: JoinerParams = JoinerParams::new(R, true, false); +const RIGHT_PROBED_OUTER: JoinerParams = JoinerParams::new(R, true, true); + +pub type LProbedInnerJoiner = FullJoiner; +pub type LProbedLeftJoiner = FullJoiner; +pub type LProbedRightJoiner = FullJoiner; +pub type LProbedFullOuterJoiner = FullJoiner; +pub type RProbedInnerJoiner = FullJoiner; +pub type RProbedLeftJoiner = FullJoiner; +pub type RProbedRightJoiner = FullJoiner; +pub type RProbedFullOuterJoiner = FullJoiner; + +pub struct FullJoiner { + join_params: JoinParams, + output_sender: Arc, + map: Arc, + map_joined: BitVec, + send_output_time: Time, + output_rows: AtomicUsize, +} + +impl FullJoiner

{ + pub fn new( + join_params: JoinParams, + map: Arc, + output_sender: Arc, + ) -> Self { + let map_joined = bitvec![0; map.data_batch().num_rows()]; + Self { + join_params, + output_sender, + map, + map_joined, + send_output_time: Time::default(), + output_rows: AtomicUsize::new(0), + } + } + + fn create_probed_key_columns(&self, probed_batch: &RecordBatch) -> Result> { + let probed_key_exprs = match P.probe_side { + L => &self.join_params.left_keys, + R => &self.join_params.right_keys, + }; + let probed_key_columns: Vec = probed_key_exprs + .iter() + .map(|expr| { + Ok(expr + .evaluate(probed_batch)? + .into_array(probed_batch.num_rows())?) + }) + .collect::>()?; + Ok(probed_key_columns) + } + + async fn flush(&self, probe_cols: Vec, build_cols: Vec) -> Result<()> { + let output_batch = RecordBatch::try_new( + self.join_params.output_schema.clone(), + match P.probe_side { + L => [probe_cols, build_cols].concat(), + R => [build_cols, probe_cols].concat(), + }, + )?; + self.output_rows.fetch_add(output_batch.num_rows(), Relaxed); + + let timer = self.send_output_time.timer(); + self.output_sender.send(Ok(output_batch), None).await; + drop(timer); + Ok(()) + } + + async fn flush_hash_joined( + mut self: Pin<&mut Self>, + probed_batch: &RecordBatch, + probed_key_columns: &[ArrayRef], + probed_joined: &mut BitVec, + mut hash_joined_probe_indices: Vec, + mut hash_joined_build_indices: Vec, + ) -> Result<()> { + filter_joined_indices( + probed_key_columns, + self.map.key_columns(), + &mut hash_joined_probe_indices, + &mut hash_joined_build_indices, + )?; + let probe_indices = hash_joined_probe_indices; + let build_indices = hash_joined_build_indices; + + let pprojected = match P.probe_side { + L => self + .join_params + .projection + .project_left(probed_batch.columns()), + R => self + .join_params + .projection + .project_right(probed_batch.columns()), + }; + let mprojected = match P.probe_side { + L => self + .join_params + .projection + .project_right(self.map.data_batch().columns()), + R => self + .join_params + .projection + .project_left(self.map.data_batch().columns()), + }; + for &idx in &probe_indices { + probed_joined.set(idx as usize, true); + } + let pcols = if probe_indices.len() == probed_batch.num_rows() && probed_joined.all() { + // fast path for the case where every probed records have 1-to-1 joined + pprojected + } else { + take_cols(&pprojected, probe_indices)? + }; + + for &idx in &build_indices { + self.map_joined.set(idx as usize, true); + } + let bcols = take_cols(&mprojected, build_indices)?; + + self.flush(pcols, bcols).await?; + Ok(()) + } +} + +#[async_trait] +impl Joiner for FullJoiner

{ + async fn join(mut self: Pin<&mut Self>, probed_batch: RecordBatch) -> Result<()> { + let mut hash_joined_probe_indices: Vec = vec![]; + let mut hash_joined_build_indices: Vec = vec![]; + let mut probed_joined = bitvec![0; probed_batch.num_rows()]; + let batch_size = self.join_params.batch_size.max(probed_batch.num_rows()); + + let probed_key_columns = self.create_probed_key_columns(&probed_batch)?; + let probed_hashes = join_create_hashes(probed_batch.num_rows(), &probed_key_columns)?; + + // join by hash code + for (row_idx, &hash) in probed_hashes.iter().enumerate() { + let mut maybe_joined = false; + if let Some(entries) = self.map.entry_indices(hash) { + for map_idx in entries { + hash_joined_probe_indices.push(row_idx as u32); + hash_joined_build_indices.push(map_idx); + } + maybe_joined = true; + } + + if maybe_joined && hash_joined_probe_indices.len() > batch_size { + self.as_mut() + .flush_hash_joined( + &probed_batch, + &probed_key_columns, + &mut probed_joined, + std::mem::take(&mut hash_joined_probe_indices), + std::mem::take(&mut hash_joined_build_indices), + ) + .await?; + } + } + if !hash_joined_probe_indices.is_empty() { + self.as_mut() + .flush_hash_joined( + &probed_batch, + &probed_key_columns, + &mut probed_joined, + hash_joined_probe_indices, + hash_joined_build_indices, + ) + .await?; + } + + // output unjoined rows of probed side + if P.probe_side_outer { + let probed_unjoined_indices = probed_joined + .iter() + .enumerate() + .filter(|(_, joined)| !**joined) + .map(|(idx, _)| idx as u32) + .collect::>(); + + let pprojected = match P.probe_side { + L => self + .join_params + .projection + .project_left(probed_batch.columns()), + R => self + .join_params + .projection + .project_right(probed_batch.columns()), + }; + let mprojected = match P.probe_side { + L => self + .join_params + .projection + .project_right(self.map.data_batch().columns()), + R => self + .join_params + .projection + .project_left(self.map.data_batch().columns()), + }; + + let bcols = mprojected + .iter() + .map(|col| new_null_array(col.data_type(), probed_unjoined_indices.len())) + .collect::>(); + + let pcols = take_cols(&pprojected, probed_unjoined_indices)?; + self.as_mut().flush(pcols, bcols).await?; + } + Ok(()) + } + + async fn finish(mut self: Pin<&mut Self>) -> Result<()> { + // output unjoined rows of probed side + let map_joined = std::mem::take(&mut self.map_joined); + if P.build_side_outer { + let map_unjoined_indices = map_joined + .into_iter() + .enumerate() + .filter(|(_, joined)| !joined) + .map(|(idx, _)| idx as u32) + .collect::>(); + + let pschema = match P.probe_side { + L => &self.join_params.left_schema, + R => &self.join_params.right_schema, + }; + let mprojected = match P.probe_side { + L => self + .join_params + .projection + .project_right(self.map.data_batch().columns()), + R => self + .join_params + .projection + .project_left(self.map.data_batch().columns()), + }; + + let pcols = pschema + .fields() + .iter() + .map(|field| new_null_array(field.data_type(), map_unjoined_indices.len())) + .collect::>(); + let bcols = take_cols(&mprojected, map_unjoined_indices)?; + self.as_mut().flush(pcols, bcols).await?; + } + Ok(()) + } + + fn total_send_output_time(&self) -> usize { + self.send_output_time.value() + } + + fn num_output_rows(&self) -> usize { + self.output_rows.load(Relaxed) + } +} diff --git a/native-engine/datafusion-ext-plans/src/joins/bhj/mod.rs b/native-engine/datafusion-ext-plans/src/joins/bhj/mod.rs new file mode 100644 index 000000000..57d934cd5 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/bhj/mod.rs @@ -0,0 +1,146 @@ +// Copyright 2022 The Blaze Authors +// +// 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 arrow::{ + array::*, + datatypes::{DataType, IntervalUnit, TimeUnit}, +}; +use datafusion::common::Result; +use datafusion_ext_commons::{df_execution_err, downcast_any}; + +pub mod full_join; +pub mod semi_join; + +#[derive(std::marker::ConstParamTy, Clone, Copy, PartialEq, Eq)] +pub enum ProbeSide { + L, + R, +} + +fn filter_joined_indices( + key_columns1: &[ArrayRef], + key_columns2: &[ArrayRef], + indices1: &mut Vec, + indices2: &mut Vec, +) -> Result<()> { + fn filter_one( + key_column1: &ArrayRef, + key_column2: &ArrayRef, + indices1: &mut Vec, + indices2: &mut Vec, + ) -> Result<()> { + macro_rules! filter_atomic { + ($cast_type:ty) => {{ + let col1 = downcast_any!(key_column1, $cast_type)?; + let col2 = downcast_any!(key_column2, $cast_type)?; + let mut valid_count = 0; + for i in 0..indices1.len() { + let idx1 = indices1[i] as usize; + let idx2 = indices2[i] as usize; + if col1.is_valid(idx1) && col2.is_valid(idx2) && { + let v1 = col1.value(idx1); + let v2 = col2.value(idx2); + v1 == v2 + } { + indices1[valid_count] = indices1[i]; + indices2[valid_count] = indices2[i]; + valid_count += 1; + } + } + indices1.truncate(valid_count); + indices2.truncate(valid_count); + }}; + } + + let dt1 = key_column1.data_type(); + let dt2 = key_column2.data_type(); + if dt1 != dt2 { + return df_execution_err!("join key data type not matched: {dt1:?} <-> {dt2:?}"); + } + match dt1 { + DataType::Null => { + indices1.clear(); + indices2.clear(); + } + DataType::Boolean => filter_atomic!(BooleanArray), + DataType::Int8 => filter_atomic!(Int8Array), + DataType::Int16 => filter_atomic!(Int16Array), + DataType::Int32 => filter_atomic!(Int32Array), + DataType::Int64 => filter_atomic!(Int64Array), + DataType::UInt8 => filter_atomic!(UInt8Array), + DataType::UInt16 => filter_atomic!(UInt16Array), + DataType::UInt32 => filter_atomic!(UInt32Array), + DataType::UInt64 => filter_atomic!(UInt64Array), + DataType::Float16 => filter_atomic!(Float16Array), + DataType::Float32 => filter_atomic!(Float32Array), + DataType::Float64 => filter_atomic!(Float64Array), + DataType::Timestamp(unit, _) => match unit { + TimeUnit::Second => filter_atomic!(TimestampSecondArray), + TimeUnit::Millisecond => filter_atomic!(TimestampMillisecondArray), + TimeUnit::Microsecond => filter_atomic!(TimestampMicrosecondArray), + TimeUnit::Nanosecond => filter_atomic!(TimestampNanosecondArray), + }, + DataType::Date32 => filter_atomic!(Date32Array), + DataType::Date64 => filter_atomic!(Date64Array), + DataType::Time32(unit) => match unit { + TimeUnit::Second => filter_atomic!(Time32SecondArray), + TimeUnit::Millisecond => filter_atomic!(Time32MillisecondArray), + TimeUnit::Microsecond => filter_atomic!(Time32MillisecondArray), + TimeUnit::Nanosecond => filter_atomic!(Time32MillisecondArray), + }, + DataType::Time64(unit) => match unit { + TimeUnit::Microsecond => filter_atomic!(Time64MicrosecondArray), + TimeUnit::Nanosecond => filter_atomic!(Time64NanosecondArray), + _ => return df_execution_err!("unsupported time64 unit: {unit:?}"), + }, + DataType::Duration(unit) => match unit { + TimeUnit::Second => filter_atomic!(DurationSecondArray), + TimeUnit::Millisecond => filter_atomic!(DurationMillisecondArray), + TimeUnit::Microsecond => filter_atomic!(DurationMicrosecondArray), + TimeUnit::Nanosecond => filter_atomic!(DurationNanosecondArray), + }, + DataType::Interval(unit) => match unit { + IntervalUnit::YearMonth => filter_atomic!(IntervalYearMonthArray), + IntervalUnit::DayTime => filter_atomic!(IntervalDayTimeArray), + IntervalUnit::MonthDayNano => filter_atomic!(IntervalMonthDayNanoArray), + }, + DataType::Binary => filter_atomic!(BinaryArray), + DataType::FixedSizeBinary(_) => filter_atomic!(FixedSizeBinaryArray), + DataType::LargeBinary => filter_atomic!(LargeBinaryArray), + DataType::Utf8 => filter_atomic!(StringArray), + DataType::LargeUtf8 => filter_atomic!(LargeStringArray), + DataType::List(_) => filter_atomic!(ListArray), + DataType::FixedSizeList(..) => filter_atomic!(FixedSizeListArray), + DataType::LargeList(_) => filter_atomic!(LargeListArray), + DataType::Struct(_) => filter_joined_indices( + key_column1.as_struct().columns(), + key_column2.as_struct().columns(), + indices1, + indices2, + )?, + DataType::Decimal128(..) => filter_atomic!(Decimal128Array), + DataType::Decimal256(..) => filter_atomic!(Decimal256Array), + DataType::Map(..) => filter_atomic!(MapArray), + dt => { + return df_execution_err!("unsupported data type: {dt:?}"); + } + } + Ok(()) + } + + for (key_column1, key_column2) in key_columns1.iter().zip(key_columns2) { + filter_one(key_column1, key_column2, indices1, indices2)?; + } + Ok(()) +} diff --git a/native-engine/datafusion-ext-plans/src/joins/bhj/semi_join.rs b/native-engine/datafusion-ext-plans/src/joins/bhj/semi_join.rs new file mode 100644 index 000000000..8c168f00c --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/bhj/semi_join.rs @@ -0,0 +1,283 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + pin::Pin, + sync::{ + atomic::{AtomicUsize, Ordering::Relaxed}, + Arc, + }, +}; + +use arrow::array::{ArrayRef, BooleanArray, RecordBatch}; +use async_trait::async_trait; +use bitvec::{bitvec, prelude::BitVec}; +use datafusion::{common::Result, physical_plan::metrics::Time}; + +use crate::{ + broadcast_join_exec::Joiner, + common::{batch_selection::take_cols, output::WrappedRecordBatchSender}, + joins::{ + bhj::{ + filter_joined_indices, + semi_join::{ + ProbeSide::{L, R}, + SemiMode::{Anti, Existence, Semi}, + }, + ProbeSide, + }, + join_hash_map::{join_create_hashes, JoinHashMap}, + JoinParams, + }, +}; + +#[derive(std::marker::ConstParamTy, Clone, Copy, PartialEq, Eq)] +pub enum SemiMode { + Semi, + Anti, + Existence, +} + +#[derive(std::marker::ConstParamTy, Clone, Copy, PartialEq, Eq)] +pub struct JoinerParams { + probe_side: ProbeSide, + probe_is_join_side: bool, + mode: SemiMode, +} + +impl JoinerParams { + const fn new(probe_side: ProbeSide, probe_is_join_side: bool, mode: SemiMode) -> Self { + Self { + probe_side, + probe_is_join_side, + mode, + } + } +} + +const LEFT_PROBED_LEFT_SEMI: JoinerParams = JoinerParams::new(L, true, Semi); +const LEFT_PROBED_LEFT_ANTI: JoinerParams = JoinerParams::new(L, true, Anti); +const LEFT_PROBED_RIGHT_SEMI: JoinerParams = JoinerParams::new(L, false, Semi); +const LEFT_PROBED_RIGHT_ANTI: JoinerParams = JoinerParams::new(L, false, Anti); +const LEFT_PROBED_EXISTENCE: JoinerParams = JoinerParams::new(L, true, Existence); +const RIGHT_PROBED_LEFT_SEMI: JoinerParams = JoinerParams::new(R, false, Semi); +const RIGHT_PROBED_LEFT_ANTI: JoinerParams = JoinerParams::new(R, false, Anti); +const RIGHT_PROBED_RIGHT_SEMI: JoinerParams = JoinerParams::new(R, true, Semi); +const RIGHT_PROBED_RIGHT_ANTI: JoinerParams = JoinerParams::new(R, true, Anti); +const RIGHT_PROBED_EXISTENCE: JoinerParams = JoinerParams::new(R, false, Existence); + +pub type LProbedLeftSemiJoiner = SemiJoiner; +pub type LProbedLeftAntiJoiner = SemiJoiner; +pub type LProbedRightSemiJoiner = SemiJoiner; +pub type LProbedRightAntiJoiner = SemiJoiner; +pub type LProbedExistenceJoiner = SemiJoiner; +pub type RProbedLeftSemiJoiner = SemiJoiner; +pub type RProbedLeftAntiJoiner = SemiJoiner; +pub type RProbedRightSemiJoiner = SemiJoiner; +pub type RProbedRightAntiJoiner = SemiJoiner; +pub type RProbedExistenceJoiner = SemiJoiner; + +pub struct SemiJoiner { + join_params: JoinParams, + output_sender: Arc, + map_joined: BitVec, + map: Arc, + send_output_time: Time, + output_rows: AtomicUsize, +} + +impl SemiJoiner

{ + pub fn new( + join_params: JoinParams, + map: Arc, + output_sender: Arc, + ) -> Self { + let map_joined = bitvec![0; map.data_batch().num_rows()]; + Self { + join_params, + output_sender, + map, + map_joined, + send_output_time: Time::new(), + output_rows: AtomicUsize::new(0), + } + } + + fn create_probed_key_columns(&self, probed_batch: &RecordBatch) -> Result> { + let probed_key_exprs = match P.probe_side { + L => &self.join_params.left_keys, + R => &self.join_params.right_keys, + }; + let probed_key_columns: Vec = probed_key_exprs + .iter() + .map(|expr| { + Ok(expr + .evaluate(probed_batch)? + .into_array(probed_batch.num_rows())?) + }) + .collect::>()?; + Ok(probed_key_columns) + } + + async fn flush(&self, cols: Vec) -> Result<()> { + let output_batch = RecordBatch::try_new(self.join_params.output_schema.clone(), cols)?; + self.output_rows.fetch_add(output_batch.num_rows(), Relaxed); + + let timer = self.send_output_time.timer(); + self.output_sender.send(Ok(output_batch), None).await; + drop(timer); + Ok(()) + } + + fn flush_hash_joined( + mut self: Pin<&mut Self>, + probed_key_columns: &[ArrayRef], + probed_joined: &mut BitVec, + mut hash_joined_probe_indices: Vec, + mut hash_joined_build_indices: Vec, + ) -> Result<()> { + filter_joined_indices( + probed_key_columns, + self.map.key_columns(), + &mut hash_joined_probe_indices, + &mut hash_joined_build_indices, + )?; + let probe_indices = hash_joined_probe_indices; + let build_indices = hash_joined_build_indices; + + for &idx in &probe_indices { + probed_joined.set(idx as usize, true); + } + for &idx in &build_indices { + self.map_joined.set(idx as usize, true); + } + Ok(()) + } +} + +#[async_trait] +impl Joiner for SemiJoiner

{ + async fn join(mut self: Pin<&mut Self>, probed_batch: RecordBatch) -> Result<()> { + let mut hash_joined_probe_indices: Vec = vec![]; + let mut hash_joined_build_indices: Vec = vec![]; + let mut probed_joined = bitvec![0; probed_batch.num_rows()]; + + let probed_key_columns = self.create_probed_key_columns(&probed_batch)?; + let probed_hashes = join_create_hashes(probed_batch.num_rows(), &probed_key_columns)?; + + // join by hash code + for (row_idx, &hash) in probed_hashes.iter().enumerate() { + let mut maybe_joined = false; + if let Some(entries) = self.map.entry_indices(hash) { + for map_idx in entries { + hash_joined_probe_indices.push(row_idx as u32); + hash_joined_build_indices.push(map_idx); + } + maybe_joined = true; + } + + if maybe_joined && hash_joined_probe_indices.len() >= self.join_params.batch_size { + self.as_mut().flush_hash_joined( + &probed_key_columns, + &mut probed_joined, + std::mem::take(&mut hash_joined_probe_indices), + std::mem::take(&mut hash_joined_build_indices), + )?; + } + } + if !hash_joined_probe_indices.is_empty() { + self.as_mut().flush_hash_joined( + &probed_key_columns, + &mut probed_joined, + hash_joined_probe_indices, + hash_joined_build_indices, + )?; + } + + if P.probe_is_join_side { + let pprojected = match P.probe_side { + L => self + .join_params + .projection + .project_left(probed_batch.columns()), + R => self + .join_params + .projection + .project_right(probed_batch.columns()), + }; + let pcols = match P.mode { + Semi | Anti => { + let probed_indices = probed_joined + .into_iter() + .enumerate() + .filter(|(_, joined)| (P.mode == Semi) ^ !joined) + .map(|(idx, _)| idx as u32) + .collect::>(); + take_cols(&pprojected, probed_indices)? + } + Existence => { + let exists_col = Arc::new(BooleanArray::from( + probed_joined.into_iter().collect::>(), + )); + [pprojected, vec![exists_col]].concat() + } + }; + self.as_mut().flush(pcols).await?; + } + Ok(()) + } + + async fn finish(mut self: Pin<&mut Self>) -> Result<()> { + if !P.probe_is_join_side { + let mprojected = match P.probe_side { + L => self + .join_params + .projection + .project_right(self.map.data_batch().columns()), + R => self + .join_params + .projection + .project_left(self.map.data_batch().columns()), + }; + let map_joined = std::mem::take(&mut self.map_joined); + let pcols = match P.mode { + Semi | Anti => { + let map_indices = map_joined + .into_iter() + .enumerate() + .filter(|(_, joined)| (P.mode == Semi) ^ !joined) + .map(|(idx, _)| idx as u32) + .collect::>(); + take_cols(&mprojected, map_indices)? + } + Existence => { + let exists_col = Arc::new(BooleanArray::from( + map_joined.into_iter().collect::>(), + )); + [mprojected, vec![exists_col]].concat() + } + }; + self.as_mut().flush(pcols).await?; + } + Ok(()) + } + + fn total_send_output_time(&self) -> usize { + self.send_output_time.value() + } + + fn num_output_rows(&self) -> usize { + self.output_rows.load(Relaxed) + } +} diff --git a/native-engine/datafusion-ext-plans/src/joins/join_hash_map.rs b/native-engine/datafusion-ext-plans/src/joins/join_hash_map.rs new file mode 100644 index 000000000..8bd1a5731 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/join_hash_map.rs @@ -0,0 +1,340 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{ + io::{Cursor, Read, Write}, + slice::{from_raw_parts, from_raw_parts_mut}, + sync::Arc, +}; + +use arrow::{ + array::{ArrayRef, AsArray, BinaryBuilder, RecordBatch}, + datatypes::{DataType, Field, FieldRef, Schema, SchemaRef}, +}; +use byteorder::{NativeEndian, ReadBytesExt, WriteBytesExt}; +use datafusion::{common::Result, physical_expr::PhysicalExprRef}; +use datafusion_ext_commons::spark_hash::create_hashes; +use hashbrown::HashMap; +use itertools::Itertools; +use once_cell::sync::OnceCell; + +use crate::common::batch_selection::take_batch; + +pub struct Table { + entry_offsets: Vec, + entry_lens: Vec, + item_indices: Vec, + item_hashes: Vec, +} + +impl Table { + pub fn new_empty() -> Self { + let num_entries = Self::num_entries_of_rows(0); + Self { + entry_offsets: vec![0; num_entries], + entry_lens: vec![0; num_entries], + item_indices: vec![], + item_hashes: vec![], + } + } + + pub fn try_from_key_columns( + num_rows: usize, + data_batch: RecordBatch, + key_columns: &[ArrayRef], + ) -> Result<(Self, RecordBatch)> { + // returns the new data batch sorted by hashes + + assert!( + num_rows < 1073741824, + "join hash table: number of rows exceeded 2^30: {num_rows}" + ); + + let num_entries = Self::num_entries_of_rows(num_rows) as u32; + let item_hashes = join_create_hashes(num_rows, &key_columns)?; + + // sort record batch by hashes for better compression and data locality + let (indices, item_hashes): (Vec, Vec) = item_hashes + .into_iter() + .enumerate() + .sorted_unstable_by_key(|(_idx, hash)| *hash) + .unzip(); + let data_batch = take_batch(data_batch, indices)?; + + let mut entries_to_row_indices: HashMap> = HashMap::new(); + for (row_idx, hash) in item_hashes.iter().enumerate() { + let entry = hash % num_entries; + entries_to_row_indices + .entry(entry) + .or_default() + .push(row_idx as u32); + } + + let mut entry_offsets = Vec::with_capacity(num_entries as usize); + let mut entry_lens = Vec::with_capacity(num_entries as usize); + let mut item_indices = Vec::with_capacity(num_rows); + for entry in 0..num_entries { + match entries_to_row_indices.get(&entry) { + Some(row_indices) => { + entry_offsets.push(item_indices.len() as u32); + entry_lens.push(row_indices.len() as u32); + item_indices.extend_from_slice(row_indices); + } + None => { + entry_offsets.push(item_indices.len() as u32); + entry_lens.push(0); + } + } + } + let new = Self { + entry_offsets, + entry_lens, + item_indices, + item_hashes, + }; + Ok((new, data_batch)) + } + + pub fn try_from_raw_bytes(raw_bytes: &[u8]) -> Result { + let mut cursor = Cursor::new(raw_bytes); + let num_rows = cursor.read_u32::()? as usize; + let num_entries = Self::num_entries_of_rows(num_rows); + + let mut new = Self { + entry_offsets: vec![0; num_entries], + entry_lens: vec![0; num_entries], + item_indices: vec![0; num_rows], + item_hashes: vec![0; num_rows], + }; + + unsafe { + // safety: read integer arrays as raw bytes + cursor.read_exact(from_raw_parts_mut( + new.entry_offsets.as_mut_ptr() as *mut u8, + num_entries * 4, + ))?; + cursor.read_exact(from_raw_parts_mut( + new.entry_lens.as_mut_ptr() as *mut u8, + num_entries * 4, + ))?; + cursor.read_exact(from_raw_parts_mut( + new.item_indices.as_mut_ptr() as *mut u8, + num_rows * 4, + ))?; + cursor.read_exact(from_raw_parts_mut( + new.item_hashes.as_mut_ptr() as *mut u8, + num_rows * 4, + ))?; + } + Ok(new) + } + + pub fn try_into_raw_bytes(self) -> Result> { + let num_entries = self.entry_offsets.len(); + let num_rows = self.item_indices.len(); + let mut raw_bytes = Vec::with_capacity(num_entries * 8 + num_rows * 4 + 4); + + raw_bytes.write_u32::(num_rows as u32)?; + unsafe { + // safety: write integer arrays as raw bytes + raw_bytes.write_all(from_raw_parts( + self.entry_offsets.as_ptr() as *const u8, + num_entries * 4, + ))?; + raw_bytes.write_all(from_raw_parts( + self.entry_lens.as_ptr() as *const u8, + num_entries * 4, + ))?; + raw_bytes.write_all(from_raw_parts( + self.item_indices.as_ptr() as *const u8, + num_rows * 4, + ))?; + raw_bytes.write_all(from_raw_parts( + self.item_hashes.as_ptr() as *const u8, + num_rows * 4, + ))?; + } + Ok(raw_bytes) + } + + pub fn entry<'a>(&'a self, hash: u32) -> Option + 'a> { + let entry = hash % (self.entry_offsets.len() as u32); + let len = self.entry_lens[entry as usize] as usize; + if len > 0 { + let offset = self.entry_offsets[entry as usize] as usize; + Some( + self.item_indices[offset..][..len] + .iter() + .cloned() + .filter(move |&idx| self.item_hashes[idx as usize] == hash), + ) + } else { + None + } + } + + fn num_entries_of_rows(num_rows: usize) -> usize { + num_rows * 3 + 1 + } +} + +pub struct JoinHashMap { + data_batch: RecordBatch, + key_columns: Vec, + table: Table, +} + +impl JoinHashMap { + pub fn try_from_data_batch( + data_batch: RecordBatch, + key_exprs: &[PhysicalExprRef], + ) -> Result { + let key_columns: Vec = key_exprs + .iter() + .map(|expr| { + Ok(expr + .evaluate(&data_batch)? + .into_array(data_batch.num_rows())?) + }) + .collect::>()?; + + let (table, data_batch) = + Table::try_from_key_columns(data_batch.num_rows(), data_batch, &key_columns)?; + Ok(JoinHashMap { + data_batch, + key_columns, + table, + }) + } + + pub fn try_from_hash_map_batch( + hash_map_batch: RecordBatch, + key_exprs: &[PhysicalExprRef], + ) -> Result { + let mut data_batch = hash_map_batch.clone(); + let table = Table::try_from_raw_bytes( + data_batch + .remove_column(data_batch.num_columns() - 1) + .as_binary::() + .value(0), + )?; + let key_columns: Vec = key_exprs + .iter() + .map(|expr| { + Ok(expr + .evaluate(&data_batch)? + .into_array(data_batch.num_rows())?) + }) + .collect::>()?; + Ok(Self { + data_batch, + key_columns, + table, + }) + } + + pub fn try_new_empty( + hash_map_schema: SchemaRef, + key_exprs: &[PhysicalExprRef], + ) -> Result { + let table = Table::new_empty(); + let data_batch = RecordBatch::new_empty(hash_map_schema); + let key_columns: Vec = key_exprs + .iter() + .map(|expr| { + Ok(expr + .evaluate(&data_batch)? + .into_array(data_batch.num_rows())?) + }) + .collect::>()?; + Ok(Self { + data_batch, + key_columns, + table, + }) + } + + pub fn data_schema(&self) -> SchemaRef { + self.data_batch().schema() + } + + pub fn data_batch(&self) -> &RecordBatch { + &self.data_batch + } + + pub fn key_columns(&self) -> &[ArrayRef] { + &self.key_columns + } + + pub fn entry_indices<'a>(&'a self, hash: u32) -> Option + 'a> { + self.table.entry(hash) + } + + pub fn into_hash_map_batch(self) -> Result { + let schema = join_hash_map_schema(&self.data_batch.schema()); + if self.data_batch.num_rows() == 0 { + return Ok(RecordBatch::new_empty(schema)); + } + let mut table_col_builder = BinaryBuilder::new(); + table_col_builder.append_value(&self.table.try_into_raw_bytes()?); + for _ in 1..self.data_batch.num_rows() { + table_col_builder.append_null(); + } + let table_col: ArrayRef = Arc::new(table_col_builder.finish()); + Ok(RecordBatch::try_new( + schema, + vec![self.data_batch.columns().to_vec(), vec![table_col]].concat(), + )?) + } +} + +#[inline] +pub fn join_data_schema(hash_map_schema: &SchemaRef) -> SchemaRef { + Arc::new(Schema::new( + hash_map_schema + .fields() + .iter() + .take(hash_map_schema.fields().len() - 1) // exclude hash map column + .cloned() + .collect::>(), + )) +} + +#[inline] +pub fn join_hash_map_schema(data_schema: &SchemaRef) -> SchemaRef { + Arc::new(Schema::new( + data_schema + .fields() + .iter() + .map(|field| Arc::new(field.as_ref().clone().with_nullable(true))) + .chain(std::iter::once(join_table_field())) + .collect::>(), + )) +} + +#[inline] +pub fn join_create_hashes(num_rows: usize, key_columns: &[ArrayRef]) -> Result> { + const JOIN_HASH_RANDOM_SEED: u32 = 0x90ec4058; + let mut hashes = vec![JOIN_HASH_RANDOM_SEED; num_rows]; + create_hashes(key_columns, &mut hashes)?; + Ok(hashes) +} + +#[inline] +fn join_table_field() -> FieldRef { + static BHJ_KEY_FIELD: OnceCell = OnceCell::new(); + BHJ_KEY_FIELD + .get_or_init(|| Arc::new(Field::new("~TABLE", DataType::Binary, true))) + .clone() +} diff --git a/native-engine/datafusion-ext-plans/src/joins/join_utils.rs b/native-engine/datafusion-ext-plans/src/joins/join_utils.rs new file mode 100644 index 000000000..076cfa165 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/join_utils.rs @@ -0,0 +1,64 @@ +// Copyright 2022 The Blaze Authors +// +// 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 datafusion::common::{DataFusionError, Result}; +use datafusion_ext_commons::df_execution_err; + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum JoinType { + Inner, + Left, + Right, + Full, + LeftAnti, + RightAnti, + LeftSemi, + RightSemi, + Existence, +} + +impl TryFrom for datafusion::prelude::JoinType { + type Error = DataFusionError; + + fn try_from(value: JoinType) -> Result { + match value { + JoinType::Inner => Ok(datafusion::prelude::JoinType::Inner), + JoinType::Left => Ok(datafusion::prelude::JoinType::Left), + JoinType::Right => Ok(datafusion::prelude::JoinType::Right), + JoinType::Full => Ok(datafusion::prelude::JoinType::Full), + JoinType::LeftAnti => Ok(datafusion::prelude::JoinType::LeftAnti), + JoinType::RightAnti => Ok(datafusion::prelude::JoinType::RightAnti), + JoinType::LeftSemi => Ok(datafusion::prelude::JoinType::LeftSemi), + JoinType::RightSemi => Ok(datafusion::prelude::JoinType::RightSemi), + other => df_execution_err!("unsupported join type: {other:?}"), + } + } +} + +impl TryFrom for JoinType { + type Error = DataFusionError; + + fn try_from(value: datafusion::prelude::JoinType) -> Result { + match value { + datafusion::prelude::JoinType::Inner => Ok(JoinType::Inner), + datafusion::prelude::JoinType::Left => Ok(JoinType::Left), + datafusion::prelude::JoinType::Right => Ok(JoinType::Right), + datafusion::prelude::JoinType::Full => Ok(JoinType::Full), + datafusion::prelude::JoinType::LeftAnti => Ok(JoinType::LeftAnti), + datafusion::prelude::JoinType::RightAnti => Ok(JoinType::RightAnti), + datafusion::prelude::JoinType::LeftSemi => Ok(JoinType::LeftSemi), + datafusion::prelude::JoinType::RightSemi => Ok(JoinType::RightSemi), + } + } +} diff --git a/native-engine/datafusion-ext-plans/src/joins/mod.rs b/native-engine/datafusion-ext-plans/src/joins/mod.rs new file mode 100644 index 000000000..3505a9a77 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/mod.rs @@ -0,0 +1,113 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use arrow::{ + array::ArrayRef, + compute::SortOptions, + datatypes::{DataType, SchemaRef}, +}; +use datafusion::{common::Result, physical_expr::PhysicalExprRef}; + +use crate::joins::{join_utils::JoinType, stream_cursor::StreamCursor}; + +pub mod join_hash_map; +pub mod join_utils; +pub mod stream_cursor; + +// join implementations +pub mod bhj; +pub mod smj; +mod test; + +#[derive(Debug, Clone)] +pub struct JoinParams { + pub join_type: JoinType, + pub left_schema: SchemaRef, + pub right_schema: SchemaRef, + pub output_schema: SchemaRef, + pub left_keys: Vec, + pub right_keys: Vec, + pub key_data_types: Vec, + pub sort_options: Vec, + pub projection: JoinProjection, + pub batch_size: usize, +} + +#[derive(Debug, Clone)] +pub struct JoinProjection { + pub schema: SchemaRef, + pub left_schema: SchemaRef, + pub right_schema: SchemaRef, + pub left: Vec, + pub right: Vec, +} + +impl JoinProjection { + pub fn try_new( + join_type: JoinType, + schema: &SchemaRef, + left_schema: &SchemaRef, + right_schema: &SchemaRef, + projection: &[usize], + ) -> Result { + let projected_schema = Arc::new(schema.project(projection)?); + let mut left = vec![]; + let mut right = vec![]; + + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + for &i in projection { + if i < left_schema.fields().len() { + left.push(i); + } else if i - left_schema.fields().len() < right_schema.fields().len() { + right.push(i - left_schema.fields().len()); + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + left = projection.to_vec(); + } + JoinType::RightAnti | JoinType::RightSemi => { + right = projection.to_vec(); + } + JoinType::Existence => { + for &i in projection { + if i < left_schema.fields().len() { + left.push(i); + } + } + } + } + Ok(Self { + schema: projected_schema, + left_schema: Arc::new(left_schema.project(&left)?), + right_schema: Arc::new(right_schema.project(&right)?), + left, + right, + }) + } + + pub fn project_left(&self, cols: &[ArrayRef]) -> Vec { + self.left.iter().map(|&i| cols[i].clone()).collect() + } + + pub fn project_right(&self, cols: &[ArrayRef]) -> Vec { + self.right.iter().map(|&i| cols[i].clone()).collect() + } +} + +pub type Idx = (usize, usize); +pub type StreamCursors = (StreamCursor, StreamCursor); diff --git a/native-engine/datafusion-ext-plans/src/joins/smj/existence_join.rs b/native-engine/datafusion-ext-plans/src/joins/smj/existence_join.rs new file mode 100644 index 000000000..5749eb01b --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/smj/existence_join.rs @@ -0,0 +1,175 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{cmp::Ordering, pin::Pin, sync::Arc}; + +use arrow::array::{ArrayRef, RecordBatch, RecordBatchOptions}; +use async_trait::async_trait; +use datafusion::{common::Result, physical_plan::metrics::Time}; +use datafusion_ext_commons::suggested_output_batch_mem_size; + +use crate::{ + common::{batch_selection::interleave_batches, output::WrappedRecordBatchSender}, + compare_cursor, cur_forward, + joins::{Idx, JoinParams, StreamCursors}, + sort_merge_join_exec::Joiner, +}; + +pub struct ExistenceJoiner { + join_params: JoinParams, + output_sender: Arc, + indices: Vec, + exists: Vec, + send_output_time: Time, + output_rows: usize, +} + +impl ExistenceJoiner { + pub fn new(join_params: JoinParams, output_sender: Arc) -> Self { + Self { + join_params, + output_sender, + indices: vec![], + exists: vec![], + send_output_time: Time::new(), + output_rows: 0, + } + } + + fn should_flush(&self, curs: &StreamCursors) -> bool { + if self.indices.len() >= self.join_params.batch_size { + return true; + } + + if curs.0.num_buffered_batches() + curs.1.num_buffered_batches() >= 6 + && curs.0.mem_size() + curs.1.mem_size() > suggested_output_batch_mem_size() + { + if let Some(first_idx) = self.indices.first() { + if first_idx.0 < curs.0.cur_idx.0 { + return true; + } + } + } + false + } + + async fn flush(mut self: Pin<&mut Self>, curs: &mut StreamCursors) -> Result<()> { + let indices = std::mem::take(&mut self.indices); + let num_rows = indices.len(); + let cols = interleave_batches( + curs.0.projected_batch_schema.clone(), + &curs.0.projected_batches, + &indices, + )?; + + let exists = std::mem::take(&mut self.exists); + let exists_col: ArrayRef = Arc::new(arrow::array::BooleanArray::from(exists)); + + let output_batch = RecordBatch::try_new_with_options( + self.join_params.output_schema.clone(), + [cols.columns().to_vec(), vec![exists_col]].concat(), + &RecordBatchOptions::new().with_row_count(Some(num_rows)), + )?; + + if output_batch.num_rows() > 0 { + self.output_rows += output_batch.num_rows(); + + let timer = self.send_output_time.timer(); + self.output_sender.send(Ok(output_batch), None).await; + drop(timer); + } + Ok(()) + } +} + +#[async_trait] +impl Joiner for ExistenceJoiner { + async fn join(mut self: Pin<&mut Self>, curs: &mut StreamCursors) -> Result<()> { + while !curs.0.finished && !curs.1.finished { + let mut lidx = curs.0.cur_idx; + let mut ridx = curs.1.cur_idx; + + match compare_cursor!(curs) { + Ordering::Less => { + self.indices.push(curs.0.cur_idx); + self.exists.push(false); + cur_forward!(curs.0); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.0 + .set_min_reserved_idx(*self.indices.first().unwrap_or(&curs.0.cur_idx)); + } + Ordering::Greater => { + cur_forward!(curs.1); + curs.1 + .set_min_reserved_idx(*self.indices.first().unwrap_or(&curs.1.cur_idx)); + } + Ordering::Equal => { + loop { + self.indices.push(lidx); + self.exists.push(true); + cur_forward!(curs.0); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.0 + .set_min_reserved_idx(*self.indices.first().unwrap_or(&lidx)); + + if !curs.0.finished && curs.0.key(curs.0.cur_idx) == curs.0.key(lidx) { + lidx = curs.0.cur_idx; + continue; + } + break; + } + + // skip all right equal rows + loop { + cur_forward!(curs.1); + curs.1.set_min_reserved_idx(ridx); + + if !curs.1.finished && curs.1.key(curs.1.cur_idx) == curs.1.key(ridx) { + ridx = curs.1.cur_idx; + continue; + } + break; + } + } + } + } + + while !curs.0.finished { + self.indices.push(curs.0.cur_idx); + self.exists.push(false); + cur_forward!(curs.0); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.0 + .set_min_reserved_idx(*self.indices.first().unwrap_or(&curs.0.cur_idx)); + } + if !self.indices.is_empty() { + self.flush(curs).await?; + } + Ok(()) + } + + fn total_send_output_time(&self) -> usize { + self.send_output_time.value() + } + + fn num_output_rows(&self) -> usize { + self.output_rows + } +} diff --git a/native-engine/datafusion-ext-plans/src/joins/smj/full_join.rs b/native-engine/datafusion-ext-plans/src/joins/smj/full_join.rs new file mode 100644 index 000000000..55967f457 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/smj/full_join.rs @@ -0,0 +1,248 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{cmp::Ordering, pin::Pin, sync::Arc}; + +use arrow::array::{RecordBatch, RecordBatchOptions}; +use async_trait::async_trait; +use datafusion::{common::Result, physical_plan::metrics::Time}; +use datafusion_ext_commons::suggested_output_batch_mem_size; +use smallvec::{smallvec, SmallVec}; + +use crate::{ + common::{batch_selection::interleave_batches, output::WrappedRecordBatchSender}, + compare_cursor, cur_forward, + joins::{Idx, JoinParams, StreamCursors}, + sort_merge_join_exec::Joiner, +}; + +pub struct FullJoiner { + join_params: JoinParams, + output_sender: Arc, + lindices: Vec, + rindices: Vec, + send_output_time: Time, + output_rows: usize, +} + +pub type InnerJoiner = FullJoiner; +pub type LeftOuterJoiner = FullJoiner; +pub type RightOuterJoiner = FullJoiner; +pub type FullOuterJoiner = FullJoiner; + +impl FullJoiner { + pub fn new(join_params: JoinParams, output_sender: Arc) -> Self { + Self { + join_params, + output_sender, + lindices: vec![], + rindices: vec![], + send_output_time: Time::new(), + output_rows: 0, + } + } + + fn should_flush(&self, curs: &StreamCursors) -> bool { + if self.lindices.len() >= self.join_params.batch_size { + return true; + } + + if curs.0.num_buffered_batches() + curs.1.num_buffered_batches() >= 6 + && curs.0.mem_size() + curs.1.mem_size() > suggested_output_batch_mem_size() + { + if let Some(first_lidx) = self.lindices.first() { + if first_lidx.0 < curs.0.cur_idx.0 { + return true; + } + } + if let Some(first_ridx) = self.rindices.first() { + if first_ridx.0 < curs.1.cur_idx.0 { + return true; + } + } + } + false + } + + async fn flush(mut self: Pin<&mut Self>, curs: &mut StreamCursors) -> Result<()> { + let lindices = std::mem::take(&mut self.lindices); + let rindices = std::mem::take(&mut self.rindices); + let num_rows = lindices.len(); + assert_eq!(lindices.len(), rindices.len()); + + let lcols = interleave_batches( + curs.0.projected_batch_schema.clone(), + &curs.0.projected_batches, + &lindices, + )?; + let rcols = interleave_batches( + curs.1.projected_batch_schema.clone(), + &curs.1.projected_batches, + &rindices, + )?; + let output_batch = RecordBatch::try_new_with_options( + self.join_params.projection.schema.clone(), + [lcols.columns(), rcols.columns()].concat(), + &RecordBatchOptions::new().with_row_count(Some(num_rows)), + )?; + + if output_batch.num_rows() > 0 { + self.output_rows += output_batch.num_rows(); + + let timer = self.send_output_time.timer(); + self.output_sender.send(Ok(output_batch), None).await; + drop(timer); + } + Ok(()) + } +} + +#[async_trait] +impl Joiner for FullJoiner { + async fn join(mut self: Pin<&mut Self>, curs: &mut StreamCursors) -> Result<()> { + while !curs.0.finished && !curs.1.finished { + let mut lidx = curs.0.cur_idx; + let mut ridx = curs.1.cur_idx; + match compare_cursor!(curs) { + Ordering::Less => { + if L_OUTER { + self.lindices.push(lidx); + self.rindices.push(Idx::default()); + } + cur_forward!(curs.0); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.0 + .set_min_reserved_idx(*self.lindices.first().unwrap_or(&lidx)); + } + Ordering::Greater => { + if R_OUTER { + self.lindices.push(Idx::default()); + self.rindices.push(ridx); + } + cur_forward!(curs.1); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.1 + .set_min_reserved_idx(*self.rindices.first().unwrap_or(&ridx)); + } + Ordering::Equal => { + cur_forward!(curs.0); + cur_forward!(curs.1); + self.lindices.push(lidx); + self.rindices.push(ridx); + + let mut equal_lindices: SmallVec<[Idx; 16]> = smallvec![lidx]; + let mut equal_rindices: SmallVec<[Idx; 16]> = smallvec![ridx]; + let mut last_lidx = lidx; + let mut last_ridx = ridx; + lidx = curs.0.cur_idx; + ridx = curs.1.cur_idx; + let mut l_equal = !curs.0.finished && curs.0.key(lidx) == curs.0.key(last_lidx); + let mut r_equal = !curs.1.finished && curs.1.key(ridx) == curs.1.key(last_ridx); + + while l_equal || r_equal { + if l_equal { + for &ridx in &equal_rindices { + self.lindices.push(lidx); + self.rindices.push(ridx); + } + if r_equal { + equal_lindices.push(lidx); + } + cur_forward!(curs.0); + last_lidx = lidx; + lidx = curs.0.cur_idx; + } else { + curs.1 + .set_min_reserved_idx(*self.rindices.first().unwrap_or(&last_ridx)); + } + + if r_equal { + for &lidx in &equal_lindices { + self.lindices.push(lidx); + self.rindices.push(ridx); + } + if l_equal { + equal_rindices.push(ridx); + } + cur_forward!(curs.1); + last_ridx = ridx; + ridx = curs.1.cur_idx; + } else { + curs.0 + .set_min_reserved_idx(*self.lindices.first().unwrap_or(&last_lidx)); + } + + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + l_equal = l_equal + && !curs.0.finished + && curs.0.key(lidx) == curs.0.key(last_lidx); + r_equal = r_equal + && !curs.1.finished + && curs.1.key(ridx) == curs.1.key(last_ridx); + } + + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.0 + .set_min_reserved_idx(*self.lindices.first().unwrap_or(&curs.0.cur_idx)); + curs.1 + .set_min_reserved_idx(*self.rindices.first().unwrap_or(&curs.1.cur_idx)); + } + } + } + + // at least one side is finished, consume the other side if it is an outer side + while L_OUTER && !curs.0.finished { + let lidx = curs.0.cur_idx; + self.lindices.push(lidx); + self.rindices.push(Idx::default()); + cur_forward!(curs.0); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.0 + .set_min_reserved_idx(*self.lindices.first().unwrap_or(&lidx)); + } + while R_OUTER && !curs.1.finished { + let ridx = curs.1.cur_idx; + self.lindices.push(Idx::default()); + self.rindices.push(ridx); + cur_forward!(curs.1); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.1 + .set_min_reserved_idx(*self.rindices.first().unwrap_or(&ridx)); + } + if !self.lindices.is_empty() { + self.flush(curs).await?; + } + Ok(()) + } + + fn total_send_output_time(&self) -> usize { + self.send_output_time.value() + } + + fn num_output_rows(&self) -> usize { + self.output_rows + } +} diff --git a/native-engine/datafusion-ext-plans/src/joins/smj/mod.rs b/native-engine/datafusion-ext-plans/src/joins/smj/mod.rs new file mode 100644 index 000000000..8bcdadff1 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/smj/mod.rs @@ -0,0 +1,17 @@ +// Copyright 2022 The Blaze Authors +// +// 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 existence_join; +pub mod full_join; +pub mod semi_join; diff --git a/native-engine/datafusion-ext-plans/src/joins/smj/semi_join.rs b/native-engine/datafusion-ext-plans/src/joins/smj/semi_join.rs new file mode 100644 index 000000000..fd5f9351e --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/smj/semi_join.rs @@ -0,0 +1,252 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::{cmp::Ordering, pin::Pin, sync::Arc}; + +use arrow::array::{RecordBatch, RecordBatchOptions}; +use async_trait::async_trait; +use datafusion::{common::Result, physical_plan::metrics::Time}; +use datafusion_ext_commons::suggested_output_batch_mem_size; + +use crate::{ + common::{batch_selection::interleave_batches, output::WrappedRecordBatchSender}, + compare_cursor, cur_forward, + joins::{ + smj::semi_join::SemiJoinSide::{L, R}, + Idx, JoinParams, StreamCursors, + }, + sort_merge_join_exec::Joiner, +}; + +#[derive(std::marker::ConstParamTy, Clone, Copy, PartialEq, Eq)] +pub enum SemiJoinSide { + L, + R, +} + +#[derive(std::marker::ConstParamTy, Clone, Copy, PartialEq, Eq)] +pub struct JoinerParams { + join_side: SemiJoinSide, + semi: bool, +} + +impl JoinerParams { + const fn new(join_side: SemiJoinSide, semi: bool) -> Self { + Self { join_side, semi } + } +} +pub struct SemiJoiner { + join_params: JoinParams, + output_sender: Arc, + indices: Vec, + send_output_time: Time, + output_rows: usize, +} + +const LEFT_SEMI: JoinerParams = JoinerParams::new(L, true); +const LEFT_ANTI: JoinerParams = JoinerParams::new(L, false); +const RIGHT_SEMI: JoinerParams = JoinerParams::new(R, true); +const RIGHT_ANTI: JoinerParams = JoinerParams::new(R, false); + +pub type LeftSemiJoiner = SemiJoiner; +pub type LeftAntiJoiner = SemiJoiner; +pub type RightSemiJoiner = SemiJoiner; +pub type RightAntiJoiner = SemiJoiner; + +impl SemiJoiner

{ + pub fn new(join_params: JoinParams, output_sender: Arc) -> Self { + Self { + join_params, + output_sender, + indices: vec![], + send_output_time: Time::new(), + output_rows: 0, + } + } + + fn should_flush(&self, curs: &StreamCursors) -> bool { + if self.indices.len() >= self.join_params.batch_size { + return true; + } + + if curs.0.num_buffered_batches() + curs.1.num_buffered_batches() >= 6 + && curs.0.mem_size() + curs.1.mem_size() > suggested_output_batch_mem_size() + { + if let Some(first_idx) = self.indices.first() { + let cur_idx = match P.join_side { + L => curs.0.cur_idx, + R => curs.1.cur_idx, + }; + if first_idx.0 < cur_idx.0 { + return true; + } + } + } + false + } + + async fn flush(mut self: Pin<&mut Self>, curs: &mut StreamCursors) -> Result<()> { + let indices = std::mem::take(&mut self.indices); + let num_rows = indices.len(); + + let cols = match P.join_side { + L => interleave_batches( + curs.0.projected_batch_schema.clone(), + &curs.0.projected_batches, + &indices, + )?, + R => interleave_batches( + curs.1.projected_batch_schema.clone(), + &curs.1.projected_batches, + &indices, + )?, + }; + let output_batch = RecordBatch::try_new_with_options( + self.join_params.projection.schema.clone(), + cols.columns().to_vec(), + &RecordBatchOptions::new().with_row_count(Some(num_rows)), + )?; + + if output_batch.num_rows() > 0 { + self.output_rows += output_batch.num_rows(); + + let timer = self.send_output_time.timer(); + self.output_sender.send(Ok(output_batch), None).await; + drop(timer); + } + Ok(()) + } +} + +#[async_trait] +impl Joiner for SemiJoiner

{ + async fn join(mut self: Pin<&mut Self>, curs: &mut StreamCursors) -> Result<()> { + while !curs.0.finished && !curs.1.finished { + let mut lidx = curs.0.cur_idx; + let mut ridx = curs.1.cur_idx; + + match compare_cursor!(curs) { + Ordering::Less => { + if P.join_side == L && !P.semi { + self.indices.push(lidx); + } + cur_forward!(curs.0); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.0.set_min_reserved_idx(match P.join_side { + L => *self.indices.first().unwrap_or(&lidx), + R => lidx, + }); + } + Ordering::Greater => { + if P.join_side == R && !P.semi { + self.indices.push(ridx); + } + cur_forward!(curs.1); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.1.set_min_reserved_idx(match P.join_side { + L => ridx, + R => *self.indices.first().unwrap_or(&ridx), + }); + } + Ordering::Equal => { + // output/skip left equal rows + loop { + if P.join_side == L && P.semi { + self.indices.push(lidx); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + } + cur_forward!(curs.0); + curs.0.set_min_reserved_idx(match P.join_side { + L => *self.indices.first().unwrap_or(&lidx), + R => lidx, + }); + + if !curs.0.finished && curs.0.key(curs.0.cur_idx) == curs.0.key(lidx) { + lidx = curs.0.cur_idx; + continue; + } + break; + } + + // output/skip right equal rows + loop { + if P.join_side == R && P.semi { + self.indices.push(ridx); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + } + cur_forward!(curs.1); + curs.1.set_min_reserved_idx(match P.join_side { + L => ridx, + R => *self.indices.first().unwrap_or(&ridx), + }); + + if !curs.1.finished && curs.1.key(curs.1.cur_idx) == curs.1.key(ridx) { + ridx = curs.1.cur_idx; + continue; + } + break; + } + } + } + } + + // at least one side is finished, consume the other side if it is an anti side + if !P.semi { + while P.join_side == L && !P.semi && !curs.0.finished { + let lidx = curs.0.cur_idx; + self.indices.push(lidx); + cur_forward!(curs.0); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.0.set_min_reserved_idx(match P.join_side { + L => *self.indices.first().unwrap_or(&lidx), + R => lidx, + }); + } + while P.join_side == R && !P.semi && !curs.1.finished { + let ridx = curs.1.cur_idx; + self.indices.push(ridx); + cur_forward!(curs.1); + if self.should_flush(curs) { + self.as_mut().flush(curs).await?; + } + curs.1.set_min_reserved_idx(match P.join_side { + L => ridx, + R => *self.indices.first().unwrap_or(&ridx), + }); + } + } + if !self.indices.is_empty() { + self.flush(curs).await?; + } + Ok(()) + } + + fn total_send_output_time(&self) -> usize { + self.send_output_time.value() + } + + fn num_output_rows(&self) -> usize { + self.output_rows + } +} diff --git a/native-engine/datafusion-ext-plans/src/joins/stream_cursor.rs b/native-engine/datafusion-ext-plans/src/joins/stream_cursor.rs new file mode 100644 index 000000000..c105bb8d7 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/stream_cursor.rs @@ -0,0 +1,235 @@ +// Copyright 2022 The Blaze Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use arrow::{ + array::{RecordBatch, RecordBatchOptions}, + buffer::NullBuffer, + datatypes::{Schema, SchemaRef}, + row::{Row, RowConverter, Rows, SortField}, +}; +use datafusion::{ + common::{JoinSide, Result}, + execution::SendableRecordBatchStream, + physical_expr::PhysicalExprRef, + physical_plan::metrics::Time, +}; +use datafusion_ext_commons::array_size::ArraySize; +use futures::{Future, StreamExt}; +use parking_lot::Mutex; + +use crate::{ + common::batch_selection::take_batch_opt, + joins::{Idx, JoinParams}, +}; + +pub struct StreamCursor { + stream: SendableRecordBatchStream, + key_converter: Arc>, + key_exprs: Vec, + poll_time: Time, + + // IMPORTANT: + // batches/rows/null_buffers always contains a `null batch` in the front + projection: Vec, + pub projected_batch_schema: SchemaRef, + pub projected_batches: Vec, + pub cur_idx: Idx, + min_reserved_idx: Idx, + keys: Vec>, + key_has_nulls: Vec>, + num_null_batches: usize, + mem_size: usize, + pub finished: bool, +} + +impl StreamCursor { + pub fn try_new( + stream: SendableRecordBatchStream, + join_params: &JoinParams, + join_side: JoinSide, + projection: &[usize], + ) -> Result { + let key_converter = Arc::new(Mutex::new(RowConverter::new( + join_params + .key_data_types + .iter() + .cloned() + .zip(&join_params.sort_options) + .map(|(dt, options)| SortField::new_with_options(dt, *options)) + .collect(), + )?)); + let key_exprs = match join_side { + JoinSide::Left => join_params.left_keys.clone(), + JoinSide::Right => join_params.right_keys.clone(), + }; + + let empty_batch = RecordBatch::new_empty(Arc::new(Schema::new( + stream + .schema() + .fields() + .iter() + .map(|f| f.as_ref().clone().with_nullable(true)) + .collect::>(), + ))); + let empty_keys = Arc::new( + key_converter.lock().convert_columns( + &key_exprs + .iter() + .map(|key| Ok(key.evaluate(&empty_batch)?.into_array(0)?)) + .collect::>>()?, + )?, + ); + let null_batch = take_batch_opt(empty_batch, [Option::::None])?; + let projected_null_batch = null_batch.project(projection)?; + let null_nb = NullBuffer::new_null(1); + + Ok(Self { + stream, + key_exprs, + key_converter, + poll_time: Time::new(), + projection: projection.to_vec(), + projected_batch_schema: projected_null_batch.schema(), + projected_batches: vec![projected_null_batch], + cur_idx: (0, 0), + min_reserved_idx: (0, 0), + keys: vec![empty_keys], + key_has_nulls: vec![Some(null_nb)], + num_null_batches: 1, + mem_size: 0, + finished: false, + }) + } + + pub fn next(&mut self) -> Option> + '_> { + self.cur_idx.1 += 1; + if self.cur_idx.1 >= self.projected_batches[self.cur_idx.0].num_rows() { + self.cur_idx.0 += 1; + self.cur_idx.1 = 0; + } + + let should_load_next_batch = self.cur_idx.0 >= self.projected_batches.len(); + if should_load_next_batch { + Some(async move { + while let Some(batch) = { + let timer = self.poll_time.timer(); + let batch = self.stream.next().await.transpose()?; + drop(timer); + batch + } { + if batch.num_rows() == 0 { + continue; + } + let key_columns = self + .key_exprs + .iter() + .map(|key| Ok(key.evaluate(&batch)?.into_array(batch.num_rows())?)) + .collect::>>()?; + let key_has_nulls = key_columns + .iter() + .map(|c| c.nulls().cloned()) + .reduce(|lhs, rhs| NullBuffer::union(lhs.as_ref(), rhs.as_ref())) + .unwrap_or(None); + let keys = Arc::new(self.key_converter.lock().convert_columns(&key_columns)?); + + self.mem_size += batch.get_array_mem_size(); + self.mem_size += key_has_nulls + .as_ref() + .map(|nb| nb.buffer().len()) + .unwrap_or_default(); + self.mem_size += keys.size(); + + self.projected_batches + .push(RecordBatch::try_new_with_options( + self.projected_batches[0].schema(), + self.projection + .iter() + .map(|&i| batch.column(i).clone()) + .collect(), + &RecordBatchOptions::new().with_row_count(Some(batch.num_rows())), + )?); + self.key_has_nulls.push(key_has_nulls); + self.keys.push(keys); + + // fill out-dated batches with null batches + if self.num_null_batches < self.min_reserved_idx.0 { + for i in self.num_null_batches..self.min_reserved_idx.0 { + self.mem_size -= self.projected_batches[i].get_array_mem_size(); + self.mem_size -= self.key_has_nulls[i] + .as_ref() + .map(|nb| nb.buffer().len()) + .unwrap_or_default(); + self.mem_size -= self.keys[i].size(); + + self.projected_batches[i] = self.projected_batches[0].clone(); + self.keys[i] = self.keys[0].clone(); + self.key_has_nulls[i] = self.key_has_nulls[0].clone(); + self.num_null_batches += 1; + } + } + return Ok(()); + } + self.finished = true; + return Ok(()); + }) + } else { + None + } + } + + #[inline] + pub fn is_null_key(&self, idx: Idx) -> bool { + self.key_has_nulls[idx.0] + .as_ref() + .map(|nb| nb.is_null(idx.1)) + .unwrap_or(false) + } + + #[inline] + pub fn key<'a>(&'a self, idx: Idx) -> Row<'a> { + let keys = &self.keys[idx.0]; + keys.row(idx.1) + } + + #[inline] + pub fn num_buffered_batches(&self) -> usize { + self.projected_batches.len() - self.num_null_batches + } + + #[inline] + pub fn mem_size(&self) -> usize { + self.mem_size + } + + #[inline] + pub fn set_min_reserved_idx(&mut self, idx: Idx) { + self.min_reserved_idx = idx; + } + + #[inline] + pub fn total_poll_time(&self) -> usize { + self.poll_time.value() + } +} + +#[macro_export] +macro_rules! cur_forward { + ($cur:expr) => {{ + if let Some(fut) = $cur.next() { + fut.await?; + } + }}; +} diff --git a/native-engine/datafusion-ext-plans/src/joins/test.rs b/native-engine/datafusion-ext-plans/src/joins/test.rs new file mode 100644 index 000000000..e0826e7d0 --- /dev/null +++ b/native-engine/datafusion-ext-plans/src/joins/test.rs @@ -0,0 +1,947 @@ +// Copyright 2022 The Blaze Authors +// +// 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. + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use arrow::{ + self, + array::*, + compute::SortOptions, + datatypes::{DataType, Field, Schema, SchemaRef}, + record_batch::RecordBatch, + }; + use datafusion::{ + assert_batches_sorted_eq, + common::JoinSide, + error::Result, + physical_expr::expressions::Column, + physical_plan::{common, joins::utils::*, memory::MemoryExec, ExecutionPlan}, + prelude::SessionContext, + }; + use TestType::*; + + use crate::{ + broadcast_join_build_hash_map_exec::BroadcastJoinBuildHashMapExec, + broadcast_join_exec::BroadcastJoinExec, + joins::join_utils::{JoinType, JoinType::*}, + sort_merge_join_exec::SortMergeJoinExec, + }; + + #[derive(Clone, Copy)] + enum TestType { + SMJ, + BHJLeftProbed, + BHJRightProbed, + } + + fn columns(schema: &Schema) -> Vec { + schema.fields().iter().map(|f| f.name().clone()).collect() + } + + fn build_table_i32( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), + ) -> RecordBatch { + let schema = Schema::new(vec![ + Field::new(a.0, DataType::Int32, false), + Field::new(b.0, DataType::Int32, false), + Field::new(c.0, DataType::Int32, false), + ]); + + RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Int32Array::from(a.1.clone())), + Arc::new(Int32Array::from(b.1.clone())), + Arc::new(Int32Array::from(c.1.clone())), + ], + ) + .unwrap() + } + + fn build_table( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), + ) -> Arc { + let batch = build_table_i32(a, b, c); + let schema = batch.schema(); + Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + } + + fn build_table_from_batches(batches: Vec) -> Arc { + let schema = batches.first().unwrap().schema(); + Arc::new(MemoryExec::try_new(&[batches], schema, None).unwrap()) + } + + fn build_date_table( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), + ) -> Arc { + let schema = Schema::new(vec![ + Field::new(a.0, DataType::Date32, false), + Field::new(b.0, DataType::Date32, false), + Field::new(c.0, DataType::Date32, false), + ]); + + let batch = RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Date32Array::from(a.1.clone())), + Arc::new(Date32Array::from(b.1.clone())), + Arc::new(Date32Array::from(c.1.clone())), + ], + ) + .unwrap(); + + let schema = batch.schema(); + Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + } + + fn build_date64_table( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), + ) -> Arc { + let schema = Schema::new(vec![ + Field::new(a.0, DataType::Date64, false), + Field::new(b.0, DataType::Date64, false), + Field::new(c.0, DataType::Date64, false), + ]); + + let batch = RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Date64Array::from(a.1.clone())), + Arc::new(Date64Array::from(b.1.clone())), + Arc::new(Date64Array::from(c.1.clone())), + ], + ) + .unwrap(); + + let schema = batch.schema(); + Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + } + + /// returns a table with 3 columns of i32 in memory + pub fn build_table_i32_nullable( + a: (&str, &Vec>), + b: (&str, &Vec>), + c: (&str, &Vec>), + ) -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new(a.0, DataType::Int32, true), + Field::new(b.0, DataType::Int32, true), + Field::new(c.0, DataType::Int32, true), + ])); + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(Int32Array::from(a.1.clone())), + Arc::new(Int32Array::from(b.1.clone())), + Arc::new(Int32Array::from(c.1.clone())), + ], + ) + .unwrap(); + Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + } + + fn build_join_schema_for_test( + left: &Schema, + right: &Schema, + join_type: JoinType, + ) -> Result { + if join_type == Existence { + let exists_field = Arc::new(Field::new("exists#0", DataType::Boolean, false)); + return Ok(Arc::new(Schema::new( + [left.fields().to_vec(), vec![exists_field]].concat(), + ))); + } + Ok(Arc::new( + build_join_schema(left, right, &join_type.try_into()?).0, + )) + } + + async fn join_collect( + test_type: TestType, + left: Arc, + right: Arc, + on: JoinOn, + join_type: JoinType, + ) -> Result<(Vec, Vec)> { + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + let schema = build_join_schema_for_test(&left.schema(), &right.schema(), join_type)?; + + let join: Arc = match test_type { + SMJ => { + let sort_options = vec![SortOptions::default(); on.len()]; + Arc::new(SortMergeJoinExec::try_new( + schema, + left, + right, + on, + join_type, + sort_options, + )?) + } + BHJLeftProbed => { + let right = Arc::new(BroadcastJoinBuildHashMapExec::new( + right, + on.iter().map(|(_, right_key)| right_key.clone()).collect(), + )); + Arc::new(BroadcastJoinExec::try_new( + schema, + left, + right, + on, + join_type, + JoinSide::Right, + None, + )?) + } + BHJRightProbed => { + let left = Arc::new(BroadcastJoinBuildHashMapExec::new( + left, + on.iter().map(|(left_key, _)| left_key.clone()).collect(), + )); + Arc::new(BroadcastJoinExec::try_new( + schema, + left, + right, + on, + join_type, + JoinSide::Left, + None, + )?) + } + }; + let columns = columns(&join.schema()); + let stream = join.execute(0, task_ctx)?; + let batches = common::collect(stream).await?; + Ok((columns, batches)) + } + + #[tokio::test] + async fn join_inner_one() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![1, 2, 3]), + ("b1", &vec![4, 5, 5]), // this has a repetition + ("c1", &vec![7, 8, 9]), + ); + let right = build_table( + ("a2", &vec![10, 20, 30]), + ("b1", &vec![4, 5, 6]), + ("c2", &vec![70, 80, 90]), + ); + + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b1", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Inner).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b1 | c2 |", + "+----+----+----+----+----+----+", + "| 1 | 4 | 7 | 10 | 4 | 70 |", + "| 2 | 5 | 8 | 20 | 5 | 80 |", + "| 3 | 5 | 9 | 20 | 5 | 80 |", + "+----+----+----+----+----+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_inner_two() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![1, 2, 2]), + ("b2", &vec![1, 2, 2]), + ("c1", &vec![7, 8, 9]), + ); + let right = build_table( + ("a1", &vec![1, 2, 3]), + ("b2", &vec![1, 2, 2]), + ("c2", &vec![70, 80, 90]), + ); + let on: JoinOn = vec![ + ( + Arc::new(Column::new_with_schema("a1", &left.schema())?), + Arc::new(Column::new_with_schema("a1", &right.schema())?), + ), + ( + Arc::new(Column::new_with_schema("b2", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + ), + ]; + + let (_columns, batches) = join_collect(test_type, left, right, on, Inner).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b2 | c1 | a1 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| 1 | 1 | 7 | 1 | 1 | 70 |", + "| 2 | 2 | 8 | 2 | 2 | 80 |", + "| 2 | 2 | 9 | 2 | 2 | 80 |", + "+----+----+----+----+----+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_inner_two_two() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![1, 1, 2]), + ("b2", &vec![1, 1, 2]), + ("c1", &vec![7, 8, 9]), + ); + let right = build_table( + ("a1", &vec![1, 1, 3]), + ("b2", &vec![1, 1, 2]), + ("c2", &vec![70, 80, 90]), + ); + let on: JoinOn = vec![ + ( + Arc::new(Column::new_with_schema("a1", &left.schema())?), + Arc::new(Column::new_with_schema("a1", &right.schema())?), + ), + ( + Arc::new(Column::new_with_schema("b2", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + ), + ]; + + let (_columns, batches) = join_collect(test_type, left, right, on, Inner).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b2 | c1 | a1 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| 1 | 1 | 7 | 1 | 1 | 70 |", + "| 1 | 1 | 7 | 1 | 1 | 80 |", + "| 1 | 1 | 8 | 1 | 1 | 70 |", + "| 1 | 1 | 8 | 1 | 1 | 80 |", + "+----+----+----+----+----+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_inner_with_nulls() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table_i32_nullable( + ("a1", &vec![Some(1), Some(1), Some(2), Some(2)]), + ("b2", &vec![None, Some(1), Some(2), Some(2)]), // null in key field + ("c1", &vec![Some(1), None, Some(8), Some(9)]), // null in non-key field + ); + let right = build_table_i32_nullable( + ("a1", &vec![Some(1), Some(1), Some(2), Some(3)]), + ("b2", &vec![None, Some(1), Some(2), Some(2)]), + ("c2", &vec![Some(10), Some(70), Some(80), Some(90)]), + ); + let on: JoinOn = vec![ + ( + Arc::new(Column::new_with_schema("a1", &left.schema())?), + Arc::new(Column::new_with_schema("a1", &right.schema())?), + ), + ( + Arc::new(Column::new_with_schema("b2", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + ), + ]; + + let (_, batches) = join_collect(test_type, left, right, on, Inner).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b2 | c1 | a1 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| 1 | 1 | | 1 | 1 | 70 |", + "| 2 | 2 | 8 | 2 | 2 | 80 |", + "| 2 | 2 | 9 | 2 | 2 | 80 |", + "+----+----+----+----+----+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_left_one() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![1, 2, 3]), + ("b1", &vec![4, 5, 7]), // 7 does not exist on the right + ("c1", &vec![7, 8, 9]), + ); + let right = build_table( + ("a2", &vec![10, 20, 30]), + ("b1", &vec![4, 5, 6]), + ("c2", &vec![70, 80, 90]), + ); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b1", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Left).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b1 | c2 |", + "+----+----+----+----+----+----+", + "| 1 | 4 | 7 | 10 | 4 | 70 |", + "| 2 | 5 | 8 | 20 | 5 | 80 |", + "| 3 | 7 | 9 | | | |", + "+----+----+----+----+----+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_right_one() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![1, 2, 3]), + ("b1", &vec![4, 5, 7]), + ("c1", &vec![7, 8, 9]), + ); + let right = build_table( + ("a2", &vec![10, 20, 30]), + ("b1", &vec![4, 5, 6]), // 6 does not exist on the left + ("c2", &vec![70, 80, 90]), + ); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b1", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Right).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b1 | c2 |", + "+----+----+----+----+----+----+", + "| 1 | 4 | 7 | 10 | 4 | 70 |", + "| 2 | 5 | 8 | 20 | 5 | 80 |", + "| | | | 30 | 6 | 90 |", + "+----+----+----+----+----+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_full_one() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![1, 2, 3]), + ("b1", &vec![4, 5, 7]), // 7 does not exist on the right + ("c1", &vec![7, 8, 9]), + ); + let right = build_table( + ("a2", &vec![10, 20, 30]), + ("b2", &vec![4, 5, 6]), + ("c2", &vec![70, 80, 90]), + ); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Full).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| | | | 30 | 6 | 90 |", + "| 1 | 4 | 7 | 10 | 4 | 70 |", + "| 2 | 5 | 8 | 20 | 5 | 80 |", + "| 3 | 7 | 9 | | | |", + "+----+----+----+----+----+----+", + ]; + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_anti() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![1, 2, 2, 3, 5]), + ("b1", &vec![4, 5, 5, 7, 7]), // 7 does not exist on the right + ("c1", &vec![7, 8, 8, 9, 11]), + ); + let right = build_table( + ("a2", &vec![10, 20, 30]), + ("b1", &vec![4, 5, 6]), + ("c2", &vec![70, 80, 90]), + ); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b1", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, LeftAnti).await?; + let expected = vec![ + "+----+----+----+", + "| a1 | b1 | c1 |", + "+----+----+----+", + "| 3 | 7 | 9 |", + "| 5 | 7 | 11 |", + "+----+----+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_semi() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![1, 2, 2, 3]), + ("b1", &vec![4, 5, 5, 7]), // 7 does not exist on the right + ("c1", &vec![7, 8, 8, 9]), + ); + let right = build_table( + ("a2", &vec![10, 20, 30]), + ("b1", &vec![4, 5, 6]), // 5 is double on the right + ("c2", &vec![70, 80, 90]), + ); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b1", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, LeftSemi).await?; + let expected = vec![ + "+----+----+----+", + "| a1 | b1 | c1 |", + "+----+----+----+", + "| 1 | 4 | 7 |", + "| 2 | 5 | 8 |", + "| 2 | 5 | 8 |", + "+----+----+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_with_duplicated_column_names() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a", &vec![1, 2, 3]), + ("b", &vec![4, 5, 7]), + ("c", &vec![7, 8, 9]), + ); + let right = build_table( + ("a", &vec![10, 20, 30]), + ("b", &vec![1, 2, 7]), + ("c", &vec![70, 80, 90]), + ); + let on: JoinOn = vec![( + // join on a=b so there are duplicate column names on unjoined columns + Arc::new(Column::new_with_schema("a", &left.schema())?), + Arc::new(Column::new_with_schema("b", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Inner).await?; + let expected = vec![ + "+---+---+---+----+---+----+", + "| a | b | c | a | b | c |", + "+---+---+---+----+---+----+", + "| 1 | 4 | 7 | 10 | 1 | 70 |", + "| 2 | 5 | 8 | 20 | 2 | 80 |", + "+---+---+---+----+---+----+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_date32() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_date_table( + ("a1", &vec![1, 2, 3]), + ("b1", &vec![19107, 19108, 19108]), // this has a repetition + ("c1", &vec![7, 8, 9]), + ); + let right = build_date_table( + ("a2", &vec![10, 20, 30]), + ("b1", &vec![19107, 19108, 19109]), + ("c2", &vec![70, 80, 90]), + ); + + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b1", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Inner).await?; + + let expected = vec![ + "+------------+------------+------------+------------+------------+------------+", + "| a1 | b1 | c1 | a2 | b1 | c2 |", + "+------------+------------+------------+------------+------------+------------+", + "| 1970-01-02 | 2022-04-25 | 1970-01-08 | 1970-01-11 | 2022-04-25 | 1970-03-12 |", + "| 1970-01-03 | 2022-04-26 | 1970-01-09 | 1970-01-21 | 2022-04-26 | 1970-03-22 |", + "| 1970-01-04 | 2022-04-26 | 1970-01-10 | 1970-01-21 | 2022-04-26 | 1970-03-22 |", + "+------------+------------+------------+------------+------------+------------+", + ]; + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_date64() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_date64_table( + ("a1", &vec![1, 2, 3]), + ("b1", &vec![1650703441000, 1650903441000, 1650903441000]), /* this has a + * repetition */ + ("c1", &vec![7, 8, 9]), + ); + let right = build_date64_table( + ("a2", &vec![10, 20, 30]), + ("b1", &vec![1650703441000, 1650503441000, 1650903441000]), + ("c2", &vec![70, 80, 90]), + ); + + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b1", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Inner).await?; + let expected = vec![ + "+-------------------------+---------------------+-------------------------+-------------------------+---------------------+-------------------------+", + "| a1 | b1 | c1 | a2 | b1 | c2 |", + "+-------------------------+---------------------+-------------------------+-------------------------+---------------------+-------------------------+", + "| 1970-01-01T00:00:00.001 | 2022-04-23T08:44:01 | 1970-01-01T00:00:00.007 | 1970-01-01T00:00:00.010 | 2022-04-23T08:44:01 | 1970-01-01T00:00:00.070 |", + "| 1970-01-01T00:00:00.002 | 2022-04-25T16:17:21 | 1970-01-01T00:00:00.008 | 1970-01-01T00:00:00.030 | 2022-04-25T16:17:21 | 1970-01-01T00:00:00.090 |", + "| 1970-01-01T00:00:00.003 | 2022-04-25T16:17:21 | 1970-01-01T00:00:00.009 | 1970-01-01T00:00:00.030 | 2022-04-25T16:17:21 | 1970-01-01T00:00:00.090 |", + "+-------------------------+---------------------+-------------------------+-------------------------+---------------------+-------------------------+", + ]; + + // The output order is important as SMJ preserves sortedness + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_left_sort_order() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![0, 1, 2, 3, 4, 5]), + ("b1", &vec![3, 4, 5, 6, 6, 7]), + ("c1", &vec![4, 5, 6, 7, 8, 9]), + ); + let right = build_table( + ("a2", &vec![0, 10, 20, 30, 40]), + ("b2", &vec![2, 4, 6, 6, 8]), + ("c2", &vec![50, 60, 70, 80, 90]), + ); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Left).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| 0 | 3 | 4 | | | |", + "| 1 | 4 | 5 | 10 | 4 | 60 |", + "| 2 | 5 | 6 | | | |", + "| 3 | 6 | 7 | 20 | 6 | 70 |", + "| 3 | 6 | 7 | 30 | 6 | 80 |", + "| 4 | 6 | 8 | 20 | 6 | 70 |", + "| 4 | 6 | 8 | 30 | 6 | 80 |", + "| 5 | 7 | 9 | | | |", + "+----+----+----+----+----+----+", + ]; + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_right_sort_order() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left = build_table( + ("a1", &vec![0, 1, 2, 3]), + ("b1", &vec![3, 4, 5, 7]), + ("c1", &vec![6, 7, 8, 9]), + ); + let right = build_table( + ("a2", &vec![0, 10, 20, 30]), + ("b2", &vec![2, 4, 5, 6]), + ("c2", &vec![60, 70, 80, 90]), + ); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Right).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| | | | 0 | 2 | 60 |", + "| 1 | 4 | 7 | 10 | 4 | 70 |", + "| 2 | 5 | 8 | 20 | 5 | 80 |", + "| | | | 30 | 6 | 90 |", + "+----+----+----+----+----+----+", + ]; + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_left_multiple_batches() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left_batch_1 = build_table_i32( + ("a1", &vec![0, 1, 2]), + ("b1", &vec![3, 4, 5]), + ("c1", &vec![4, 5, 6]), + ); + let left_batch_2 = build_table_i32( + ("a1", &vec![3, 4, 5, 6]), + ("b1", &vec![6, 6, 7, 9]), + ("c1", &vec![7, 8, 9, 9]), + ); + let right_batch_1 = build_table_i32( + ("a2", &vec![0, 10, 20]), + ("b2", &vec![2, 4, 6]), + ("c2", &vec![50, 60, 70]), + ); + let right_batch_2 = build_table_i32( + ("a2", &vec![30, 40]), + ("b2", &vec![6, 8]), + ("c2", &vec![80, 90]), + ); + let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); + let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Left).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| 0 | 3 | 4 | | | |", + "| 1 | 4 | 5 | 10 | 4 | 60 |", + "| 2 | 5 | 6 | | | |", + "| 3 | 6 | 7 | 20 | 6 | 70 |", + "| 3 | 6 | 7 | 30 | 6 | 80 |", + "| 4 | 6 | 8 | 20 | 6 | 70 |", + "| 4 | 6 | 8 | 30 | 6 | 80 |", + "| 5 | 7 | 9 | | | |", + "| 6 | 9 | 9 | | | |", + "+----+----+----+----+----+----+", + ]; + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_right_multiple_batches() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let right_batch_1 = build_table_i32( + ("a2", &vec![0, 1, 2]), + ("b2", &vec![3, 4, 5]), + ("c2", &vec![4, 5, 6]), + ); + let right_batch_2 = build_table_i32( + ("a2", &vec![3, 4, 5, 6]), + ("b2", &vec![6, 6, 7, 9]), + ("c2", &vec![7, 8, 9, 9]), + ); + let left_batch_1 = build_table_i32( + ("a1", &vec![0, 10, 20]), + ("b1", &vec![2, 4, 6]), + ("c1", &vec![50, 60, 70]), + ); + let left_batch_2 = build_table_i32( + ("a1", &vec![30, 40]), + ("b1", &vec![6, 8]), + ("c1", &vec![80, 90]), + ); + let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); + let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Right).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| | | | 0 | 3 | 4 |", + "| 10 | 4 | 60 | 1 | 4 | 5 |", + "| | | | 2 | 5 | 6 |", + "| 20 | 6 | 70 | 3 | 6 | 7 |", + "| 30 | 6 | 80 | 3 | 6 | 7 |", + "| 20 | 6 | 70 | 4 | 6 | 8 |", + "| 30 | 6 | 80 | 4 | 6 | 8 |", + "| | | | 5 | 7 | 9 |", + "| | | | 6 | 9 | 9 |", + "+----+----+----+----+----+----+", + ]; + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_full_multiple_batches() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left_batch_1 = build_table_i32( + ("a1", &vec![0, 1, 2]), + ("b1", &vec![3, 4, 5]), + ("c1", &vec![4, 5, 6]), + ); + let left_batch_2 = build_table_i32( + ("a1", &vec![3, 4, 5, 6]), + ("b1", &vec![6, 6, 7, 9]), + ("c1", &vec![7, 8, 9, 9]), + ); + let right_batch_1 = build_table_i32( + ("a2", &vec![0, 10, 20]), + ("b2", &vec![2, 4, 6]), + ("c2", &vec![50, 60, 70]), + ); + let right_batch_2 = build_table_i32( + ("a2", &vec![30, 40]), + ("b2", &vec![6, 8]), + ("c2", &vec![80, 90]), + ); + let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); + let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Full).await?; + let expected = vec![ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| | | | 0 | 2 | 50 |", + "| | | | 40 | 8 | 90 |", + "| 0 | 3 | 4 | | | |", + "| 1 | 4 | 5 | 10 | 4 | 60 |", + "| 2 | 5 | 6 | | | |", + "| 3 | 6 | 7 | 20 | 6 | 70 |", + "| 3 | 6 | 7 | 30 | 6 | 80 |", + "| 4 | 6 | 8 | 20 | 6 | 70 |", + "| 4 | 6 | 8 | 30 | 6 | 80 |", + "| 5 | 7 | 9 | | | |", + "| 6 | 9 | 9 | | | |", + "+----+----+----+----+----+----+", + ]; + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } + + #[tokio::test] + async fn join_existence_multiple_batches() -> Result<()> { + for test_type in [SMJ, BHJLeftProbed, BHJRightProbed] { + let left_batch_1 = build_table_i32( + ("a1", &vec![0, 1, 2]), + ("b1", &vec![3, 4, 5]), + ("c1", &vec![4, 5, 6]), + ); + let left_batch_2 = build_table_i32( + ("a1", &vec![3, 4, 5, 6]), + ("b1", &vec![6, 6, 7, 9]), + ("c1", &vec![7, 8, 9, 9]), + ); + let right_batch_1 = build_table_i32( + ("a2", &vec![0, 10, 20]), + ("b2", &vec![2, 4, 6]), + ("c2", &vec![50, 60, 70]), + ); + let right_batch_2 = build_table_i32( + ("a2", &vec![30, 40]), + ("b2", &vec![6, 8]), + ("c2", &vec![80, 90]), + ); + let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); + let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); + let on: JoinOn = vec![( + Arc::new(Column::new_with_schema("b1", &left.schema())?), + Arc::new(Column::new_with_schema("b2", &right.schema())?), + )]; + + let (_, batches) = join_collect(test_type, left, right, on, Existence).await?; + let expected = vec![ + "+----+----+----+----------+", + "| a1 | b1 | c1 | exists#0 |", + "+----+----+----+----------+", + "| 0 | 3 | 4 | false |", + "| 1 | 4 | 5 | true |", + "| 2 | 5 | 6 | false |", + "| 3 | 6 | 7 | true |", + "| 4 | 6 | 8 | true |", + "| 5 | 7 | 9 | false |", + "| 6 | 9 | 9 | false |", + "+----+----+----+----------+", + ]; + assert_batches_sorted_eq!(expected, &batches); + } + Ok(()) + } +} diff --git a/native-engine/datafusion-ext-plans/src/lib.rs b/native-engine/datafusion-ext-plans/src/lib.rs index a0797fb09..a48fb56a2 100644 --- a/native-engine/datafusion-ext-plans/src/lib.rs +++ b/native-engine/datafusion-ext-plans/src/lib.rs @@ -13,32 +13,38 @@ // limitations under the License. #![feature(get_mut_unchecked)] -#![feature(io_error_other)] +#![feature(adt_const_params)] -pub mod agg; +// execution plan implementations pub mod agg_exec; +pub mod broadcast_join_build_hash_map_exec; pub mod broadcast_join_exec; -pub mod broadcast_nested_loop_join_exec; -pub mod common; pub mod debug_exec; pub mod empty_partitions_exec; pub mod expand_exec; pub mod ffi_reader_exec; pub mod filter_exec; -pub mod generate; pub mod generate_exec; pub mod ipc_reader_exec; pub mod ipc_writer_exec; pub mod limit_exec; -pub mod memmgr; pub mod parquet_exec; pub mod parquet_sink_exec; pub mod project_exec; pub mod rename_columns_exec; pub mod rss_shuffle_writer_exec; -mod shuffle; pub mod shuffle_writer_exec; pub mod sort_exec; pub mod sort_merge_join_exec; -pub mod window; pub mod window_exec; + +// memory management +pub mod memmgr; + +// helper modules +pub mod agg; +pub mod common; +pub mod generate; +pub mod joins; +mod shuffle; +pub mod window; diff --git a/native-engine/datafusion-ext-plans/src/parquet_exec.rs b/native-engine/datafusion-ext-plans/src/parquet_exec.rs index 8fd5f57f6..f7c206273 100644 --- a/native-engine/datafusion-ext-plans/src/parquet_exec.rs +++ b/native-engine/datafusion-ext-plans/src/parquet_exec.rs @@ -20,7 +20,7 @@ use std::{any::Any, fmt, fmt::Formatter, ops::Range, sync::Arc}; use arrow::{ - array::ArrayRef, + array::{Array, ArrayRef, AsArray, ListArray}, datatypes::{DataType, SchemaRef}, }; use base64::{prelude::BASE64_URL_SAFE_NO_PAD, Engine}; @@ -56,7 +56,6 @@ use datafusion::{ use datafusion_ext_commons::{ batch_size, df_execution_err, hadoop_fs::{FsDataInputStream, FsProvider}, - streams::coalesce_stream::CoalesceInput, }; use fmt::Debug; use futures::{future::BoxFuture, stream::once, FutureExt, StreamExt, TryStreamExt}; @@ -71,7 +70,61 @@ fn schema_adapter_cast_column( col: &ArrayRef, data_type: &DataType, ) -> Result { - datafusion_ext_commons::cast::cast_scan_input_array(col.as_ref(), data_type) + macro_rules! handle_decimal { + ($s:ident, $t:ident, $tnative:ty, $prec:expr, $scale:expr) => {{ + use arrow::{array::*, datatypes::*}; + type DecimalBuilder = paste::paste! {[<$t Builder>]}; + type IntType = paste::paste! {[<$s Type>]}; + + let col = col.as_primitive::(); + let mut decimal_builder = DecimalBuilder::new(); + for i in 0..col.len() { + if col.is_valid(i) { + decimal_builder.append_value(col.value(i) as $tnative); + } else { + decimal_builder.append_null(); + } + } + Ok(Arc::new( + decimal_builder + .finish() + .with_precision_and_scale($prec, $scale)?, + )) + }}; + } + match data_type { + DataType::Decimal128(prec, scale) => match col.data_type() { + DataType::Int8 => handle_decimal!(Int8, Decimal128, i128, *prec, *scale), + DataType::Int16 => handle_decimal!(Int16, Decimal128, i128, *prec, *scale), + DataType::Int32 => handle_decimal!(Int32, Decimal128, i128, *prec, *scale), + DataType::Int64 => handle_decimal!(Int64, Decimal128, i128, *prec, *scale), + DataType::Decimal128(p, s) if p == prec && s == scale => Ok(col.clone()), + _ => df_execution_err!( + "schema_adapter_cast_column unsupported type: {:?} => {:?}", + col.data_type(), + data_type, + ), + }, + DataType::List(to_field) => match col.data_type() { + DataType::List(_from_field) => { + let col = col.as_list::(); + let from_inner = col.values(); + let to_inner = schema_adapter_cast_column(from_inner, to_field.data_type())?; + Ok(Arc::new(ListArray::try_new( + to_field.clone(), + col.offsets().clone(), + to_inner, + col.nulls().cloned(), + )?)) + } + _ => df_execution_err!( + "schema_adapter_cast_column unsupported type: {:?} => {:?}", + col.data_type(), + data_type, + ), + }, + _ => datafusion_ext_commons::cast::cast_scan_input_array(col.as_ref(), data_type), + } } /// Execution plan for scanning one or more Parquet partitions @@ -231,6 +284,9 @@ impl ExecutionPlan for ParquetExec { None => (0..self.base_config.file_schema.fields().len()).collect(), }; + let page_filtering_enabled = conf::PARQUET_ENABLE_PAGE_FILTERING.value()?; + let bloom_filter_enabled = conf::PARQUET_ENABLE_BLOOM_FILTER.value()?; + let opener = ParquetOpener { partition_index, projection: Arc::from(projection), @@ -243,10 +299,10 @@ impl ExecutionPlan for ParquetExec { metadata_size_hint: None, metrics: self.metrics.clone(), parquet_file_reader_factory: Arc::new(FsReaderFactory::new(fs_provider)), - pushdown_filters: false, - reorder_filters: false, - enable_page_index: false, - enable_bloom_filter: false, + pushdown_filters: page_filtering_enabled, + reorder_filters: page_filtering_enabled, + enable_page_index: page_filtering_enabled, + enable_bloom_filter: bloom_filter_enabled, }; let baseline_metrics_cloned = baseline_metrics.clone(); @@ -274,7 +330,7 @@ impl ExecutionPlan for ParquetExec { }) .try_flatten(), )); - context.coalesce_with_default_batch_size(timed_stream, &baseline_metrics) + Ok(timed_stream) } fn metrics(&self) -> Option { diff --git a/native-engine/datafusion-ext-plans/src/rename_columns_exec.rs b/native-engine/datafusion-ext-plans/src/rename_columns_exec.rs index f2dff1dbb..69b46cf77 100644 --- a/native-engine/datafusion-ext-plans/src/rename_columns_exec.rs +++ b/native-engine/datafusion-ext-plans/src/rename_columns_exec.rs @@ -35,7 +35,6 @@ use datafusion::{ SendableRecordBatchStream, Statistics, }, }; -use datafusion_ext_commons::df_execution_err; use futures::{Stream, StreamExt}; use crate::agg::AGG_BUF_COLUMN_NAME; @@ -56,7 +55,12 @@ impl RenameColumnsExec { let input_schema = input.schema(); let mut new_names = vec![]; - for (i, field) in input_schema.fields().iter().enumerate() { + for (i, field) in input_schema + .fields() + .iter() + .take(renamed_column_names.len()) + .enumerate() + { if field.name() != AGG_BUF_COLUMN_NAME { new_names.push(renamed_column_names[i].clone()); } else { @@ -64,11 +68,9 @@ impl RenameColumnsExec { break; } } - if new_names.len() != input_schema.fields().len() { - df_execution_err!( - "renamed_column_names length not matched with input schema, \ - renames: {renamed_column_names:?}, input schema: {input_schema}", - )?; + + while new_names.len() < input_schema.fields().len() { + new_names.push(input_schema.field(new_names.len()).name().clone()); } let renamed_column_names = new_names; let renamed_schema = Arc::new(Schema::new( diff --git a/native-engine/datafusion-ext-plans/src/sort_exec.rs b/native-engine/datafusion-ext-plans/src/sort_exec.rs index 33a2818e8..56a8ec6e8 100644 --- a/native-engine/datafusion-ext-plans/src/sort_exec.rs +++ b/native-engine/datafusion-ext-plans/src/sort_exec.rs @@ -49,7 +49,6 @@ use datafusion_ext_commons::{ downcast_any, ds::loser_tree::{ComparableForLoserTree, LoserTree}, io::{read_len, read_one_batch, write_len, write_one_batch}, - staging_mem_size_for_partial_sort, streams::coalesce_stream::CoalesceInput, }; use futures::{lock::Mutex, stream::once, StreamExt, TryStreamExt}; @@ -59,7 +58,7 @@ use parking_lot::Mutex as SyncMutex; use crate::{ common::{ - batch_selection::interleave_batches, + batch_selection::{interleave_batches, take_batch}, batch_statisitcs::{stat_input, InputBatchStatistics}, column_pruning::ExecuteWithColumnPruning, output::{TaskOutputter, WrappedRecordBatchSender}, @@ -242,11 +241,9 @@ impl MemConsumer for ExternalSorter { #[derive(Default)] struct BufferedData { - staging_batches: Vec, sorted_key_stores: Vec>, sorted_key_stores_mem_used: usize, sorted_batches: Vec, - staging_mem_used: usize, sorted_batches_mem_used: usize, num_rows: usize, } @@ -271,34 +268,15 @@ impl BufferedData { } fn mem_used(&self) -> usize { - self.staging_mem_used + self.sorted_batches_mem_used + self.sorted_key_stores_mem_used + self.sorted_batches_mem_used + self.sorted_key_stores_mem_used } fn add_batch(&mut self, batch: RecordBatch, sorter: &ExternalSorter) -> Result<()> { self.num_rows += batch.num_rows(); - self.staging_mem_used += batch.get_array_mem_size(); - self.staging_batches.push(batch); - if self.staging_mem_used >= staging_mem_size_for_partial_sort() { - self.flush_staging_batches(sorter)?; - } - Ok(()) - } - - fn flush_staging_batches(&mut self, sorter: &ExternalSorter) -> Result<()> { - let staging_batches = std::mem::take(&mut self.staging_batches); - self.staging_mem_used = 0; - - let schema = sorter.prune_sort_keys_from_batch.pruned_schema.clone(); - let (key_rows, batches): (Vec, Vec) = staging_batches - .into_iter() - .map(|batch| sorter.prune_sort_keys_from_batch.prune(batch)) - .collect::>>()? - .into_iter() - .unzip(); + let (key_rows, batch) = sorter.prune_sort_keys_from_batch.prune(batch)?; // sort the batch and append to sorter - let mut sorted_key_store = - Vec::with_capacity(key_rows.iter().map(|rows| rows.size()).sum::()); + let mut sorted_key_store = Vec::with_capacity(key_rows.size()); let mut key_writer = SortedKeysWriter::default(); let mut num_rows = 0; let sorted_batch; @@ -307,32 +285,28 @@ impl BufferedData { let cur_sorted_indices = key_rows .iter() .enumerate() - .flat_map(|(batch_idx, rows)| { - rows.iter() - .map(|key| unsafe { - // safety: keys have the same lifetime with key_rows - std::mem::transmute::<_, &'static [u8]>(key.as_ref()) - }) - .enumerate() - .map(move |(row_idx, key)| (key, batch_idx as u32, row_idx as u32)) + .map(|(row_idx, key)| { + let key = unsafe { + // safety: keys have the same lifetime with key_rows + std::mem::transmute::<_, &'static [u8]>(key.as_ref()) + }; + (key, row_idx as u32) }) .sorted_unstable_by_key(|&(key, ..)| key) .take(sorter.limit) - .map(|(key, batch_idx, row_idx)| { + .map(|(key, row_idx)| { num_rows += 1; key_writer.write_key(key, &mut sorted_key_store).unwrap(); - (batch_idx as usize, row_idx as usize) + row_idx as usize }) .collect::>(); - sorted_batch = interleave_batches(schema, &batches, &cur_sorted_indices)?; + sorted_batch = take_batch(batch, cur_sorted_indices)?; } else { key_rows .iter() - .flat_map(|rows| { - rows.iter().map(|key| unsafe { - // safety: keys have the same lifetime with key_rows - std::mem::transmute::<_, &'static [u8]>(key.as_ref()) - }) + .map(|key| unsafe { + // safety: keys have the same lifetime with key_rows + std::mem::transmute::<_, &'static [u8]>(key.as_ref()) }) .sorted_unstable() .take(sorter.limit) @@ -351,13 +325,10 @@ impl BufferedData { } fn into_sorted_batches<'a, KC: KeyCollector>( - mut self, + self, batch_size: usize, sorter: &ExternalSorter, ) -> Result> { - if !self.staging_batches.is_empty() { - self.flush_staging_batches(sorter)?; - } struct Cursor { idx: usize, row_idx: usize, diff --git a/native-engine/datafusion-ext-plans/src/sort_merge_join_exec.rs b/native-engine/datafusion-ext-plans/src/sort_merge_join_exec.rs index 8459d476d..d4e5cda37 100644 --- a/native-engine/datafusion-ext-plans/src/sort_merge_join_exec.rs +++ b/native-engine/datafusion-ext-plans/src/sort_merge_join_exec.rs @@ -12,135 +12,151 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::{any::Any, cmp::Ordering, fmt::Formatter, sync::Arc}; - -use arrow::{ - array::*, - buffer::NullBuffer, - compute::{prep_null_mask_filter, SortOptions}, - datatypes::{DataType, Schema, SchemaRef}, - record_batch::{RecordBatch, RecordBatchOptions}, - row::{Row, RowConverter, Rows, SortField}, +use std::{ + any::Any, + fmt::Formatter, + pin::Pin, + sync::Arc, + time::{Duration, Instant}, }; + +use arrow::{compute::SortOptions, datatypes::SchemaRef}; +use async_trait::async_trait; use datafusion::{ - common::JoinSide, + common::{DataFusionError, JoinSide}, error::Result, execution::context::TaskContext, - logical_expr::{JoinType, JoinType::*}, - physical_expr::{expressions::Column, PhysicalSortExpr}, + physical_expr::{PhysicalExprRef, PhysicalSortExpr}, physical_plan::{ - joins::utils::{build_join_schema, check_join_is_valid, ColumnIndex, JoinFilter, JoinOn}, - metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, ScopedTimerGuard}, + joins::utils::JoinOn, + metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, stream::RecordBatchStreamAdapter, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }, }; use datafusion_ext_commons::{ - array_size::ArraySize, batch_size, df_execution_err, downcast_any, - streams::coalesce_stream::CoalesceInput, suggested_output_batch_mem_size, + batch_size, df_execution_err, streams::coalesce_stream::CoalesceInput, }; -use futures::{StreamExt, TryStreamExt}; -use parking_lot::Mutex as SyncMutex; +use futures::TryStreamExt; -use crate::common::{ - batch_selection::{interleave_batches, take_batch_opt}, - column_pruning::ExecuteWithColumnPruning, - output::{TaskOutputter, WrappedRecordBatchSender}, +use crate::{ + common::{ + column_pruning::ExecuteWithColumnPruning, + output::{TaskOutputter, WrappedRecordBatchSender}, + }, + cur_forward, + joins::{ + join_utils::{JoinType, JoinType::*}, + smj::{ + existence_join::ExistenceJoiner, + full_join::{FullOuterJoiner, InnerJoiner, LeftOuterJoiner, RightOuterJoiner}, + semi_join::{LeftAntiJoiner, LeftSemiJoiner, RightAntiJoiner, RightSemiJoiner}, + }, + stream_cursor::StreamCursor, + JoinParams, JoinProjection, StreamCursors, + }, }; #[derive(Debug)] pub struct SortMergeJoinExec { - /// Left sorted joining execution plan left: Arc, - /// Right sorting joining execution plan right: Arc, - /// Set of common columns used to join on on: JoinOn, - /// How the join is performed join_type: JoinType, - /// Optional filter before outputting - join_filter: Option, - /// The schema once the join is applied + sort_options: Vec, schema: SchemaRef, - /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Sort options of join columns used in sorting left and right execution - /// plans - sort_options: Vec, } impl SortMergeJoinExec { pub fn try_new( + schema: SchemaRef, left: Arc, right: Arc, on: JoinOn, join_type: JoinType, - join_filter: Option, sort_options: Vec, ) -> Result { - let left_schema = left.schema(); - let right_schema = right.schema(); - - if matches!(join_type, LeftSemi | LeftAnti | RightSemi | RightAnti,) { - if join_filter.is_some() { - df_execution_err!("Semi/Anti join with filter is not supported yet")?; - } - } - - check_join_is_valid(&left_schema, &right_schema, &on)?; - if sort_options.len() != on.len() { - df_execution_err!( - "Expected number of sort options: {}, actual: {}", - on.len(), - sort_options.len(), - )?; - } - - let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); Ok(Self { + schema, left, right, on, join_type, - join_filter, - schema, - metrics: ExecutionPlanMetricsSet::new(), sort_options, + metrics: ExecutionPlanMetricsSet::new(), }) } - fn create_join_params(&self, batch_size: usize) -> JoinParams { - let on_left: Vec = self + fn create_join_params(&self, projection: &[usize]) -> Result { + let left_schema = self.left.schema(); + let right_schema = self.right.schema(); + let (left_keys, right_keys): (Vec, Vec) = + self.on.iter().cloned().unzip(); + let key_data_types = self .on .iter() - .map(|on| downcast_any!(on.0, Column).unwrap().index()) - .collect(); - let on_right: Vec = self - .on - .iter() - .map(|on| downcast_any!(on.1, Column).unwrap().index()) - .collect(); - let on_data_types = on_left - .iter() - .map(|&i| self.left.schema().field(i).data_type().clone()) - .collect::>(); - let sub_batch_size = batch_size / batch_size.ilog10() as usize; + .map(|(left_key, right_key)| { + Ok({ + let left_dt = left_key.data_type(&left_schema)?; + let right_dt = right_key.data_type(&right_schema)?; + if left_dt != right_dt { + df_execution_err!( + "join key data type differs {left_dt:?} <-> {right_dt:?}" + )?; + } + left_dt + }) + }) + .collect::>()?; - // use smaller batch size and coalesce batches at the end, to avoid buffer - // overflowing - JoinParams { + let projection = JoinProjection::try_new( + self.join_type, + &self.schema, + &left_schema, + &right_schema, + projection, + )?; + Ok(JoinParams { join_type: self.join_type, + left_schema, + right_schema, output_schema: self.schema(), - on_left, - on_right, - on_data_types, - join_filter: self.join_filter.clone(), + left_keys, + right_keys, + key_data_types, sort_options: self.sort_options.clone(), - batch_size: sub_batch_size, - left_output_projection: (0..self.left.schema().fields().len()).collect(), - right_output_projection: (0..self.right.schema().fields().len()).collect(), - } + projection, + batch_size: batch_size(), + }) + } + + fn execute_with_projection( + &self, + partition: usize, + context: Arc, + projection: Vec, + ) -> Result { + let metrics = Arc::new(BaselineMetrics::new(&self.metrics, partition)); + let join_params = self.create_join_params(&projection)?; + let left = self.left.execute(partition, context.clone())?; + let right = self.right.execute(partition, context.clone())?; + + let metrics_cloned = metrics.clone(); + let context_cloned = context.clone(); + let output_stream = Box::pin(RecordBatchStreamAdapter::new( + join_params.projection.schema.clone(), + futures::stream::once(async move { + context_cloned.output_with_sender( + "SortMergeJoin", + join_params.projection.schema.clone(), + move |sender| execute_join(left, right, join_params, metrics_cloned, sender), + ) + }) + .try_flatten(), + )); + Ok(context.coalesce_with_default_batch_size(output_stream, &metrics)?) } } @@ -154,6 +170,17 @@ impl DisplayAs for SortMergeJoinExec { } } +impl ExecuteWithColumnPruning for SortMergeJoinExec { + fn execute_projected( + &self, + partition: usize, + context: Arc, + projection: &[usize], + ) -> Result { + self.execute_with_projection(partition, context, projection.to_vec()) + } +} + impl ExecutionPlan for SortMergeJoinExec { fn as_any(&self) -> &dyn Any { self @@ -169,7 +196,7 @@ impl ExecutionPlan for SortMergeJoinExec { fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { match self.join_type { - Left | LeftSemi | LeftAnti => self.left.output_ordering(), + Left | LeftSemi | LeftAnti | Existence => self.left.output_ordering(), Right | RightSemi | RightAnti => self.right.output_ordering(), Inner => self.left.output_ordering(), Full => None, @@ -185,11 +212,11 @@ impl ExecutionPlan for SortMergeJoinExec { children: Vec>, ) -> Result> { Ok(Arc::new(SortMergeJoinExec::try_new( + self.schema(), children[0].clone(), children[1].clone(), self.on.clone(), self.join_type, - self.join_filter.clone(), self.sort_options.clone(), )?)) } @@ -199,12 +226,8 @@ impl ExecutionPlan for SortMergeJoinExec { partition: usize, context: Arc, ) -> Result { - let metrics = Arc::new(BaselineMetrics::new(&self.metrics, partition)); - let batch_size = batch_size(); - let join_params = self.create_join_params(batch_size); - let left = self.left.execute(partition, context.clone())?; - let right = self.right.execute(partition, context.clone())?; - execute_with_join_params(context, join_params, left, right, metrics) + let projection = (0..self.schema.fields().len()).collect(); + self.execute_with_projection(partition, context, projection) } fn metrics(&self) -> Option { @@ -216,1549 +239,76 @@ impl ExecutionPlan for SortMergeJoinExec { } } -impl ExecuteWithColumnPruning for SortMergeJoinExec { - fn execute_projected( - &self, - partition: usize, - context: Arc, - projection: &[usize], - ) -> Result { - let metrics = Arc::new(BaselineMetrics::new(&self.metrics, partition)); - let batch_size = batch_size(); - - let (join_params, left_projection, right_projection) = - self.create_join_params(batch_size).project(projection)?; - let left = self - .left - .execute_projected(partition, context.clone(), &left_projection)?; - let right = self - .right - .execute_projected(partition, context.clone(), &right_projection)?; - execute_with_join_params(context, join_params, left, right, metrics) - } -} - -#[derive(Clone)] -struct JoinParams { - join_type: JoinType, - output_schema: SchemaRef, - on_left: Vec, - on_right: Vec, - on_data_types: Vec, - sort_options: Vec, - join_filter: Option, - left_output_projection: Vec, - right_output_projection: Vec, - batch_size: usize, -} - -impl JoinParams { - fn project(&self, projection: &[usize]) -> Result<(Self, Vec, Vec)> { - let num_left_fields = self.left_output_projection.len(); - let mut left_projection = vec![]; - let mut right_projection = vec![]; - - for &i in projection { - match self.join_type { - Inner | Left | Right | Full => { - if i < num_left_fields { - left_projection.push(i); - } else { - right_projection.push(i - num_left_fields); - } - } - LeftSemi | LeftAnti => { - left_projection.push(i); - } - RightSemi | RightAnti => { - right_projection.push(i); - } - } - } - let num_left_output_columns = left_projection.len(); - let num_right_output_columns = right_projection.len(); - - let mut on_left_projected = vec![]; - let mut on_right_projected = vec![]; - for &l in &self.on_left { - on_left_projected.push(left_projection.iter().position(|&i| i == l).unwrap_or_else( - || { - left_projection.push(l); - left_projection.len() - 1 - }, - )); - } - for &r in &self.on_right { - on_right_projected.push( - right_projection - .iter() - .position(|&i| i == r) - .unwrap_or_else(|| { - right_projection.push(r); - right_projection.len() - 1 - }), - ); - } - - let mut join_filter_projected = None; - if let Some(join_filter) = &self.join_filter { - join_filter_projected = Some(JoinFilter::new( - join_filter.expression().clone(), - join_filter - .column_indices() - .iter() - .map(|ci| { - let projected_index = match ci.side { - JoinSide::Left => left_projection - .iter() - .position(|&i| i == ci.index) - .unwrap_or_else(|| { - left_projection.push(ci.index); - left_projection.len() - 1 - }), - JoinSide::Right => right_projection - .iter() - .position(|&i| i == ci.index) - .unwrap_or_else(|| { - right_projection.push(ci.index); - right_projection.len() - 1 - }), - }; - ColumnIndex { - index: projected_index, - side: ci.side, - } - }) - .collect(), - join_filter.schema().clone(), - )); - } - - let projected = Self { - join_type: self.join_type, - output_schema: Arc::new(self.output_schema.project(projection)?), - on_left: on_left_projected, - on_right: on_right_projected, - on_data_types: self.on_data_types.clone(), - sort_options: self.sort_options.clone(), - join_filter: join_filter_projected, - batch_size: self.batch_size, - left_output_projection: (0..num_left_output_columns).collect(), - right_output_projection: (0..num_right_output_columns).collect(), - }; - Ok((projected, left_projection, right_projection)) - } -} - -fn execute_with_join_params( - context: Arc, - join_params: JoinParams, - left: SendableRecordBatchStream, - right: SendableRecordBatchStream, - metrics: Arc, -) -> Result { - let metrics_cloned = metrics.clone(); - let context_cloned = context.clone(); - let output_schema = join_params.output_schema.clone(); - let output_stream = Box::pin(RecordBatchStreamAdapter::new( - join_params.output_schema.clone(), - futures::stream::once(async move { - context_cloned.output_with_sender("SortMergeJoin", output_schema, move |sender| { - execute_join(left, right, join_params, metrics_cloned, sender) - }) - }) - .try_flatten(), - )); - Ok(context.coalesce_with_default_batch_size(output_stream, &metrics)?) -} - -async fn execute_join( +pub async fn execute_join( lstream: SendableRecordBatchStream, rstream: SendableRecordBatchStream, join_params: JoinParams, metrics: Arc, sender: Arc, ) -> Result<()> { - let elapsed_time = metrics.elapsed_compute().clone(); - let mut timer = elapsed_time.timer(); - - let on_row_converter = Arc::new(SyncMutex::new(RowConverter::new( - join_params - .on_data_types - .iter() - .zip(&join_params.sort_options) - .map(|(data_type, sort_option)| { - SortField::new_with_options(data_type.clone(), *sort_option) - }) - .collect(), - )?)); - - let mut lcur = StreamCursor::try_new( - lstream, - on_row_converter.clone(), - join_params.on_left.clone(), - join_params.left_output_projection.clone(), - )?; - let mut rcur = StreamCursor::try_new( - rstream, - on_row_converter.clone(), - join_params.on_right.clone(), - join_params.right_output_projection.clone(), + let start_time = Instant::now(); + + let mut curs = ( + StreamCursor::try_new( + lstream, + &join_params, + JoinSide::Left, + &join_params.projection.left, + )?, + StreamCursor::try_new( + rstream, + &join_params, + JoinSide::Right, + &join_params.projection.right, + )?, + ); + + // start first batches of both side asynchronously + tokio::try_join!( + async { Ok::<_, DataFusionError>(cur_forward!(curs.0)) }, + async { Ok::<_, DataFusionError>(cur_forward!(curs.1)) }, )?; - macro_rules! forward { - ($cur:expr) => {{ - if $cur.next() == NextAction::LoadNextBatch { - $cur.next_batch(&mut timer).await?; - } - }}; - } - - // load first record - forward!(lcur); - forward!(rcur); - let join_type = join_params.join_type; - let mut joiner = Joiner::new(); - let mut leqs = vec![]; - let mut reqs = vec![]; - - macro_rules! joiner_accept_pair { - ($lidx:expr, $ridx:expr) => {{ - let lidx = $lidx; - let ridx = $ridx; - let r = joiner.accept_pair(&join_params, &mut lcur, &mut rcur, lidx, ridx)?; - if let Some(batch) = r { - metrics.record_output(batch.num_rows()); - sender.send(Ok(batch), Some(&mut timer)).await; - } - }}; - } - - // process records until one side is exhausted - while !lcur.finished && !rcur.finished { - let r = compare_cursor(&lcur, lcur.cur_idx, &rcur, rcur.cur_idx); - match r { - Ordering::Less => { - if matches!(join_type, Left | LeftAnti | Full) { - joiner_accept_pair!(Some(lcur.cur_idx), None); - } - forward!(lcur); - lcur.clear_outdated(joiner.l_min_reserved_bidx); - } - Ordering::Greater => { - if matches!(join_type, Right | RightAnti | Full) { - joiner_accept_pair!(None, Some(rcur.cur_idx)); - } - forward!(rcur); - rcur.clear_outdated(joiner.r_min_reserved_bidx); - } - Ordering::Equal => { - let lidx0 = lcur.cur_idx; - let ridx0 = rcur.cur_idx; - leqs.push(lidx0); - reqs.push(ridx0); - forward!(lcur); - forward!(rcur); - - let mut leq = true; - let mut req = true; - while leq && req { - if leq && !lcur.finished && lcur.row(lcur.cur_idx) == lcur.row(lidx0) { - leqs.push(lcur.cur_idx); - forward!(lcur); - } else { - leq = false; - } - if req && !rcur.finished && rcur.row(rcur.cur_idx) == rcur.row(ridx0) { - reqs.push(rcur.cur_idx); - forward!(rcur); - } else { - req = false; - } - } - - match join_type { - Inner | Left | Right | Full => { - for &l in &leqs { - for &r in &reqs { - joiner_accept_pair!(Some(l), Some(r)); - } - } - } - LeftSemi => { - for &l in &leqs { - joiner_accept_pair!(Some(l), None); - } - } - RightSemi => { - for &r in &reqs { - joiner_accept_pair!(None, Some(r)); - } - } - LeftAnti | RightAnti => {} - } - - if leq { - while !lcur.finished && lcur.row(lcur.cur_idx) == rcur.row(ridx0) { - match join_type { - Inner | Left | Right | Full => { - for &r in &reqs { - joiner_accept_pair!(Some(lcur.cur_idx), Some(r)); - } - } - LeftSemi => { - joiner_accept_pair!(Some(lcur.cur_idx), None); - } - RightSemi | LeftAnti | RightAnti => {} - } - forward!(lcur); - lcur.clear_outdated(joiner.l_min_reserved_bidx); - } - } - if req { - while !rcur.finished && rcur.row(rcur.cur_idx) == lcur.row(lidx0) { - match join_type { - Inner | Left | Right | Full => { - for &l in &leqs { - joiner_accept_pair!(Some(l), Some(rcur.cur_idx)); - } - } - RightSemi => { - joiner_accept_pair!(None, Some(rcur.cur_idx)); - } - LeftSemi | LeftAnti | RightAnti => {} - } - forward!(rcur); - rcur.clear_outdated(joiner.r_min_reserved_bidx); - } - } - leqs.clear(); - reqs.clear(); - lcur.clear_outdated(joiner.l_min_reserved_bidx); - rcur.clear_outdated(joiner.r_min_reserved_bidx); - } - } - - // flush joiner if cursors buffered too many batches - if !joiner.is_empty() && (lcur.num_buffered_batches() + rcur.num_buffered_batches() > 5) - || (lcur.mem_size() + rcur.mem_size() > suggested_output_batch_mem_size() - && lcur.num_buffered_batches() > 1 - && rcur.num_buffered_batches() > 1) - { - if let Some(batch) = joiner.flush_pairs(&join_params, &mut lcur, &mut rcur)? { - metrics.record_output(batch.num_rows()); - sender.send(Ok(batch), Some(&mut timer)).await; - } - } - } - - // process rest records in inexhausted side - if matches!(join_type, Left | LeftAnti | Full) { - while !lcur.finished { - joiner_accept_pair!(Some(lcur.cur_idx), None); - forward!(lcur); - lcur.clear_outdated(joiner.l_min_reserved_bidx); - } - } - if matches!(join_type, Right | RightAnti | Full) { - while !rcur.finished { - joiner_accept_pair!(None, Some(rcur.cur_idx)); - forward!(rcur); - rcur.clear_outdated(joiner.r_min_reserved_bidx); - } - } - - // flush joiner - if !joiner.is_empty() { - if let Some(batch) = joiner.flush_pairs(&join_params, &mut lcur, &mut rcur)? { - metrics.record_output(batch.num_rows()); - sender.send(Ok(batch), Some(&mut timer)).await; - } - } + let mut joiner: Pin> = match join_type { + Inner => Box::pin(InnerJoiner::new(join_params, sender)), + Left => Box::pin(LeftOuterJoiner::new(join_params, sender)), + Right => Box::pin(RightOuterJoiner::new(join_params, sender)), + Full => Box::pin(FullOuterJoiner::new(join_params, sender)), + LeftSemi => Box::pin(LeftSemiJoiner::new(join_params, sender)), + RightSemi => Box::pin(RightSemiJoiner::new(join_params, sender)), + LeftAnti => Box::pin(LeftAntiJoiner::new(join_params, sender)), + RightAnti => Box::pin(RightAntiJoiner::new(join_params, sender)), + Existence => Box::pin(ExistenceJoiner::new(join_params, sender)), + }; + joiner.as_mut().join(&mut curs).await?; + metrics.record_output(joiner.num_output_rows()); + + // discount poll input and send output batch time + let mut join_time_ns = (Instant::now() - start_time).as_nanos() as u64; + join_time_ns -= joiner.total_send_output_time() as u64; + join_time_ns -= curs.0.total_poll_time() as u64; + join_time_ns -= curs.1.total_poll_time() as u64; + metrics + .elapsed_compute() + .add_duration(Duration::from_nanos(join_time_ns)); Ok(()) } -struct StreamCursor { - stream: SendableRecordBatchStream, - on_row_converter: Arc>, - on_columns: Vec, - - // IMPORTANT: - // batches/rows/null_buffers always contains a `null batch` in the front - batches: Vec, - projected_batches: Vec, - projection: Vec, - on_rows: Vec>, - on_row_null_buffers: Vec>, - cur_idx: (usize, usize), - num_null_batches: usize, - mem_size: usize, - finished: bool, -} - -#[derive(Clone, Copy, PartialEq, Eq)] -enum NextAction { - None, - LoadNextBatch, -} - -impl StreamCursor { - fn try_new( - stream: SendableRecordBatchStream, - on_row_converter: Arc>, - on_columns: Vec, - projection: Vec, - ) -> Result { - let empty_batch = RecordBatch::new_empty(Arc::new(Schema::new( - stream - .schema() - .fields() - .iter() - .map(|f| f.as_ref().clone().with_nullable(true)) - .collect::>(), - ))); - let null_batch = take_batch_opt(empty_batch, [Option::::None])?; - let null_on_rows = Arc::new( - on_row_converter - .lock() - .convert_columns(null_batch.project(&on_columns)?.columns())?, - ); - let null_nb = NullBuffer::new_null(1); - - Ok(Self { - stream, - on_row_converter, - on_columns, - projected_batches: vec![null_batch.project(&projection)?], - batches: vec![null_batch], - projection, - on_rows: vec![null_on_rows], - on_row_null_buffers: vec![Some(null_nb)], - cur_idx: (0, 0), - num_null_batches: 1, - mem_size: 0, - finished: false, - }) - } - - fn next(&mut self) -> NextAction { - let mut next_action = NextAction::None; - let mut cur_idx = self.cur_idx; - - if cur_idx.1 + 1 < self.batches[cur_idx.0].num_rows() { - cur_idx.1 += 1; - } else { - cur_idx.0 += 1; - cur_idx.1 = 0; - next_action = NextAction::LoadNextBatch; - } - self.cur_idx = cur_idx; - next_action - } - - async fn next_batch(&mut self, stop_timer: &mut ScopedTimerGuard<'_>) -> Result { - stop_timer.stop(); - if let Some(batch) = self.stream.next().await.transpose()? { - stop_timer.restart(); - let on_columns = batch.project(&self.on_columns)?.columns().to_vec(); - let on_row_null_buffer = on_columns - .iter() - .map(|c| c.nulls().cloned()) - .reduce(|lhs, rhs| NullBuffer::union(lhs.as_ref(), rhs.as_ref())) - .unwrap_or(None); - let on_rows = Arc::new(self.on_row_converter.lock().convert_columns(&on_columns)?); - - self.mem_size += batch.get_array_mem_size(); - self.mem_size += on_row_null_buffer - .as_ref() - .map(|nb| nb.buffer().len()) - .unwrap_or_default(); - self.mem_size += on_rows.size(); - - self.projected_batches - .push(batch.project(&self.projection)?); - self.batches.push(batch); - self.on_row_null_buffers.push(on_row_null_buffer); - self.on_rows.push(on_rows); - return Ok(true); - } else { - stop_timer.restart(); +#[macro_export] +macro_rules! compare_cursor { + ($curs:expr) => {{ + match ($curs.0.cur_idx, $curs.1.cur_idx) { + (lidx, _) if $curs.0.is_null_key(lidx) => Ordering::Less, + (_, ridx) if $curs.1.is_null_key(ridx) => Ordering::Greater, + (lidx, ridx) => $curs.0.key(lidx).cmp(&$curs.1.key(ridx)), } - self.finished = true; - Ok(false) - } - - #[inline] - fn row<'a>(&'a self, idx: (usize, usize)) -> Row<'a> { - let bidx = idx.0; - let ridx = idx.1; - self.on_rows[bidx].row(ridx) - } - - #[inline] - fn num_buffered_batches(&self) -> usize { - self.batches.len() - self.num_null_batches - } - - #[inline] - fn mem_size(&self) -> usize { - self.mem_size - } - - #[inline] - fn clear_outdated(&mut self, min_reserved_bidx: usize) { - // fill out-dated batches with null batches - for i in self.num_null_batches..min_reserved_bidx.min(self.cur_idx.0) { - self.mem_size -= self.batches[i].get_array_mem_size(); - self.mem_size -= self.on_row_null_buffers[i] - .as_ref() - .map(|nb| nb.buffer().len()) - .unwrap_or_default(); - self.mem_size -= self.on_rows[i].size(); - - self.projected_batches[i] = self.projected_batches[0].clone(); - self.batches[i] = self.batches[0].clone(); - self.on_rows[i] = self.on_rows[0].clone(); - self.on_row_null_buffers[i] = self.on_row_null_buffers[0].clone(); - self.num_null_batches += 1; - } - } -} - -#[derive(Default)] -struct Joiner { - ljoins: Vec<(usize, usize)>, - rjoins: Vec<(usize, usize)>, - l_min_reserved_bidx: usize, - r_min_reserved_bidx: usize, -} - -impl Joiner { - fn new() -> Self { - Self { - ljoins: vec![], - rjoins: vec![], - l_min_reserved_bidx: usize::MAX, - r_min_reserved_bidx: usize::MAX, - } - } - - fn accept_pair( - &mut self, - join_params: &JoinParams, - lcur: &mut StreamCursor, - rcur: &mut StreamCursor, - l: Option<(usize, usize)>, - r: Option<(usize, usize)>, - ) -> Result> { - if let Some((bidx, ridx)) = l { - self.ljoins.push((bidx, ridx)); - self.l_min_reserved_bidx = self.l_min_reserved_bidx.min(bidx); - } else { - self.ljoins.push((0, 0)); - } - - if let Some((bidx, ridx)) = r { - self.rjoins.push((bidx, ridx)); - self.r_min_reserved_bidx = self.r_min_reserved_bidx.min(bidx); - } else { - self.rjoins.push((0, 0)); - } - - let batch_size = join_params.batch_size; - if self.ljoins.len() >= batch_size || self.rjoins.len() >= batch_size { - return self.flush_pairs(join_params, lcur, rcur); - } - Ok(None) - } - - fn is_empty(&self) -> bool { - self.ljoins.is_empty() && self.rjoins.is_empty() - } - - fn flush_pairs( - &mut self, - join_params: &JoinParams, - lcur: &mut StreamCursor, - rcur: &mut StreamCursor, - ) -> Result> { - self.l_min_reserved_bidx = usize::MAX; - self.r_min_reserved_bidx = usize::MAX; - - if let Some(join_filter) = &join_params.join_filter { - let num_intermediate_rows = std::cmp::max(self.ljoins.len(), self.rjoins.len()); - - // get intermediate batch - let intermediate_columns = join_filter - .column_indices() - .iter() - .map(|ci| { - let (cur, joins) = match ci.side { - JoinSide::Left => (&lcur, &self.ljoins), - JoinSide::Right => (&rcur, &self.rjoins), - }; - let arrays = cur - .batches - .iter() - .map(|b| b.column(ci.index).as_ref()) - .collect::>(); - Ok(arrow::compute::interleave(&arrays, joins)?) - }) - .collect::>>()?; - - let intermediate_batch = RecordBatch::try_new_with_options( - Arc::new(join_filter.schema().clone()), - intermediate_columns, - &RecordBatchOptions::new().with_row_count(Some(num_intermediate_rows)), - )?; - - // evalute filter - let filtered_array = join_filter - .expression() - .evaluate(&intermediate_batch)? - .into_array(intermediate_batch.num_rows())?; - let filtered = as_boolean_array(&filtered_array); - let filtered = if filtered.null_count() > 0 { - prep_null_mask_filter(filtered) - } else { - filtered.clone() - }; - - // apply filter - let mut retained = 0; - for (i, selected) in filtered.values().iter().enumerate() { - if selected { - self.ljoins[retained] = self.ljoins[i]; - self.rjoins[retained] = self.rjoins[i]; - retained += 1; - } - } - self.ljoins.truncate(retained); - self.rjoins.truncate(retained); - if retained == 0 { - return Ok(None); - } - } - - let lcols = || -> Result> { - Ok(if !lcur.projection.is_empty() { - interleave_batches( - lcur.projected_batches[0].schema(), - &lcur.projected_batches, - &self.ljoins, - )? - .columns() - .to_vec() - } else { - vec![] - }) - }; - let rcols = || -> Result> { - Ok(if !rcur.projection.is_empty() { - interleave_batches( - rcur.projected_batches[0].schema(), - &rcur.projected_batches, - &self.rjoins, - )? - .columns() - .to_vec() - } else { - vec![] - }) - }; - - let output_columns = match join_params.join_type { - LeftSemi | LeftAnti => lcols()?, - RightSemi | RightAnti => rcols()?, - _ => [lcols()?, rcols()?].concat(), - }; - let num_output_records = std::cmp::max(self.ljoins.len(), self.rjoins.len()); - self.ljoins.clear(); - self.rjoins.clear(); - let batch = RecordBatch::try_new_with_options( - join_params.output_schema.clone(), - output_columns, - &RecordBatchOptions::new().with_row_count(Some(num_output_records)), - )?; - Ok(Some(batch)) - } -} - -fn compare_cursor( - lcur: &StreamCursor, - lidx: (usize, usize), - rcur: &StreamCursor, - ridx: (usize, usize), -) -> Ordering { - match (&lcur.on_rows.get(lidx.0), &rcur.on_rows.get(ridx.0)) { - (None, _) => Ordering::Greater, - (_, None) => Ordering::Less, - (Some(lrows), Some(rrows)) => { - let lkey = &lrows.row(lidx.1); - let rkey = &rrows.row(ridx.1); - match lkey.cmp(rkey) { - Ordering::Greater => Ordering::Greater, - Ordering::Less => Ordering::Less, - _ => { - if let Some(nb) = &lcur.on_row_null_buffers[lidx.0] { - if nb.is_null(lidx.1) { - return Ordering::Less; - } - } - Ordering::Equal - } - } - } - } + }}; } -#[cfg(test)] -mod tests { - use std::sync::Arc; - - use arrow::{ - self, - array::*, - compute::SortOptions, - datatypes::{DataType, Field, Schema}, - record_batch::RecordBatch, - }; - use datafusion::{ - assert_batches_sorted_eq, - error::Result, - logical_expr::{JoinType, JoinType::*}, - physical_expr::expressions::Column, - physical_plan::{common, joins::utils::*, memory::MemoryExec, ExecutionPlan}, - prelude::SessionContext, - }; - - use crate::sort_merge_join_exec::SortMergeJoinExec; - - fn columns(schema: &Schema) -> Vec { - schema.fields().iter().map(|f| f.name().clone()).collect() - } - - fn build_table_i32( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), - ) -> RecordBatch { - let schema = Schema::new(vec![ - Field::new(a.0, DataType::Int32, false), - Field::new(b.0, DataType::Int32, false), - Field::new(c.0, DataType::Int32, false), - ]); - - RecordBatch::try_new( - Arc::new(schema), - vec![ - Arc::new(Int32Array::from(a.1.clone())), - Arc::new(Int32Array::from(b.1.clone())), - Arc::new(Int32Array::from(c.1.clone())), - ], - ) - .unwrap() - } - - fn build_table( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), - ) -> Arc { - let batch = build_table_i32(a, b, c); - let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) - } - - fn build_table_from_batches(batches: Vec) -> Arc { - let schema = batches.first().unwrap().schema(); - Arc::new(MemoryExec::try_new(&[batches], schema, None).unwrap()) - } - - fn build_date_table( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), - ) -> Arc { - let schema = Schema::new(vec![ - Field::new(a.0, DataType::Date32, false), - Field::new(b.0, DataType::Date32, false), - Field::new(c.0, DataType::Date32, false), - ]); - - let batch = RecordBatch::try_new( - Arc::new(schema), - vec![ - Arc::new(Date32Array::from(a.1.clone())), - Arc::new(Date32Array::from(b.1.clone())), - Arc::new(Date32Array::from(c.1.clone())), - ], - ) - .unwrap(); - - let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) - } - - fn build_date64_table( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), - ) -> Arc { - let schema = Schema::new(vec![ - Field::new(a.0, DataType::Date64, false), - Field::new(b.0, DataType::Date64, false), - Field::new(c.0, DataType::Date64, false), - ]); - - let batch = RecordBatch::try_new( - Arc::new(schema), - vec![ - Arc::new(Date64Array::from(a.1.clone())), - Arc::new(Date64Array::from(b.1.clone())), - Arc::new(Date64Array::from(c.1.clone())), - ], - ) - .unwrap(); - - let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) - } - - /// returns a table with 3 columns of i32 in memory - pub fn build_table_i32_nullable( - a: (&str, &Vec>), - b: (&str, &Vec>), - c: (&str, &Vec>), - ) -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new(a.0, DataType::Int32, true), - Field::new(b.0, DataType::Int32, true), - Field::new(c.0, DataType::Int32, true), - ])); - let batch = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(Int32Array::from(a.1.clone())), - Arc::new(Int32Array::from(b.1.clone())), - Arc::new(Int32Array::from(c.1.clone())), - ], - ) - .unwrap(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) - } - - fn join_with_options( - left: Arc, - right: Arc, - on: JoinOn, - join_type: JoinType, - sort_options: Vec, - ) -> Result { - SortMergeJoinExec::try_new(left, right, on, join_type, None, sort_options) - } - - async fn join_collect( - left: Arc, - right: Arc, - on: JoinOn, - join_type: JoinType, - ) -> Result<(Vec, Vec)> { - let sort_options = vec![SortOptions::default(); on.len()]; - join_collect_with_options(left, right, on, join_type, sort_options).await - } - - async fn join_collect_with_options( - left: Arc, - right: Arc, - on: JoinOn, - join_type: JoinType, - sort_options: Vec, - ) -> Result<(Vec, Vec)> { - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); - let join = join_with_options(left, right, on, join_type, sort_options)?; - let columns = columns(&join.schema()); - - let stream = join.execute(0, task_ctx)?; - let batches = common::collect(stream).await?; - Ok((columns, batches)) - } - - #[tokio::test] - async fn join_inner_one() -> Result<()> { - let left = build_table( - ("a1", &vec![1, 2, 3]), - ("b1", &vec![4, 5, 5]), // this has a repetition - ("c1", &vec![7, 8, 9]), - ); - let right = build_table( - ("a2", &vec![10, 20, 30]), - ("b1", &vec![4, 5, 6]), - ("c2", &vec![70, 80, 90]), - ); - - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b1", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Inner).await?; - - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b1 | c2 |", - "+----+----+----+----+----+----+", - "| 1 | 4 | 7 | 10 | 4 | 70 |", - "| 2 | 5 | 8 | 20 | 5 | 80 |", - "| 3 | 5 | 9 | 20 | 5 | 80 |", - "+----+----+----+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_inner_two() -> Result<()> { - let left = build_table( - ("a1", &vec![1, 2, 2]), - ("b2", &vec![1, 2, 2]), - ("c1", &vec![7, 8, 9]), - ); - let right = build_table( - ("a1", &vec![1, 2, 3]), - ("b2", &vec![1, 2, 2]), - ("c2", &vec![70, 80, 90]), - ); - let on: JoinOn = vec![ - ( - Arc::new(Column::new_with_schema("a1", &left.schema())?), - Arc::new(Column::new_with_schema("a1", &right.schema())?), - ), - ( - Arc::new(Column::new_with_schema("b2", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - ), - ]; - - let (_columns, batches) = join_collect(left, right, on, Inner).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b2 | c1 | a1 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| 1 | 1 | 7 | 1 | 1 | 70 |", - "| 2 | 2 | 8 | 2 | 2 | 80 |", - "| 2 | 2 | 9 | 2 | 2 | 80 |", - "+----+----+----+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_inner_two_two() -> Result<()> { - let left = build_table( - ("a1", &vec![1, 1, 2]), - ("b2", &vec![1, 1, 2]), - ("c1", &vec![7, 8, 9]), - ); - let right = build_table( - ("a1", &vec![1, 1, 3]), - ("b2", &vec![1, 1, 2]), - ("c2", &vec![70, 80, 90]), - ); - let on: JoinOn = vec![ - ( - Arc::new(Column::new_with_schema("a1", &left.schema())?), - Arc::new(Column::new_with_schema("a1", &right.schema())?), - ), - ( - Arc::new(Column::new_with_schema("b2", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - ), - ]; - - let (_columns, batches) = join_collect(left, right, on, Inner).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b2 | c1 | a1 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| 1 | 1 | 7 | 1 | 1 | 70 |", - "| 1 | 1 | 7 | 1 | 1 | 80 |", - "| 1 | 1 | 8 | 1 | 1 | 70 |", - "| 1 | 1 | 8 | 1 | 1 | 80 |", - "+----+----+----+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_inner_with_nulls() -> Result<()> { - let left = build_table_i32_nullable( - ("a1", &vec![Some(1), Some(1), Some(2), Some(2)]), - ("b2", &vec![None, Some(1), Some(2), Some(2)]), // null in key field - ("c1", &vec![Some(1), None, Some(8), Some(9)]), // null in non-key field - ); - let right = build_table_i32_nullable( - ("a1", &vec![Some(1), Some(1), Some(2), Some(3)]), - ("b2", &vec![None, Some(1), Some(2), Some(2)]), - ("c2", &vec![Some(10), Some(70), Some(80), Some(90)]), - ); - let on: JoinOn = vec![ - ( - Arc::new(Column::new_with_schema("a1", &left.schema())?), - Arc::new(Column::new_with_schema("a1", &right.schema())?), - ), - ( - Arc::new(Column::new_with_schema("b2", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - ), - ]; - - let (_, batches) = join_collect(left, right, on, Inner).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b2 | c1 | a1 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| 1 | 1 | | 1 | 1 | 70 |", - "| 2 | 2 | 8 | 2 | 2 | 80 |", - "| 2 | 2 | 9 | 2 | 2 | 80 |", - "+----+----+----+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_inner_with_nulls_with_options() -> Result<()> { - let left = build_table_i32_nullable( - ("a1", &vec![Some(2), Some(2), Some(1), Some(1)]), - ("b2", &vec![Some(2), Some(2), Some(1), None]), // null in key field - ("c1", &vec![Some(9), Some(8), None, Some(1)]), // null in non-key field - ); - let right = build_table_i32_nullable( - ("a1", &vec![Some(3), Some(2), Some(1), Some(1)]), - ("b2", &vec![Some(2), Some(2), Some(1), None]), - ("c2", &vec![Some(90), Some(80), Some(70), Some(10)]), - ); - let on: JoinOn = vec![ - ( - Arc::new(Column::new_with_schema("a1", &left.schema())?), - Arc::new(Column::new_with_schema("a1", &right.schema())?), - ), - ( - Arc::new(Column::new_with_schema("b2", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - ), - ]; - let (_, batches) = join_collect_with_options( - left, - right, - on, - Inner, - vec![ - SortOptions { - descending: true, - nulls_first: false - }; - 2 - ], - // null_equals_null=false - ) - .await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b2 | c1 | a1 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| 2 | 2 | 9 | 2 | 2 | 80 |", - "| 2 | 2 | 8 | 2 | 2 | 80 |", - "| 1 | 1 | | 1 | 1 | 70 |", - "+----+----+----+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_left_one() -> Result<()> { - let left = build_table( - ("a1", &vec![1, 2, 3]), - ("b1", &vec![4, 5, 7]), // 7 does not exist on the right - ("c1", &vec![7, 8, 9]), - ); - let right = build_table( - ("a2", &vec![10, 20, 30]), - ("b1", &vec![4, 5, 6]), - ("c2", &vec![70, 80, 90]), - ); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b1", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Left).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b1 | c2 |", - "+----+----+----+----+----+----+", - "| 1 | 4 | 7 | 10 | 4 | 70 |", - "| 2 | 5 | 8 | 20 | 5 | 80 |", - "| 3 | 7 | 9 | | | |", - "+----+----+----+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_right_one() -> Result<()> { - let left = build_table( - ("a1", &vec![1, 2, 3]), - ("b1", &vec![4, 5, 7]), - ("c1", &vec![7, 8, 9]), - ); - let right = build_table( - ("a2", &vec![10, 20, 30]), - ("b1", &vec![4, 5, 6]), // 6 does not exist on the left - ("c2", &vec![70, 80, 90]), - ); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b1", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Right).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b1 | c2 |", - "+----+----+----+----+----+----+", - "| 1 | 4 | 7 | 10 | 4 | 70 |", - "| 2 | 5 | 8 | 20 | 5 | 80 |", - "| | | | 30 | 6 | 90 |", - "+----+----+----+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_full_one() -> Result<()> { - let left = build_table( - ("a1", &vec![1, 2, 3]), - ("b1", &vec![4, 5, 7]), // 7 does not exist on the right - ("c1", &vec![7, 8, 9]), - ); - let right = build_table( - ("a2", &vec![10, 20, 30]), - ("b2", &vec![4, 5, 6]), - ("c2", &vec![70, 80, 90]), - ); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Full).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| | | | 30 | 6 | 90 |", - "| 1 | 4 | 7 | 10 | 4 | 70 |", - "| 2 | 5 | 8 | 20 | 5 | 80 |", - "| 3 | 7 | 9 | | | |", - "+----+----+----+----+----+----+", - ]; - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_anti() -> Result<()> { - let left = build_table( - ("a1", &vec![1, 2, 2, 3, 5]), - ("b1", &vec![4, 5, 5, 7, 7]), // 7 does not exist on the right - ("c1", &vec![7, 8, 8, 9, 11]), - ); - let right = build_table( - ("a2", &vec![10, 20, 30]), - ("b1", &vec![4, 5, 6]), - ("c2", &vec![70, 80, 90]), - ); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b1", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, LeftAnti).await?; - let expected = vec![ - "+----+----+----+", - "| a1 | b1 | c1 |", - "+----+----+----+", - "| 3 | 7 | 9 |", - "| 5 | 7 | 11 |", - "+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_semi() -> Result<()> { - let left = build_table( - ("a1", &vec![1, 2, 2, 3]), - ("b1", &vec![4, 5, 5, 7]), // 7 does not exist on the right - ("c1", &vec![7, 8, 8, 9]), - ); - let right = build_table( - ("a2", &vec![10, 20, 30]), - ("b1", &vec![4, 5, 6]), // 5 is double on the right - ("c2", &vec![70, 80, 90]), - ); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b1", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, LeftSemi).await?; - let expected = vec![ - "+----+----+----+", - "| a1 | b1 | c1 |", - "+----+----+----+", - "| 1 | 4 | 7 |", - "| 2 | 5 | 8 |", - "| 2 | 5 | 8 |", - "+----+----+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_with_duplicated_column_names() -> Result<()> { - let left = build_table( - ("a", &vec![1, 2, 3]), - ("b", &vec![4, 5, 7]), - ("c", &vec![7, 8, 9]), - ); - let right = build_table( - ("a", &vec![10, 20, 30]), - ("b", &vec![1, 2, 7]), - ("c", &vec![70, 80, 90]), - ); - let on: JoinOn = vec![( - // join on a=b so there are duplicate column names on unjoined columns - Arc::new(Column::new_with_schema("a", &left.schema())?), - Arc::new(Column::new_with_schema("b", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Inner).await?; - let expected = vec![ - "+---+---+---+----+---+----+", - "| a | b | c | a | b | c |", - "+---+---+---+----+---+----+", - "| 1 | 4 | 7 | 10 | 1 | 70 |", - "| 2 | 5 | 8 | 20 | 2 | 80 |", - "+---+---+---+----+---+----+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_date32() -> Result<()> { - let left = build_date_table( - ("a1", &vec![1, 2, 3]), - ("b1", &vec![19107, 19108, 19108]), // this has a repetition - ("c1", &vec![7, 8, 9]), - ); - let right = build_date_table( - ("a2", &vec![10, 20, 30]), - ("b1", &vec![19107, 19108, 19109]), - ("c2", &vec![70, 80, 90]), - ); - - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b1", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Inner).await?; - - let expected = vec![ - "+------------+------------+------------+------------+------------+------------+", - "| a1 | b1 | c1 | a2 | b1 | c2 |", - "+------------+------------+------------+------------+------------+------------+", - "| 1970-01-02 | 2022-04-25 | 1970-01-08 | 1970-01-11 | 2022-04-25 | 1970-03-12 |", - "| 1970-01-03 | 2022-04-26 | 1970-01-09 | 1970-01-21 | 2022-04-26 | 1970-03-22 |", - "| 1970-01-04 | 2022-04-26 | 1970-01-10 | 1970-01-21 | 2022-04-26 | 1970-03-22 |", - "+------------+------------+------------+------------+------------+------------+", - ]; - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_date64() -> Result<()> { - let left = build_date64_table( - ("a1", &vec![1, 2, 3]), - ("b1", &vec![1650703441000, 1650903441000, 1650903441000]), // this has a repetition - ("c1", &vec![7, 8, 9]), - ); - let right = build_date64_table( - ("a2", &vec![10, 20, 30]), - ("b1", &vec![1650703441000, 1650503441000, 1650903441000]), - ("c2", &vec![70, 80, 90]), - ); - - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b1", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Inner).await?; - let expected = vec![ - "+-------------------------+---------------------+-------------------------+-------------------------+---------------------+-------------------------+", - "| a1 | b1 | c1 | a2 | b1 | c2 |", - "+-------------------------+---------------------+-------------------------+-------------------------+---------------------+-------------------------+", - "| 1970-01-01T00:00:00.001 | 2022-04-23T08:44:01 | 1970-01-01T00:00:00.007 | 1970-01-01T00:00:00.010 | 2022-04-23T08:44:01 | 1970-01-01T00:00:00.070 |", - "| 1970-01-01T00:00:00.002 | 2022-04-25T16:17:21 | 1970-01-01T00:00:00.008 | 1970-01-01T00:00:00.030 | 2022-04-25T16:17:21 | 1970-01-01T00:00:00.090 |", - "| 1970-01-01T00:00:00.003 | 2022-04-25T16:17:21 | 1970-01-01T00:00:00.009 | 1970-01-01T00:00:00.030 | 2022-04-25T16:17:21 | 1970-01-01T00:00:00.090 |", - "+-------------------------+---------------------+-------------------------+-------------------------+---------------------+-------------------------+", - ]; - - // The output order is important as SMJ preserves sortedness - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_left_sort_order() -> Result<()> { - let left = build_table( - ("a1", &vec![0, 1, 2, 3, 4, 5]), - ("b1", &vec![3, 4, 5, 6, 6, 7]), - ("c1", &vec![4, 5, 6, 7, 8, 9]), - ); - let right = build_table( - ("a2", &vec![0, 10, 20, 30, 40]), - ("b2", &vec![2, 4, 6, 6, 8]), - ("c2", &vec![50, 60, 70, 80, 90]), - ); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Left).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| 0 | 3 | 4 | | | |", - "| 1 | 4 | 5 | 10 | 4 | 60 |", - "| 2 | 5 | 6 | | | |", - "| 3 | 6 | 7 | 20 | 6 | 70 |", - "| 3 | 6 | 7 | 30 | 6 | 80 |", - "| 4 | 6 | 8 | 20 | 6 | 70 |", - "| 4 | 6 | 8 | 30 | 6 | 80 |", - "| 5 | 7 | 9 | | | |", - "+----+----+----+----+----+----+", - ]; - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_right_sort_order() -> Result<()> { - let left = build_table( - ("a1", &vec![0, 1, 2, 3]), - ("b1", &vec![3, 4, 5, 7]), - ("c1", &vec![6, 7, 8, 9]), - ); - let right = build_table( - ("a2", &vec![0, 10, 20, 30]), - ("b2", &vec![2, 4, 5, 6]), - ("c2", &vec![60, 70, 80, 90]), - ); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Right).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| | | | 0 | 2 | 60 |", - "| 1 | 4 | 7 | 10 | 4 | 70 |", - "| 2 | 5 | 8 | 20 | 5 | 80 |", - "| | | | 30 | 6 | 90 |", - "+----+----+----+----+----+----+", - ]; - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_left_multiple_batches() -> Result<()> { - let left_batch_1 = build_table_i32( - ("a1", &vec![0, 1, 2]), - ("b1", &vec![3, 4, 5]), - ("c1", &vec![4, 5, 6]), - ); - let left_batch_2 = build_table_i32( - ("a1", &vec![3, 4, 5, 6]), - ("b1", &vec![6, 6, 7, 9]), - ("c1", &vec![7, 8, 9, 9]), - ); - let right_batch_1 = build_table_i32( - ("a2", &vec![0, 10, 20]), - ("b2", &vec![2, 4, 6]), - ("c2", &vec![50, 60, 70]), - ); - let right_batch_2 = build_table_i32( - ("a2", &vec![30, 40]), - ("b2", &vec![6, 8]), - ("c2", &vec![80, 90]), - ); - let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); - let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Left).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| 0 | 3 | 4 | | | |", - "| 1 | 4 | 5 | 10 | 4 | 60 |", - "| 2 | 5 | 6 | | | |", - "| 3 | 6 | 7 | 20 | 6 | 70 |", - "| 3 | 6 | 7 | 30 | 6 | 80 |", - "| 4 | 6 | 8 | 20 | 6 | 70 |", - "| 4 | 6 | 8 | 30 | 6 | 80 |", - "| 5 | 7 | 9 | | | |", - "| 6 | 9 | 9 | | | |", - "+----+----+----+----+----+----+", - ]; - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_right_multiple_batches() -> Result<()> { - let right_batch_1 = build_table_i32( - ("a2", &vec![0, 1, 2]), - ("b2", &vec![3, 4, 5]), - ("c2", &vec![4, 5, 6]), - ); - let right_batch_2 = build_table_i32( - ("a2", &vec![3, 4, 5, 6]), - ("b2", &vec![6, 6, 7, 9]), - ("c2", &vec![7, 8, 9, 9]), - ); - let left_batch_1 = build_table_i32( - ("a1", &vec![0, 10, 20]), - ("b1", &vec![2, 4, 6]), - ("c1", &vec![50, 60, 70]), - ); - let left_batch_2 = build_table_i32( - ("a1", &vec![30, 40]), - ("b1", &vec![6, 8]), - ("c1", &vec![80, 90]), - ); - let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); - let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Right).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| | | | 0 | 3 | 4 |", - "| 10 | 4 | 60 | 1 | 4 | 5 |", - "| | | | 2 | 5 | 6 |", - "| 20 | 6 | 70 | 3 | 6 | 7 |", - "| 30 | 6 | 80 | 3 | 6 | 7 |", - "| 20 | 6 | 70 | 4 | 6 | 8 |", - "| 30 | 6 | 80 | 4 | 6 | 8 |", - "| | | | 5 | 7 | 9 |", - "| | | | 6 | 9 | 9 |", - "+----+----+----+----+----+----+", - ]; - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn join_full_multiple_batches() -> Result<()> { - let left_batch_1 = build_table_i32( - ("a1", &vec![0, 1, 2]), - ("b1", &vec![3, 4, 5]), - ("c1", &vec![4, 5, 6]), - ); - let left_batch_2 = build_table_i32( - ("a1", &vec![3, 4, 5, 6]), - ("b1", &vec![6, 6, 7, 9]), - ("c1", &vec![7, 8, 9, 9]), - ); - let right_batch_1 = build_table_i32( - ("a2", &vec![0, 10, 20]), - ("b2", &vec![2, 4, 6]), - ("c2", &vec![50, 60, 70]), - ); - let right_batch_2 = build_table_i32( - ("a2", &vec![30, 40]), - ("b2", &vec![6, 8]), - ("c2", &vec![80, 90]), - ); - let left = build_table_from_batches(vec![left_batch_1, left_batch_2]); - let right = build_table_from_batches(vec![right_batch_1, right_batch_2]); - let on: JoinOn = vec![( - Arc::new(Column::new_with_schema("b1", &left.schema())?), - Arc::new(Column::new_with_schema("b2", &right.schema())?), - )]; - - let (_, batches) = join_collect(left, right, on, Full).await?; - let expected = vec![ - "+----+----+----+----+----+----+", - "| a1 | b1 | c1 | a2 | b2 | c2 |", - "+----+----+----+----+----+----+", - "| | | | 0 | 2 | 50 |", - "| | | | 40 | 8 | 90 |", - "| 0 | 3 | 4 | | | |", - "| 1 | 4 | 5 | 10 | 4 | 60 |", - "| 2 | 5 | 6 | | | |", - "| 3 | 6 | 7 | 20 | 6 | 70 |", - "| 3 | 6 | 7 | 30 | 6 | 80 |", - "| 4 | 6 | 8 | 20 | 6 | 70 |", - "| 4 | 6 | 8 | 30 | 6 | 80 |", - "| 5 | 7 | 9 | | | |", - "| 6 | 9 | 9 | | | |", - "+----+----+----+----+----+----+", - ]; - assert_batches_sorted_eq!(expected, &batches); - Ok(()) - } +#[async_trait] +pub trait Joiner { + async fn join(self: Pin<&mut Self>, curs: &mut StreamCursors) -> Result<()>; + fn total_send_output_time(&self) -> usize; + fn num_output_rows(&self) -> usize; } diff --git a/pom.xml b/pom.xml index 8598cfd26..f5d8fa23d 100644 --- a/pom.xml +++ b/pom.xml @@ -13,7 +13,7 @@ - 2.0.9.1-SNAPSHOT + 3.0.0-SNAPSHOT UTF-8 15.0.2 3.21.9 @@ -107,6 +107,13 @@ + + + com.google.code.findbugs + jsr305 + 2.0.2 + + scala-compile-first diff --git a/rust-toolchain.toml b/rust-toolchain.toml index 42d866a3b..1e25db4d0 100755 --- a/rust-toolchain.toml +++ b/rust-toolchain.toml @@ -16,5 +16,5 @@ # under the License. [toolchain] -channel = "nightly-2023-08-01" -components = ["cargo", "rustfmt", "clippy"] +channel = "nightly-2024-06-27" +components = ["rust-src", "cargo", "rustfmt", "clippy"] diff --git a/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala b/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala index 823a2bcb9..8dd9ea84b 100644 --- a/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala +++ b/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala @@ -79,8 +79,6 @@ import org.apache.spark.sql.execution.blaze.plan.NativeAggBase.AggExecMode import org.apache.spark.sql.execution.blaze.plan.NativeAggExec import org.apache.spark.sql.execution.blaze.plan.NativeBroadcastJoinBase import org.apache.spark.sql.execution.blaze.plan.NativeBroadcastJoinExec -import org.apache.spark.sql.execution.blaze.plan.NativeBroadcastNestedLoopJoinBase -import org.apache.spark.sql.execution.blaze.plan.NativeBroadcastNestedLoopJoinExec import org.apache.spark.sql.execution.blaze.plan.NativeExpandBase import org.apache.spark.sql.execution.blaze.plan.NativeExpandExec import org.apache.spark.sql.execution.blaze.plan.NativeFilterBase @@ -114,6 +112,7 @@ import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.types.DataType import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.execution.blaze.plan.BroadcastSide import org.apache.spark.sql.execution.blaze.plan.NativeParquetSinkBase import org.apache.spark.sql.execution.blaze.plan.NativeParquetSinkExec import org.blaze.{protobuf => pb} @@ -153,7 +152,7 @@ class ShimsImpl extends Shims with Logging { leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, - condition: Option[Expression]): NativeBroadcastJoinBase = + buildSide: BroadcastSide): NativeBroadcastJoinBase = NativeBroadcastJoinExec( left, right, @@ -161,14 +160,7 @@ class ShimsImpl extends Shims with Logging { leftKeys, rightKeys, joinType, - condition) - - override def createNativeBroadcastNestedLoopJoinExec( - left: SparkPlan, - right: SparkPlan, - joinType: JoinType, - condition: Option[Expression]): NativeBroadcastNestedLoopJoinBase = - NativeBroadcastNestedLoopJoinExec(left, right, joinType, condition) + buildSide) override def createNativeSortMergeJoinExec( left: SparkPlan, diff --git a/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastJoinExec.scala b/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastJoinExec.scala index 75d1b7c22..3101587d3 100644 --- a/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastJoinExec.scala +++ b/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastJoinExec.scala @@ -19,8 +19,9 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.joins import org.apache.spark.sql.execution.joins.BuildLeft +import org.apache.spark.sql.execution.joins.BuildRight +import org.apache.spark.sql.execution.joins.BuildSide import org.apache.spark.sql.execution.joins.HashJoin case class NativeBroadcastJoinExec( @@ -30,7 +31,7 @@ case class NativeBroadcastJoinExec( override val leftKeys: Seq[Expression], override val rightKeys: Seq[Expression], override val joinType: JoinType, - override val condition: Option[Expression]) + broadcastSide: BroadcastSide) extends NativeBroadcastJoinBase( left, right, @@ -38,10 +39,15 @@ case class NativeBroadcastJoinExec( leftKeys, rightKeys, joinType, - condition) + broadcastSide) with HashJoin { - override val buildSide: joins.BuildSide = BuildLeft + override val condition: Option[Expression] = None + + override val buildSide: BuildSide = broadcastSide match { + case BroadcastLeft => BuildLeft + case BroadcastRight => BuildRight + } override def withNewChildren(newChildren: Seq[SparkPlan]): SparkPlan = copy(left = newChildren(0), right = newChildren(1)) diff --git a/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastNestedLoopJoinExec.scala b/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastNestedLoopJoinExec.scala deleted file mode 100644 index 7b215cea0..000000000 --- a/spark-extension-shims-spark303/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastNestedLoopJoinExec.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Copyright 2022 The Blaze Authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution.blaze.plan - -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.execution.SparkPlan - -case class NativeBroadcastNestedLoopJoinExec( - override val left: SparkPlan, - override val right: SparkPlan, - joinType: JoinType, - condition: Option[Expression]) - extends NativeBroadcastNestedLoopJoinBase(left, right, joinType, condition) { - - override def withNewChildren(newChildren: Seq[SparkPlan]): SparkPlan = - copy(left = newChildren(0), right = newChildren(1)) -} diff --git a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala index 1d867a565..a394cf934 100644 --- a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala +++ b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/blaze/ShimsImpl.scala @@ -105,7 +105,6 @@ import org.apache.spark.sql.execution.blaze.plan.NativeWindowExec import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.CoalescedMapperPartitionSpec import org.apache.spark.sql.execution.joins.blaze.plan.NativeBroadcastJoinExec -import org.apache.spark.sql.execution.joins.blaze.plan.NativeBroadcastNestedLoopJoinExec import org.apache.spark.sql.execution.joins.blaze.plan.NativeSortMergeJoinExec import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.types.DataType @@ -150,7 +149,7 @@ class ShimsImpl extends Shims with Logging { leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, - condition: Option[Expression]): NativeBroadcastJoinBase = + broadcastSide: BroadcastSide): NativeBroadcastJoinBase = NativeBroadcastJoinExec( left, right, @@ -158,14 +157,7 @@ class ShimsImpl extends Shims with Logging { leftKeys, rightKeys, joinType, - condition) - - override def createNativeBroadcastNestedLoopJoinExec( - left: SparkPlan, - right: SparkPlan, - joinType: JoinType, - condition: Option[Expression]): NativeBroadcastNestedLoopJoinBase = - NativeBroadcastNestedLoopJoinExec(left, right, joinType, condition) + broadcastSide) override def createNativeSortMergeJoinExec( left: SparkPlan, diff --git a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeBlockStoreShuffleReader.scala b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeBlockStoreShuffleReader.scala index 292f23321..fdd3a2453 100644 --- a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeBlockStoreShuffleReader.scala +++ b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeBlockStoreShuffleReader.scala @@ -20,7 +20,6 @@ import java.io.InputStream import org.apache.spark.MapOutputTracker import org.apache.spark.SparkEnv import org.apache.spark.TaskContext - import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.io.CompressionCodec @@ -28,30 +27,21 @@ import org.apache.spark.shuffle.BaseShuffleHandle import org.apache.spark.shuffle.ShuffleReadMetricsReporter import org.apache.spark.storage.BlockId import org.apache.spark.storage.BlockManager +import org.apache.spark.storage.BlockManagerId import org.apache.spark.storage.ShuffleBlockFetcherIterator class BlazeBlockStoreShuffleReader[K, C]( handle: BaseShuffleHandle[K, _, C], - startPartition: Int, - endPartition: Int, + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], context: TaskContext, readMetrics: ShuffleReadMetricsReporter, blockManager: BlockManager = SparkEnv.get.blockManager, mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, - startMapId: Option[Int] = None, - endMapId: Option[Int] = None, shouldBatchFetch: Boolean = false) extends BlazeBlockStoreShuffleReaderBase[K, C](handle, context) with Logging { override def readBlocks(): Iterator[(BlockId, InputStream)] = { - val blocksByAddress = mapOutputTracker.getMapSizesByExecutorId( - handle.shuffleId, - startMapId.getOrElse(0), - endMapId.getOrElse(Int.MaxValue), - startPartition, - endPartition) - new ShuffleBlockFetcherIterator( context, blockManager.blockStoreClient, diff --git a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeShuffleManager.scala b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeShuffleManager.scala index a7390ee01..83decb32a 100644 --- a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeShuffleManager.scala +++ b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeShuffleManager.scala @@ -22,6 +22,7 @@ import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.shuffle._ import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.shuffle.sort.SortShuffleManager.canUseBatchFetch import org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleDependency.isArrowShuffle class BlazeShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { @@ -54,16 +55,27 @@ class BlazeShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { if (isArrowShuffle(handle)) { + val baseShuffleHandle = handle.asInstanceOf[BaseShuffleHandle[K, _, C]] + val (blocksByAddress, canEnableBatchFetch) = + if (baseShuffleHandle.dependency.isShuffleMergeFinalizedMarked) { + val res = SparkEnv.get.mapOutputTracker.getPushBasedShuffleMapSizesByExecutorId( + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) + (res.iter, res.enableBatchFetch) + } else { + val address = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) + (address, true) + } + new BlazeBlockStoreShuffleReader( handle.asInstanceOf[BaseShuffleHandle[K, _, C]], - startPartition, - endPartition, + blocksByAddress, context, metrics, SparkEnv.get.blockManager, SparkEnv.get.mapOutputTracker, - startMapId = Some(startMapIndex), - endMapId = Some(endMapIndex)) + shouldBatchFetch = + canEnableBatchFetch && canUseBatchFetch(startPartition, endPartition, context)) } else { sortShuffleManager.getReader( handle, diff --git a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/joins/blaze/plan/NativeBroadcastJoinExec.scala b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/joins/blaze/plan/NativeBroadcastJoinExec.scala index 3fc664959..de3f5f887 100644 --- a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/joins/blaze/plan/NativeBroadcastJoinExec.scala +++ b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/joins/blaze/plan/NativeBroadcastJoinExec.scala @@ -21,12 +21,16 @@ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.optimizer.BuildLeft +import org.apache.spark.sql.catalyst.optimizer.BuildRight import org.apache.spark.sql.catalyst.optimizer.BuildSide import org.apache.spark.sql.catalyst.plans.physical.BroadcastDistribution import org.apache.spark.sql.catalyst.plans.physical.Distribution import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.blaze.plan.BroadcastLeft +import org.apache.spark.sql.execution.blaze.plan.BroadcastRight +import org.apache.spark.sql.execution.blaze.plan.BroadcastSide import org.apache.spark.sql.execution.blaze.plan.NativeBroadcastJoinBase import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode import org.apache.spark.sql.execution.joins.HashedRelationInfo @@ -39,7 +43,7 @@ case class NativeBroadcastJoinExec( override val leftKeys: Seq[Expression], override val rightKeys: Seq[Expression], override val joinType: JoinType, - override val condition: Option[Expression]) + broadcastSide: BroadcastSide) extends NativeBroadcastJoinBase( left, right, @@ -47,9 +51,11 @@ case class NativeBroadcastJoinExec( leftKeys, rightKeys, joinType, - condition) + broadcastSide) with HashJoin { + override def condition: Option[Expression] = None + override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildBoundKeys, isNullAware = false) BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil @@ -65,7 +71,10 @@ case class NativeBroadcastJoinExec( throw new NotImplementedError("NativeBroadcastJoin dose not support codegen") } - override def buildSide: BuildSide = BuildLeft + override def buildSide: BuildSide = broadcastSide match { + case BroadcastLeft => BuildLeft + case BroadcastRight => BuildRight + } override protected def withNewChildrenInternal( newLeft: SparkPlan, diff --git a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/joins/blaze/plan/NativeBroadcastNestedLoopJoinExec.scala b/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/joins/blaze/plan/NativeBroadcastNestedLoopJoinExec.scala deleted file mode 100644 index a129e91c0..000000000 --- a/spark-extension-shims-spark333/src/main/scala/org/apache/spark/sql/execution/joins/blaze/plan/NativeBroadcastNestedLoopJoinExec.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright 2022 The Blaze Authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution.joins.blaze.plan - -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.blaze.plan.NativeBroadcastNestedLoopJoinBase - -case class NativeBroadcastNestedLoopJoinExec( - override val left: SparkPlan, - override val right: SparkPlan, - joinType: JoinType, - condition: Option[Expression]) - extends NativeBroadcastNestedLoopJoinBase(left, right, joinType, condition) { - - override protected def withNewChildrenInternal( - newLeft: SparkPlan, - newRight: SparkPlan): SparkPlan = - copy(left = newLeft, right = newRight) -} diff --git a/spark-extension/src/main/java/org/apache/spark/sql/blaze/BlazeConf.java b/spark-extension/src/main/java/org/apache/spark/sql/blaze/BlazeConf.java index 31c3b9ac5..f7b1f97a7 100644 --- a/spark-extension/src/main/java/org/apache/spark/sql/blaze/BlazeConf.java +++ b/spark-extension/src/main/java/org/apache/spark/sql/blaze/BlazeConf.java @@ -27,15 +27,13 @@ public enum BlazeConf { /// actual off-heap memory usage is expected to be spark.executor.memoryOverhead * fraction. MEMORY_FRACTION("spark.blaze.memoryFraction", 0.6), - /// translates inequality smj to native. improves performance in most cases, however some - /// issues are found in special cases, like tpcds q72. - SMJ_INEQUALITY_JOIN_ENABLE("spark.blaze.enable.smjInequalityJoin", false), - /// fallbacks to SortMergeJoin when executing BroadcastHashJoin with big broadcasted table. - BHJ_FALLBACKS_TO_SMJ_ENABLE("spark.blaze.enable.bhjFallbacksToSmj", true), + /// not available in blaze 3.0+ + BHJ_FALLBACKS_TO_SMJ_ENABLE("spark.blaze.enable.bhjFallbacksToSmj", false), /// fallbacks to SortMergeJoin when BroadcastHashJoin has a broadcasted table with rows more /// than this threshold. requires spark.blaze.enable.bhjFallbacksToSmj = true. + /// not available in blaze 3.0+ BHJ_FALLBACKS_TO_SMJ_ROWS_THRESHOLD("spark.blaze.bhjFallbacksToSmj.rows", 1000000), /// fallbacks to SortMergeJoin when BroadcastHashJoin has a broadcasted table with memory usage @@ -44,7 +42,7 @@ public enum BlazeConf { /// enable converting upper/lower functions to native, special cases may provide different /// outputs from spark due to different unicode versions. - CASE_CONVERT_FUNCTIONS_ENABLE("spark.blaze.enable.caseconvert.functions", false), + CASE_CONVERT_FUNCTIONS_ENABLE("spark.blaze.enable.caseconvert.functions", true), /// number of threads evaluating UDFs /// improves performance for special case that UDF concurrency matters @@ -64,6 +62,12 @@ public enum BlazeConf { /// mininum number of rows to trigger partial aggregate skipping PARTIAL_AGG_SKIPPING_MIN_ROWS("spark.blaze.partialAggSkipping.minRows", BATCH_SIZE.intConf() * 2), + + // parquet enable page filtering + PARQUET_ENABLE_PAGE_FILTERING("spark.blaze.parquet.enable.pageFiltering", false), + + // parqeut enable bloom filter + PARQUET_ENABLE_BLOOM_FILTER("spark.blaze.parquet.enable.bloomFilter", false), ; private String key; diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeCallNativeWrapper.scala b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeCallNativeWrapper.scala index 09bf85e04..c24888d63 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeCallNativeWrapper.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeCallNativeWrapper.scala @@ -28,7 +28,6 @@ import org.apache.arrow.c.ArrowArray import org.apache.arrow.c.ArrowSchema import org.apache.arrow.c.CDataDictionaryProvider import org.apache.arrow.c.Data -import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.types.pojo.Schema import org.apache.spark.Partition diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeConvertStrategy.scala b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeConvertStrategy.scala index 9f7eb610a..a7ab81de9 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeConvertStrategy.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeConvertStrategy.scala @@ -46,7 +46,8 @@ object BlazeConvertStrategy extends Logging { val convertibleTag: TreeNodeTag[Boolean] = TreeNodeTag("blaze.convertible") val convertStrategyTag: TreeNodeTag[ConvertStrategy] = TreeNodeTag("blaze.convert.strategy") - val childOrderingRequiredTag: TreeNodeTag[Boolean] = TreeNodeTag("blaze.child.ordering.required") + val childOrderingRequiredTag: TreeNodeTag[Boolean] = TreeNodeTag( + "blaze.child.ordering.required") def apply(exec: SparkPlan): Unit = { exec.foreach(_.setTagValue(convertibleTag, true)) diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeConverters.scala b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeConverters.scala index 99d0172a4..7e76b4391 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeConverters.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/BlazeConverters.scala @@ -15,11 +15,8 @@ */ package org.apache.spark.sql.blaze -import java.util.UUID - import scala.annotation.tailrec import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat import org.apache.spark.SparkEnv @@ -57,7 +54,6 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec import org.apache.spark.sql.execution.blaze.plan._ import org.apache.spark.sql.execution.blaze.plan.NativeAggBase -import org.apache.spark.sql.execution.blaze.plan.NativeProjectBase import org.apache.spark.sql.execution.blaze.plan.NativeUnionBase import org.apache.spark.sql.execution.blaze.plan.Util import org.apache.spark.sql.execution.command.DataWritingCommandExec @@ -128,7 +124,9 @@ object BlazeConverters extends Logging { var newExec = exec.withNewChildren(newChildren) exec.getTagValue(convertibleTag).foreach(newExec.setTagValue(convertibleTag, _)) exec.getTagValue(convertStrategyTag).foreach(newExec.setTagValue(convertStrategyTag, _)) - exec.getTagValue(childOrderingRequiredTag).foreach(newExec.setTagValue(childOrderingRequiredTag, _)) + exec + .getTagValue(childOrderingRequiredTag) + .foreach(newExec.setTagValue(childOrderingRequiredTag, _)) if (!isNeverConvert(newExec)) { newExec = convertSparkPlan(newExec) } @@ -333,45 +331,14 @@ object BlazeConverters extends Logging { val (leftKeys, rightKeys, joinType, condition, left, right) = (exec.leftKeys, exec.rightKeys, exec.joinType, exec.condition, exec.left, exec.right) logDebug(s"Converting SortMergeJoinExec: ${Shims.get.simpleStringWithNodeId(exec)}") - var nativeLeft = convertToNative(left) - var nativeRight = convertToNative(right) - var modifiedLeftKeys = leftKeys - var modifiedRightKeys = rightKeys - var needPostProject = false - - if (leftKeys.exists(!_.isInstanceOf[AttributeReference])) { - val (keys, exec) = buildJoinColumnsProject(nativeLeft, leftKeys) - modifiedLeftKeys = keys - nativeLeft = exec - needPostProject = true - } - if (rightKeys.exists(!_.isInstanceOf[AttributeReference])) { - val (keys, exec) = buildJoinColumnsProject(nativeRight, rightKeys) - modifiedRightKeys = keys - nativeRight = exec - needPostProject = true - } - val smjOrig = SortMergeJoinExec( - modifiedLeftKeys, - modifiedRightKeys, + Shims.get.createNativeSortMergeJoinExec( + addRenameColumnsExec(convertToNative(left)), + addRenameColumnsExec(convertToNative(right)), + leftKeys, + rightKeys, joinType, - condition, - addRenameColumnsExec(nativeLeft), - addRenameColumnsExec(nativeRight)) - val smj = Shims.get.createNativeSortMergeJoinExec( - smjOrig.left, - smjOrig.right, - smjOrig.leftKeys, - smjOrig.rightKeys, - smjOrig.joinType, - smjOrig.condition) - - if (needPostProject) { - buildPostJoinProject(smj, exec.output) - } else { - smj - } + condition) } def convertBroadcastHashJoinExec(exec: BroadcastHashJoinExec): SparkPlan = { @@ -385,84 +352,33 @@ object BlazeConverters extends Logging { exec.left, exec.right) logDebug(s"Converting BroadcastHashJoinExec: ${Shims.get.simpleStringWithNodeId(exec)}") - logDebug(s" leftKeys: ${exec.leftKeys}") - logDebug(s" rightKeys: ${exec.rightKeys}") - logDebug(s" joinType: ${exec.joinType}") - logDebug(s" buildSide: ${exec.buildSide}") - logDebug(s" condition: ${exec.condition}") - var (hashed, hashedKeys, nativeProbed, probedKeys) = buildSide match { + logDebug(s" leftKeys: $leftKeys") + logDebug(s" rightKeys: $rightKeys") + logDebug(s" joinType: $joinType") + logDebug(s" buildSide: $buildSide") + logDebug(s" condition: $condition") + assert(condition.isEmpty, "join condition is not supported") + + // verify build side is native + buildSide match { case BuildRight => assert(NativeHelper.isNative(right), "broadcast join build side is not native") - val convertedLeft = convertToNative(left) - (right, rightKeys, convertedLeft, leftKeys) - case BuildLeft => assert(NativeHelper.isNative(left), "broadcast join build side is not native") - val convertedRight = convertToNative(right) - (left, leftKeys, convertedRight, rightKeys) - - case _ => - // scalastyle:off throwerror - throw new NotImplementedError( - "Ignore BroadcastHashJoin with unsupported children structure") } - var modifiedHashedKeys = hashedKeys - var modifiedProbedKeys = probedKeys - var needPostProject = false - - if (hashedKeys.exists(!_.isInstanceOf[AttributeReference])) { - val (keys, exec) = buildJoinColumnsProject(hashed, hashedKeys) - modifiedHashedKeys = keys - hashed = exec - needPostProject = true - } - if (probedKeys.exists(!_.isInstanceOf[AttributeReference])) { - val (keys, exec) = buildJoinColumnsProject(nativeProbed, probedKeys) - modifiedProbedKeys = keys - nativeProbed = exec - needPostProject = true - } + Shims.get.createNativeBroadcastJoinExec( + addRenameColumnsExec(convertToNative(left)), + addRenameColumnsExec(convertToNative(right)), + exec.outputPartitioning, + leftKeys, + rightKeys, + joinType, + buildSide match { + case BuildLeft => BroadcastLeft + case BuildRight => BroadcastRight + }) - val modifiedJoinType = buildSide match { - case BuildLeft => joinType - case BuildRight => - needPostProject = true - val modifiedJoinType = joinType match { // reverse join type - case Inner => Inner - case FullOuter => FullOuter - case LeftOuter => RightOuter - case RightOuter => LeftOuter - case _ => - throw new NotImplementedError( - "BHJ Semi/Anti join with BuildRight is not yet supported") - } - modifiedJoinType - } - - val bhjOrig = BroadcastHashJoinExec( - modifiedHashedKeys, - modifiedProbedKeys, - modifiedJoinType, - BuildLeft, - condition, - addRenameColumnsExec(hashed), - addRenameColumnsExec(nativeProbed)) - - val bhj = Shims.get.createNativeBroadcastJoinExec( - bhjOrig.left, - bhjOrig.right, - bhjOrig.outputPartitioning, - bhjOrig.leftKeys, - bhjOrig.rightKeys, - bhjOrig.joinType, - bhjOrig.condition) - - if (needPostProject) { - buildPostJoinProject(bhj, exec.output) - } else { - bhj - } } catch { case e @ (_: NotImplementedError | _: Exception) => val underlyingBroadcast = exec.buildSide match { @@ -483,60 +399,29 @@ object BlazeConverters extends Logging { logDebug(s" joinType: ${exec.joinType}") logDebug(s" buildSide: ${exec.buildSide}") logDebug(s" condition: ${exec.condition}") - val (broadcasted, nativeProbed) = buildSide match { + assert(condition.isEmpty, "join condition is not supported") + + // verify build side is native + buildSide match { case BuildRight => assert(NativeHelper.isNative(right), "broadcast join build side is not native") - val convertedLeft = convertToNative(left) - (right, convertedLeft) - case BuildLeft => assert(NativeHelper.isNative(left), "broadcast join build side is not native") - val convertedRight = convertToNative(right) - (left, convertedRight) - - case _ => - // scalastyle:off throwerror - throw new NotImplementedError( - "Ignore BroadcastNestedLoopJoin with unsupported children structure") - } - - // the in-memory inner table is not the same in different join types - // reference: https://docs.rs/datafusion/latest/datafusion/physical_plan/joins/struct.NestedLoopJoinExec.html - var needPostProject = false - val (modifiedLeft, modifiedRight, modifiedJoinType) = (buildSide, joinType) match { - case (BuildLeft, RightOuter | FullOuter) => - (broadcasted, nativeProbed, joinType) // RightOuter, FullOuter => BuildLeft - case (BuildRight, Inner | LeftOuter | LeftSemi | LeftAnti) => - ( - nativeProbed, - broadcasted, - joinType - ) // Inner, LeftOuter, LeftSemi, LeftAnti => BuildRight - case _ => - needPostProject = true - val modifiedJoinType = joinType match { - case Inner => - (nativeProbed, broadcasted, Inner) // Inner + BuildLeft => BuildRight - case FullOuter => - (broadcasted, nativeProbed, FullOuter) // FullOuter + BuildRight => BuildLeft - case _ => - throw new NotImplementedError( - s"BNLJ $joinType with $buildSide is not yet supported") - } - modifiedJoinType } - val bnlj = Shims.get.createNativeBroadcastNestedLoopJoinExec( - addRenameColumnsExec(modifiedLeft), - addRenameColumnsExec(modifiedRight), - modifiedJoinType, - condition) + // reuse NativeBroadcastJoin with empty equility keys + Shims.get.createNativeBroadcastJoinExec( + addRenameColumnsExec(convertToNative(left)), + addRenameColumnsExec(convertToNative(right)), + exec.outputPartitioning, + Nil, + Nil, + joinType, + buildSide match { + case BuildLeft => BroadcastLeft + case BuildRight => BroadcastRight + }) - if (needPostProject) { - buildPostJoinProject(bnlj, exec.output) - } else { - bnlj - } } catch { case e @ (_: NotImplementedError | _: Exception) => val underlyingBroadcast = exec.buildSide match { @@ -851,44 +736,6 @@ object BlazeConverters extends Logging { exec } - private def buildJoinColumnsProject( - child: SparkPlan, - joinKeys: Seq[Expression]): (Seq[AttributeReference], NativeProjectBase) = { - val extraProjectList = ArrayBuffer[NamedExpression]() - val transformedKeys = ArrayBuffer[AttributeReference]() - - joinKeys.foreach { - case attr: AttributeReference => transformedKeys.append(attr) - case expr => - val aliasExpr = - Alias(expr, s"JOIN_KEY:${expr.toString()} (${UUID.randomUUID().toString})")() - extraProjectList.append(aliasExpr) - - val attr = AttributeReference( - aliasExpr.name, - aliasExpr.dataType, - aliasExpr.nullable, - aliasExpr.metadata)(aliasExpr.exprId, aliasExpr.qualifier) - transformedKeys.append(attr) - } - ( - transformedKeys, - Shims.get - .createNativeProjectExec(child.output ++ extraProjectList, addRenameColumnsExec(child))) - } - - private def buildPostJoinProject( - child: SparkPlan, - output: Seq[Attribute]): NativeProjectBase = { - val projectList = output - .filter(!_.name.startsWith("JOIN_KEY:")) - .map(attr => - AttributeReference(attr.name, attr.dataType, attr.nullable, attr.metadata)( - attr.exprId, - attr.qualifier)) - Shims.get.createNativeProjectExec(projectList, child) - } - private def getPartialAggProjection( aggregateExprs: Seq[AggregateExpression], groupingExprs: Seq[NamedExpression]) diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/NativeConverters.scala b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/NativeConverters.scala index 1cbfcc8b0..355444a7d 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/NativeConverters.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/NativeConverters.scala @@ -52,12 +52,11 @@ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.LeafExpression +import org.apache.spark.sql.catalyst.plans.ExistenceJoin import org.apache.spark.sql.execution.blaze.plan.Util import org.apache.spark.sql.execution.ScalarSubquery -import org.apache.spark.sql.execution.aggregate.ScalaUDAF import org.apache.spark.sql.hive.blaze.HiveUDFUtil import org.apache.spark.sql.hive.blaze.HiveUDFUtil.getFunctionClassName -import org.apache.spark.sql.hive.blaze.HiveUDFUtil.isHiveSimpleUDF import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.ArrayType import org.apache.spark.sql.types.AtomicType @@ -1110,6 +1109,7 @@ object NativeConverters extends Logging { case FullOuter => pb.JoinType.FULL case LeftSemi => pb.JoinType.SEMI case LeftAnti => pb.JoinType.ANTI + case _: ExistenceJoin => pb.JoinType.EXISTENCE case _ => throw new NotImplementedError(s"unsupported join type: ${joinType}") } } diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/Shims.scala b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/Shims.scala index a8aaad261..12deb83c6 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/Shims.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/Shims.scala @@ -79,13 +79,7 @@ abstract class Shims { leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, - condition: Option[Expression]): NativeBroadcastJoinBase - - def createNativeBroadcastNestedLoopJoinExec( - left: SparkPlan, - right: SparkPlan, - joinType: JoinType, - condition: Option[Expression]): NativeBroadcastNestedLoopJoinBase + broadcastSide: BroadcastSide): NativeBroadcastJoinBase def createNativeSortMergeJoinExec( left: SparkPlan, diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/SparkUDFWrapperContext.scala b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/SparkUDFWrapperContext.scala index ff27e53ca..e9498137c 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/SparkUDFWrapperContext.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/SparkUDFWrapperContext.scala @@ -16,7 +16,6 @@ package org.apache.spark.sql.blaze import java.nio.ByteBuffer - import org.apache.arrow.c.ArrowArray import org.apache.arrow.c.Data import org.apache.arrow.vector.VectorSchemaRoot diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/util/Using.scala b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/util/Using.scala index b78eb080d..b1039694d 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/blaze/util/Using.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/blaze/util/Using.scala @@ -19,15 +19,14 @@ import scala.util.control.{ControlThrowable, NonFatal} import scala.util.Try /** - * A utility for performing automatic resource management. It can be used to perform an - * operation using resources, after which it releases the resources in reverse order - * of their creation. + * A utility for performing automatic resource management. It can be used to perform an operation + * using resources, after which it releases the resources in reverse order of their creation. * * ==Usage== * - * There are multiple ways to automatically manage resources with `Using`. If you only need - * to manage a single resource, the [[Using.apply `apply`]] method is easiest; it wraps the - * resource opening, operation, and resource releasing in a `Try`. + * There are multiple ways to automatically manage resources with `Using`. If you only need to + * manage a single resource, the [[Using.apply `apply`]] method is easiest; it wraps the resource + * opening, operation, and resource releasing in a `Try`. * * Example: * {{{ @@ -37,9 +36,9 @@ import scala.util.Try * } * }}} * - * If you need to manage multiple resources, [[Using.Manager$.apply `Using.Manager`]] should - * be used. It allows the managing of arbitrarily many resources, whose creation, use, and - * release are all wrapped in a `Try`. + * If you need to manage multiple resources, [[Using.Manager$.apply `Using.Manager`]] should be + * used. It allows the managing of arbitrarily many resources, whose creation, use, and release + * are all wrapped in a `Try`. * * Example: * {{{ @@ -70,43 +69,44 @@ import scala.util.Try * * ==Suppression Behavior== * - * If two exceptions are thrown (e.g., by an operation and closing a resource), - * one of them is re-thrown, and the other is - * [[java.lang.Throwable.addSuppressed(Throwable) added to it as a suppressed exception]]. - * If the two exceptions are of different 'severities' (see below), the one of a higher - * severity is re-thrown, and the one of a lower severity is added to it as a suppressed - * exception. If the two exceptions are of the same severity, the one thrown first is - * re-thrown, and the one thrown second is added to it as a suppressed exception. - * If an exception is a [[scala.util.control.ControlThrowable `ControlThrowable`]], or - * if it does not support suppression (see - * [[java.lang.Throwable `Throwable`'s constructor with an `enableSuppression` parameter]]), - * an exception that would have been suppressed is instead discarded. + * If two exceptions are thrown (e.g., by an operation and closing a resource), one of them is + * re-thrown, and the other is + * [[java.lang.Throwable.addSuppressed(Throwable) added to it as a suppressed exception]]. If the + * two exceptions are of different 'severities' (see below), the one of a higher severity is + * re-thrown, and the one of a lower severity is added to it as a suppressed exception. If the two + * exceptions are of the same severity, the one thrown first is re-thrown, and the one thrown + * second is added to it as a suppressed exception. If an exception is a + * [[scala.util.control.ControlThrowable `ControlThrowable`]], or if it does not support + * suppression (see + * [[java.lang.Throwable `Throwable`'s constructor with an `enableSuppression` parameter]]), an + * exception that would have been suppressed is instead discarded. * * Exceptions are ranked from highest to lowest severity as follows: * - `java.lang.VirtualMachineError` * - `java.lang.LinkageError` * - `java.lang.InterruptedException` and `java.lang.ThreadDeath` - * - [[scala.util.control.NonFatal fatal exceptions]], excluding `scala.util.control.ControlThrowable` + * - [[scala.util.control.NonFatal fatal exceptions]], excluding + * `scala.util.control.ControlThrowable` * - `scala.util.control.ControlThrowable` * - all other exceptions * - * When more than two exceptions are thrown, the first two are combined and - * re-thrown as described above, and each successive exception thrown is combined - * as it is thrown. + * When more than two exceptions are thrown, the first two are combined and re-thrown as described + * above, and each successive exception thrown is combined as it is thrown. * - * @define suppressionBehavior See the main doc for [[Using `Using`]] for full details of - * suppression behavior. + * @define suppressionBehavior + * See the main doc for [[Using `Using`]] for full details of suppression behavior. */ object Using { /** - * Performs an operation using a resource, and then releases the resource, - * even if the operation throws an exception. + * Performs an operation using a resource, and then releases the resource, even if the operation + * throws an exception. * * $suppressionBehavior * - * @return a [[Try]] containing an exception if one or more were thrown, - * or the result of the operation if no exceptions were thrown + * @return + * a [[Try]] containing an exception if one or more were thrown, or the result of the + * operation if no exceptions were thrown */ def apply[R: Releasable, A](resource: => R)(f: R => A): Try[A] = Try { Using.resource(resource)(f) @@ -115,20 +115,20 @@ object Using { /** * A resource manager. * - * Resources can be registered with the manager by calling [[acquire `acquire`]]; - * such resources will be released in reverse order of their acquisition - * when the manager is closed, regardless of any exceptions thrown - * during use. + * Resources can be registered with the manager by calling [[acquire `acquire`]]; such resources + * will be released in reverse order of their acquisition when the manager is closed, regardless + * of any exceptions thrown during use. * * $suppressionBehavior * - * @note It is recommended for API designers to require an implicit `Manager` - * for the creation of custom resources, and to call `acquire` during those - * resources' construction. Doing so guarantees that the resource ''must'' be - * automatically managed, and makes it impossible to forget to do so. + * @note + * It is recommended for API designers to require an implicit `Manager` for the creation of + * custom resources, and to call `acquire` during those resources' construction. Doing so + * guarantees that the resource ''must'' be automatically managed, and makes it impossible to + * forget to do so. * - * Example: - * {{{ + * Example: + * {{{ * class SafeFileReader(file: File)(implicit manager: Using.Manager) * extends BufferedReader(new FileReader(file)) { * @@ -136,7 +136,7 @@ object Using { * * manager.acquire(this) * } - * }}} + * }}} */ final class Manager private { import Manager._ @@ -145,9 +145,8 @@ object Using { private[this] var resources: List[Resource[_]] = Nil /** - * Registers the specified resource with this manager, so that - * the resource is released when the manager is closed, and then - * returns the (unmodified) resource. + * Registers the specified resource with this manager, so that the resource is released when + * the manager is closed, and then returns the (unmodified) resource. */ def apply[R: Releasable](resource: R): R = { acquire(resource) @@ -155,8 +154,8 @@ object Using { } /** - * Registers the specified resource with this manager, so that - * the resource is released when the manager is closed. + * Registers the specified resource with this manager, so that the resource is released when + * the manager is closed. */ def acquire[R: Releasable](resource: R): Unit = { if (resource == null) throw new NullPointerException("null resource") @@ -194,8 +193,8 @@ object Using { object Manager { /** - * Performs an operation using a `Manager`, then closes the `Manager`, - * releasing its resources (in reverse order of acquisition). + * Performs an operation using a `Manager`, then closes the `Manager`, releasing its resources + * (in reverse order of acquisition). * * Example: * {{{ @@ -204,9 +203,8 @@ object Using { * } * }}} * - * If using resources which require an implicit `Manager` as a parameter, - * this method should be invoked with an `implicit` modifier before the function - * parameter: + * If using resources which require an implicit `Manager` as a parameter, this method should + * be invoked with an `implicit` modifier before the function parameter: * * Example: * {{{ @@ -217,10 +215,13 @@ object Using { * * See the main doc for [[Using `Using`]] for full details of suppression behavior. * - * @param op the operation to perform using the manager - * @tparam A the return type of the operation - * @return a [[Try]] containing an exception if one or more were thrown, - * or the result of the operation if no exceptions were thrown + * @param op + * the operation to perform using the manager + * @tparam A + * the return type of the operation + * @return + * a [[Try]] containing an exception if one or more were thrown, or the result of the + * operation if no exceptions were thrown */ def apply[A](op: Manager => A): Try[A] = Try { (new Manager).manage(op) } @@ -247,18 +248,21 @@ object Using { } /** - * Performs an operation using a resource, and then releases the resource, - * even if the operation throws an exception. This method behaves similarly - * to Java's try-with-resources. + * Performs an operation using a resource, and then releases the resource, even if the operation + * throws an exception. This method behaves similarly to Java's try-with-resources. * * $suppressionBehavior * - * @param resource the resource - * @param body the operation to perform with the resource - * @tparam R the type of the resource - * @tparam A the return type of the operation - * @return the result of the operation, if neither the operation nor - * releasing the resource throws + * @param resource + * the resource + * @param body + * the operation to perform with the resource + * @tparam R + * the type of the resource + * @tparam A + * the return type of the operation + * @return + * the result of the operation, if neither the operation nor releasing the resource throws */ def resource[R, A](resource: R)(body: R => A)(implicit releasable: Releasable[R]): A = { if (resource == null) throw new NullPointerException("null resource") @@ -281,20 +285,26 @@ object Using { } /** - * Performs an operation using two resources, and then releases the resources - * in reverse order, even if the operation throws an exception. This method - * behaves similarly to Java's try-with-resources. + * Performs an operation using two resources, and then releases the resources in reverse order, + * even if the operation throws an exception. This method behaves similarly to Java's + * try-with-resources. * * $suppressionBehavior * - * @param resource1 the first resource - * @param resource2 the second resource - * @param body the operation to perform using the resources - * @tparam R1 the type of the first resource - * @tparam R2 the type of the second resource - * @tparam A the return type of the operation - * @return the result of the operation, if neither the operation nor - * releasing the resources throws + * @param resource1 + * the first resource + * @param resource2 + * the second resource + * @param body + * the operation to perform using the resources + * @tparam R1 + * the type of the first resource + * @tparam R2 + * the type of the second resource + * @tparam A + * the return type of the operation + * @return + * the result of the operation, if neither the operation nor releasing the resources throws */ def resources[R1: Releasable, R2: Releasable, A](resource1: R1, resource2: => R2)( body: (R1, R2) => A): A = @@ -305,22 +315,30 @@ object Using { } /** - * Performs an operation using three resources, and then releases the resources - * in reverse order, even if the operation throws an exception. This method - * behaves similarly to Java's try-with-resources. + * Performs an operation using three resources, and then releases the resources in reverse + * order, even if the operation throws an exception. This method behaves similarly to Java's + * try-with-resources. * * $suppressionBehavior * - * @param resource1 the first resource - * @param resource2 the second resource - * @param resource3 the third resource - * @param body the operation to perform using the resources - * @tparam R1 the type of the first resource - * @tparam R2 the type of the second resource - * @tparam R3 the type of the third resource - * @tparam A the return type of the operation - * @return the result of the operation, if neither the operation nor - * releasing the resources throws + * @param resource1 + * the first resource + * @param resource2 + * the second resource + * @param resource3 + * the third resource + * @param body + * the operation to perform using the resources + * @tparam R1 + * the type of the first resource + * @tparam R2 + * the type of the second resource + * @tparam R3 + * the type of the third resource + * @tparam A + * the return type of the operation + * @return + * the result of the operation, if neither the operation nor releasing the resources throws */ def resources[R1: Releasable, R2: Releasable, R3: Releasable, A]( resource1: R1, @@ -335,24 +353,34 @@ object Using { } /** - * Performs an operation using four resources, and then releases the resources - * in reverse order, even if the operation throws an exception. This method - * behaves similarly to Java's try-with-resources. + * Performs an operation using four resources, and then releases the resources in reverse order, + * even if the operation throws an exception. This method behaves similarly to Java's + * try-with-resources. * * $suppressionBehavior * - * @param resource1 the first resource - * @param resource2 the second resource - * @param resource3 the third resource - * @param resource4 the fourth resource - * @param body the operation to perform using the resources - * @tparam R1 the type of the first resource - * @tparam R2 the type of the second resource - * @tparam R3 the type of the third resource - * @tparam R4 the type of the fourth resource - * @tparam A the return type of the operation - * @return the result of the operation, if neither the operation nor - * releasing the resources throws + * @param resource1 + * the first resource + * @param resource2 + * the second resource + * @param resource3 + * the third resource + * @param resource4 + * the fourth resource + * @param body + * the operation to perform using the resources + * @tparam R1 + * the type of the first resource + * @tparam R2 + * the type of the second resource + * @tparam R3 + * the type of the third resource + * @tparam R4 + * the type of the fourth resource + * @tparam A + * the return type of the operation + * @return + * the result of the operation, if neither the operation nor releasing the resources throws */ def resources[R1: Releasable, R2: Releasable, R3: Releasable, R4: Releasable, A]( resource1: R1, @@ -372,17 +400,18 @@ object Using { /** * A typeclass describing how to release a particular type of resource. * - * A resource is anything which needs to be released, closed, or otherwise cleaned up - * in some way after it is finished being used, and for which waiting for the object's - * garbage collection to be cleaned up would be unacceptable. For example, an instance of - * [[java.io.OutputStream]] would be considered a resource, because it is important to close - * the stream after it is finished being used. + * A resource is anything which needs to be released, closed, or otherwise cleaned up in some + * way after it is finished being used, and for which waiting for the object's garbage + * collection to be cleaned up would be unacceptable. For example, an instance of + * [[java.io.OutputStream]] would be considered a resource, because it is important to close the + * stream after it is finished being used. * - * An instance of `Releasable` is needed in order to automatically manage a resource - * with [[Using `Using`]]. An implicit instance is provided for all types extending + * An instance of `Releasable` is needed in order to automatically manage a resource with + * [[Using `Using`]]. An implicit instance is provided for all types extending * [[java.lang.AutoCloseable]]. * - * @tparam R the type of the resource + * @tparam R + * the type of the resource */ trait Releasable[-R] { diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/arrowio/util/ArrowUtils.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/arrowio/util/ArrowUtils.scala index 6e18f4712..f6ddfc607 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/arrowio/util/ArrowUtils.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/arrowio/util/ArrowUtils.scala @@ -15,7 +15,8 @@ */ package org.apache.spark.sql.execution.blaze.arrowio.util -import scala.collection.JavaConverters._ +import scala.collection.JavaConverters.asScalaBufferConverter +import scala.collection.JavaConverters.seqAsJavaListConverter import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.memory.RootAllocator @@ -31,7 +32,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.ShutdownHookManager object ArrowUtils { - val rootAllocator = new RootAllocator(Long.MaxValue) ShutdownHookManager.addShutdownHook(() => rootAllocator.close()) @@ -128,7 +128,7 @@ object ArrowUtils { ArrayType(elementType, containsNull = elementField.isNullable) case ArrowType.Struct.INSTANCE => - val fields = field.getChildren().asScala.map { child => + val fields = field.getChildren.asScala.map { child => val dt = fromArrowField(child) StructField(child.getName, dt, child.isNullable) } diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/ConvertToNativeBase.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/ConvertToNativeBase.scala index 052262325..852e5332d 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/ConvertToNativeBase.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/ConvertToNativeBase.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.blaze.NativeHelper import org.apache.spark.sql.execution.blaze.arrowio.ArrowFFIExportIterator import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.OneToOneDependency -import org.apache.spark.sql.blaze.BlazeConf import org.blaze.protobuf.FFIReaderExecNode import org.blaze.protobuf.PhysicalPlanNode import org.blaze.protobuf.Schema diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastExchangeBase.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastExchangeBase.scala index 5525fb473..6fcbd4786 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastExchangeBase.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastExchangeBase.scala @@ -24,22 +24,19 @@ import java.util.concurrent.Future import java.util.concurrent.TimeoutException import java.util.concurrent.TimeUnit -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConverters._ import scala.collection.immutable.SortedMap import scala.concurrent.Promise +import org.apache.commons.lang3.reflect.MethodUtils import org.apache.spark.OneToOneDependency import org.apache.spark.Partition import org.apache.spark.SparkException import org.apache.spark.TaskContext import org.apache.spark.broadcast -import org.blaze.{protobuf => pb} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.blaze.BlazeCallNativeWrapper -import org.apache.spark.sql.blaze.BlazeConf import org.apache.spark.sql.blaze.JniBridge import org.apache.spark.sql.blaze.MetricNode import org.apache.spark.sql.blaze.NativeConverters @@ -49,7 +46,10 @@ import org.apache.spark.sql.blaze.NativeSupports import org.apache.spark.sql.blaze.Shims import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.BoundReference import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.InterpretedUnsafeProjection import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.sql.catalyst.plans.physical.BroadcastPartitioning import org.apache.spark.sql.catalyst.plans.physical.IdentityBroadcastMode @@ -63,6 +63,8 @@ import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.types.BinaryType +import org.blaze.{protobuf => pb} abstract class NativeBroadcastExchangeBase(mode: BroadcastMode, override val child: SparkPlan) extends BroadcastExchangeLike @@ -71,10 +73,15 @@ abstract class NativeBroadcastExchangeBase(mode: BroadcastMode, override val chi override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) + def broadcastMode: BroadcastMode = this.mode + + protected val hashMapOutput: Seq[Attribute] = output + .map(_.withNullability(true)) :+ AttributeReference("~TABLE", BinaryType, nullable = true)() + protected val nativeSchema: pb.Schema = Util.getNativeSchema(output) + protected val nativeHashMapSchema: pb.Schema = Util.getNativeSchema(hashMapOutput) def getRunId: UUID - override lazy val metrics: Map[String, SQLMetric] = SortedMap[String, SQLMetric]() ++ Map( NativeHelper .getDefaultNativeMetrics(sparkContext) @@ -93,9 +100,6 @@ abstract class NativeBroadcastExchangeBase(mode: BroadcastMode, override val chi override def doPrepare(): Unit = { // Materialize the future. relationFuture - relationFuture - relationFuture - relationFuture } override def doExecuteBroadcast[T](): Broadcast[T] = { @@ -103,17 +107,31 @@ abstract class NativeBroadcastExchangeBase(mode: BroadcastMode, override val chi override def index: Int = 0 } val broadcastReadNativePlan = doExecuteNative().nativePlan(singlePartition, null) - val rows = NativeHelper.executeNativePlan( + val rowsIter = NativeHelper.executeNativePlan( broadcastReadNativePlan, MetricNode(Map(), Nil, None), singlePartition, None) - val v = mode.transform(rows.toArray) - + val pruneKeyField = new InterpretedUnsafeProjection( + output.zipWithIndex + .map(v => BoundReference(v._2, v._1.dataType, v._1.nullable)) + .toArray) + + val dataRows = rowsIter + .map(pruneKeyField) + .map(_.copy()) + .toArray + + val broadcast = relationFuture.get // bloadcast must be resolved + val v = mode.transform(dataRows) val dummyBroadcasted = new Broadcast[Any](-1) { override protected def getValue(): Any = v - override protected def doUnpersist(blocking: Boolean): Unit = {} - override protected def doDestroy(blocking: Boolean): Unit = {} + override protected def doUnpersist(blocking: Boolean): Unit = { + MethodUtils.invokeMethod(broadcast, true, "doUnpersist", Array(blocking)) + } + override protected def doDestroy(blocking: Boolean): Unit = { + MethodUtils.invokeMethod(broadcast, true, "doDestroy", Array(blocking)) + } } dummyBroadcasted.asInstanceOf[Broadcast[T]] } @@ -154,13 +172,14 @@ abstract class NativeBroadcastExchangeBase(mode: BroadcastMode, override val chi Channels.newChannel(new ByteArrayInputStream(bytes)) }) } + JniBridge.resourcesMap.put(resourceId, () => provideIpcIterator()) pb.PhysicalPlanNode .newBuilder() .setIpcReader( pb.IpcReaderExecNode .newBuilder() - .setSchema(nativeSchema) + .setSchema(nativeHashMapSchema) .setNumPartitions(1) .setIpcProviderResourceId(resourceId) .build()) @@ -267,39 +286,21 @@ object NativeBroadcastExchangeBase { keys: Seq[Expression], nativeSchema: pb.Schema): Array[Array[Byte]] = { - if (!BlazeConf.BHJ_FALLBACKS_TO_SMJ_ENABLE.booleanConf() || keys.isEmpty) { - return collectedData // no need to sort data in driver side - } - val readerIpcProviderResourceId = s"BuildBroadcastDataReader:${UUID.randomUUID()}" val readerExec = pb.IpcReaderExecNode .newBuilder() .setSchema(nativeSchema) .setIpcProviderResourceId(readerIpcProviderResourceId) - val sortExec = pb.SortExecNode + val buildHashMapExec = pb.BroadcastJoinBuildHashMapExecNode .newBuilder() .setInput(pb.PhysicalPlanNode.newBuilder().setIpcReader(readerExec)) - .addAllExpr( - keys - .map(key => { - pb.PhysicalExprNode - .newBuilder() - .setSort( - pb.PhysicalSortExprNode - .newBuilder() - .setExpr(NativeConverters.convertExpr(key)) - .setAsc(true) - .setNullsFirst(true) - .build()) - .build() - }) - .asJava) + .addAllKeys(keys.map(key => NativeConverters.convertExpr(key)).asJava) val writerIpcProviderResourceId = s"BuildBroadcastDataWriter:${UUID.randomUUID()}" val writerExec = pb.IpcWriterExecNode .newBuilder() - .setInput(pb.PhysicalPlanNode.newBuilder().setSort(sortExec)) + .setInput(pb.PhysicalPlanNode.newBuilder().setBroadcastJoinBuildHashMap(buildHashMapExec)) .setIpcConsumerResourceId(writerIpcProviderResourceId) // build native sorter diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastJoinBase.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastJoinBase.scala index ec13b8fae..dc27d2aad 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastJoinBase.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastJoinBase.scala @@ -20,21 +20,24 @@ import scala.collection.immutable.SortedMap import org.apache.spark.OneToOneDependency import org.apache.spark.Partition -import org.apache.spark.sql.blaze.BlazeConf import org.apache.spark.sql.blaze.MetricNode import org.apache.spark.sql.blaze.NativeConverters import org.apache.spark.sql.blaze.NativeHelper import org.apache.spark.sql.blaze.NativeRDD import org.apache.spark.sql.blaze.NativeSupports +import org.apache.spark.sql.blaze.Shims +import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.LeftAnti -import org.apache.spark.sql.catalyst.plans.LeftSemi import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.BinaryExecNode +import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec +import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode +import org.apache.spark.sql.types.LongType import org.blaze.{protobuf => pb} +import org.blaze.protobuf.JoinOn abstract class NativeBroadcastJoinBase( override val left: SparkPlan, @@ -43,82 +46,114 @@ abstract class NativeBroadcastJoinBase( leftKeys: Seq[Expression], rightKeys: Seq[Expression], joinType: JoinType, - condition: Option[Expression]) + broadcastSide: BroadcastSide) extends BinaryExecNode with NativeSupports { - assert( - (joinType != LeftSemi && joinType != LeftAnti) || condition.isEmpty, - "Semi/Anti join with filter is not supported yet") - - assert( - !BlazeConf.BHJ_FALLBACKS_TO_SMJ_ENABLE.booleanConf() || BlazeConf.SMJ_INEQUALITY_JOIN_ENABLE - .booleanConf() || condition.isEmpty, - "Join filter is not supported when BhjFallbacksToSmj and SmjInequalityJoin both enabled") - override lazy val metrics: Map[String, SQLMetric] = SortedMap[String, SQLMetric]() ++ Map( NativeHelper .getDefaultNativeMetrics(sparkContext) .toSeq: _*) + private val isLongHashRelation = { + val baseBroadcast = broadcastSide match { + case BroadcastLeft => Shims.get.getUnderlyingBroadcast(left) + case BroadcastRight => Shims.get.getUnderlyingBroadcast(right) + } + val mode = baseBroadcast match { + case b: BroadcastExchangeExec => b.mode + case b: NativeBroadcastExchangeBase => b.broadcastMode + } + mode match { + case mode: HashedRelationBroadcastMode + if mode.key.length == 1 && mode.key.head.dataType == LongType => + true + case _ => false + } + } + + private def nativeSchema = Util.getNativeSchema(output) + private def nativeJoinOn = leftKeys.zip(rightKeys).map { case (leftKey, rightKey) => - val leftColumn = NativeConverters.convertExpr(leftKey).getColumn match { - case column if column.getName.isEmpty => - throw new NotImplementedError(s"BHJ leftKey is not column: ${leftKey}") - case column => column + val leftKeyExpr = leftKey match { + case k if !isLongHashRelation || k.dataType == LongType => k + case k => Cast(k, LongType) } - val rightColumn = NativeConverters.convertExpr(rightKey).getColumn match { - case column if column.getName.isEmpty => - throw new NotImplementedError(s"BHJ rightKey is not column: ${rightKey}") - case column => column + val rightKeyExpr = rightKey match { + case k if !isLongHashRelation || k.dataType == LongType => k + case k => Cast(k, LongType) } - pb.JoinOn + JoinOn .newBuilder() - .setLeft(leftColumn) - .setRight(rightColumn) + .setLeft(NativeConverters.convertExpr(leftKeyExpr)) + .setRight(NativeConverters.convertExpr(rightKeyExpr)) .build() } private def nativeJoinType = NativeConverters.convertJoinType(joinType) - private def nativeJoinFilter = - condition.map(NativeConverters.convertJoinFilter(_, left.output, right.output)) + private def nativeBroadcastSide = broadcastSide match { + case BroadcastLeft => pb.JoinSide.LEFT_SIDE + case BroadcastRight => pb.JoinSide.RIGHT_SIDE + } // check whether native converting is supported + nativeSchema nativeJoinType - nativeJoinFilter + nativeJoinOn + nativeBroadcastSide override def doExecuteNative(): NativeRDD = { val leftRDD = NativeHelper.executeNative(left) val rightRDD = NativeHelper.executeNative(right) val nativeMetrics = MetricNode(metrics, leftRDD.metrics :: rightRDD.metrics :: Nil) + val nativeSchema = this.nativeSchema val nativeJoinType = this.nativeJoinType val nativeJoinOn = this.nativeJoinOn - val nativeJoinFilter = this.nativeJoinFilter - val partitions = rightRDD.partitions + + val (probedRDD, builtRDD) = broadcastSide match { + case BroadcastLeft => (rightRDD, leftRDD) + case BroadcastRight => (leftRDD, rightRDD) + } new NativeRDD( sparkContext, nativeMetrics, - partitions, - rddDependencies = new OneToOneDependency(rightRDD) :: Nil, - rightRDD.isShuffleReadFull, + probedRDD.partitions, + rddDependencies = new OneToOneDependency(probedRDD) :: Nil, + probedRDD.isShuffleReadFull, (partition, context) => { val partition0 = new Partition() { override def index: Int = 0 } - val leftChild = leftRDD.nativePlan(partition0, context) - val rightChild = rightRDD.nativePlan(rightRDD.partitions(partition.index), context) + val (leftChild, rightChild) = broadcastSide match { + case BroadcastLeft => + ( + leftRDD.nativePlan(partition0, context), + rightRDD.nativePlan(rightRDD.partitions(partition.index), context)) + case BroadcastRight => + ( + leftRDD.nativePlan(leftRDD.partitions(partition.index), context), + rightRDD.nativePlan(partition0, context)) + } + val cachedBuildHashMapId = s"bhm_stage${context.stageId}_rdd${builtRDD.id}" + val broadcastJoinExec = pb.BroadcastJoinExecNode .newBuilder() + .setSchema(nativeSchema) .setLeft(leftChild) .setRight(rightChild) .setJoinType(nativeJoinType) + .setBroadcastSide(nativeBroadcastSide) + .setCachedBuildHashMapId(cachedBuildHashMapId) .addAllOn(nativeJoinOn.asJava) - nativeJoinFilter.foreach(joinFilter => broadcastJoinExec.setJoinFilter(joinFilter)) pb.PhysicalPlanNode.newBuilder().setBroadcastJoin(broadcastJoinExec).build() }, friendlyName = "NativeRDD.BroadcastJoin") } } + +class BroadcastSide {} +case object BroadcastLeft extends BroadcastSide {} +case object BroadcastRight extends BroadcastSide {} diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastNestedLoopJoinBase.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastNestedLoopJoinBase.scala deleted file mode 100644 index bfcf74752..000000000 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeBroadcastNestedLoopJoinBase.scala +++ /dev/null @@ -1,144 +0,0 @@ -/* - * Copyright 2022 The Blaze Authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution.blaze.plan - -import scala.collection.immutable.SortedMap - -import org.apache.spark.OneToOneDependency -import org.apache.spark.Partition -import org.apache.spark.sql.blaze.MetricNode -import org.apache.spark.sql.blaze.NativeConverters -import org.apache.spark.sql.blaze.NativeHelper -import org.apache.spark.sql.blaze.NativeRDD -import org.apache.spark.sql.blaze.NativeSupports -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.ExistenceJoin -import org.apache.spark.sql.catalyst.plans.FullOuter -import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.InnerLike -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.LeftAnti -import org.apache.spark.sql.catalyst.plans.LeftExistence -import org.apache.spark.sql.catalyst.plans.LeftOuter -import org.apache.spark.sql.catalyst.plans.LeftSemi -import org.apache.spark.sql.catalyst.plans.RightOuter -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.BinaryExecNode -import org.blaze.{protobuf => pb} - -abstract class NativeBroadcastNestedLoopJoinBase( - override val left: SparkPlan, - override val right: SparkPlan, - joinType: JoinType, - condition: Option[Expression]) - extends BinaryExecNode - with NativeSupports { - - override lazy val metrics: Map[String, SQLMetric] = SortedMap[String, SQLMetric]() ++ Map( - NativeHelper - .getDefaultNativeMetrics(sparkContext) - .filterKeys( - Set( - "stage_id", - "output_rows", - "elapsed_compute", - "input_batch_count", - "input_batch_mem_size", - "input_row_count")) - .toSeq: _*) - - override def output: Seq[Attribute] = { - joinType match { - case _: InnerLike => - left.output ++ right.output - case LeftOuter => - left.output ++ right.output.map(_.withNullability(true)) - case RightOuter => - left.output.map(_.withNullability(true)) ++ right.output - case FullOuter => - left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) - case j: ExistenceJoin => - left.output :+ j.exists - case LeftExistence(_) => - left.output - case x => - throw new IllegalArgumentException( - s"BroadcastNestedLoopJoin should not take $x as the JoinType") - } - } - - private def nativeJoinType = NativeConverters.convertJoinType(joinType) - private def nativeJoinFilter = - condition.map(NativeConverters.convertJoinFilter(_, left.output, right.output)) - - // check whether native converting is supported - nativeJoinType - nativeJoinFilter - - private val probedSide = joinType match { - case Inner | LeftOuter | LeftSemi | LeftAnti => "left" - case RightOuter | FullOuter => "right" - case other => s"NativeBroadcastNestedLoopJoin does not support join type $other" - } - - override def doExecuteNative(): NativeRDD = { - val leftRDD = NativeHelper.executeNative(left) - val rightRDD = NativeHelper.executeNative(right) - val nativeMetrics = MetricNode(metrics, leftRDD.metrics :: rightRDD.metrics :: Nil) - val nativeJoinType = this.nativeJoinType - val nativeJoinFilter = this.nativeJoinFilter - val partitions = probedSide match { - case "left" => leftRDD.partitions - case "right" => rightRDD.partitions - } - - new NativeRDD( - sparkContext, - nativeMetrics, - partitions, - rddDependencies = probedSide match { - case "left" => new OneToOneDependency(leftRDD) :: Nil - case "right" => new OneToOneDependency(rightRDD) :: Nil - }, - rightRDD.isShuffleReadFull, - (partition, context) => { - val partition0 = new Partition() { - override def index: Int = 0 - } - val (leftChild, rightChild) = probedSide match { - case "left" => - ( - leftRDD.nativePlan(leftRDD.partitions(partition.index), context), - rightRDD.nativePlan(partition0, context)) - case "right" => - ( - leftRDD.nativePlan(partition0, context), - rightRDD.nativePlan(rightRDD.partitions(partition.index), context)) - } - val bnlj = pb.BroadcastNestedLoopJoinExecNode - .newBuilder() - .setLeft(leftChild) - .setRight(rightChild) - .setJoinType(nativeJoinType) - - nativeJoinFilter.foreach(joinFilter => bnlj.setJoinFilter(joinFilter)) - pb.PhysicalPlanNode.newBuilder().setBroadcastNestedLoopJoin(bnlj).build() - }, - friendlyName = "NativeRDD.BroadcastNestedLoopJoin") - } -} diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeGenerateBase.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeGenerateBase.scala index 2349cc9e8..dc0e371a4 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeGenerateBase.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeGenerateBase.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.OneToOneDependency import org.apache.spark.sql.blaze.MetricNode import org.apache.spark.sql.blaze.NativeConverters -import org.apache.spark.sql.blaze.NativeConverters.convertExprWithFallback import org.apache.spark.sql.blaze.NativeHelper import org.apache.spark.sql.blaze.NativeRDD import org.apache.spark.sql.blaze.NativeSupports diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeParquetScanBase.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeParquetScanBase.scala index 1bbfdc3a5..276fd532d 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeParquetScanBase.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeParquetScanBase.scala @@ -167,7 +167,7 @@ abstract class NativeParquetScanBase(basedFileScan: FileSourceScanExec) partitions.asInstanceOf[Array[Partition]], Nil, rddShuffleReadFull = true, - (partition, context) => { + (partition, _context) => { val resourceId = s"NativeParquetScanExec:${UUID.randomUUID().toString}" val sharedConf = broadcastedHadoopConf.value.value JniBridge.resourcesMap.put( diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeParquetSinkBase.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeParquetSinkBase.scala index 8e81d43f7..cd2e5a3ff 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeParquetSinkBase.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeParquetSinkBase.scala @@ -151,6 +151,6 @@ abstract class NativeParquetSinkBase( "ParquetSink") } - protected def newHadoopConf(tableDesc: TableDesc): Configuration = + protected def newHadoopConf(_tableDesc: TableDesc): Configuration = sparkSession.sessionState.newHadoopConf() } diff --git a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeSortMergeJoinBase.scala b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeSortMergeJoinBase.scala index 52efbcdc0..831211b06 100644 --- a/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeSortMergeJoinBase.scala +++ b/spark-extension/src/main/scala/org/apache/spark/sql/execution/blaze/plan/NativeSortMergeJoinBase.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.plans.RightOuter import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.OneToOneDependency -import org.apache.spark.sql.blaze.BlazeConf import org.apache.spark.sql.blaze.MetricNode import org.apache.spark.sql.blaze.NativeConverters import org.apache.spark.sql.blaze.NativeHelper @@ -52,13 +51,7 @@ abstract class NativeSortMergeJoinBase( extends BinaryExecNode with NativeSupports { - assert( - (joinType != LeftSemi && joinType != LeftAnti) || condition.isEmpty, - "Semi/Anti join with filter is not supported yet") - - assert( - BlazeConf.SMJ_INEQUALITY_JOIN_ENABLE.booleanConf() || condition.isEmpty, - "inequality sort-merge join is not enabled") + assert(condition.isEmpty, "inequality join is not supported") override lazy val metrics: Map[String, SQLMetric] = SortedMap[String, SQLMetric]() ++ Map( NativeHelper @@ -81,21 +74,15 @@ abstract class NativeSortMergeJoinBase( keys.map(SortOrder(_, Ascending)) } + private def nativeSchema = Util.getNativeSchema(output) + private def nativeJoinOn = leftKeys.zip(rightKeys).map { case (leftKey, rightKey) => - val leftColumn = NativeConverters.convertExpr(leftKey).getColumn match { - case column if column.getName.isEmpty => - throw new NotImplementedError(s"SMJ leftKey is not column: ${leftKey}") - case column => column - } - val rightColumn = NativeConverters.convertExpr(rightKey).getColumn match { - case column if column.getName.isEmpty => - throw new NotImplementedError(s"SMJ rightKey is not column: ${rightKey}") - case column => column - } + val leftKeyExpr = NativeConverters.convertExpr(leftKey) + val rightKeyExpr = NativeConverters.convertExpr(rightKey) JoinOn .newBuilder() - .setLeft(leftColumn) - .setRight(rightColumn) + .setLeft(leftKeyExpr) + .setRight(rightKeyExpr) .build() } @@ -109,14 +96,11 @@ abstract class NativeSortMergeJoinBase( private def nativeJoinType = NativeConverters.convertJoinType(joinType) - private def nativeJoinFilter = - condition.map(NativeConverters.convertJoinFilter(_, left.output, right.output)) - // check whether native converting is supported + nativeSchema nativeSortOptions nativeJoinOn nativeJoinType - nativeJoinFilter override def doExecuteNative(): NativeRDD = { val leftRDD = NativeHelper.executeNative(left) @@ -125,7 +109,6 @@ abstract class NativeSortMergeJoinBase( val nativeSortOptions = this.nativeSortOptions val nativeJoinOn = this.nativeJoinOn val nativeJoinType = this.nativeJoinType - val nativeJoinFilter = this.nativeJoinFilter val partitions = if (joinType != RightOuter) { leftRDD.partitions @@ -161,13 +144,12 @@ abstract class NativeSortMergeJoinBase( val sortMergeJoinExec = SortMergeJoinExecNode .newBuilder() + .setSchema(nativeSchema) .setLeft(leftChild) .setRight(rightChild) .setJoinType(nativeJoinType) .addAllOn(nativeJoinOn.asJava) .addAllSortOptions(nativeSortOptions.asJava) - - nativeJoinFilter.foreach(joinFilter => sortMergeJoinExec.setJoinFilter(joinFilter)) PhysicalPlanNode.newBuilder().setSortMergeJoin(sortMergeJoinExec).build() }, friendlyName = "NativeRDD.SortMergeJoin")