From 2b95ac4431b00fd87cd95a25426b3beefc584842 Mon Sep 17 00:00:00 2001 From: Chao Sun <sunchao@apple.com> Date: Tue, 12 Oct 2021 13:52:56 -0700 Subject: [PATCH 1/4] Initial PR Co-authored-by: Liang-Chi Hsieh <liangchi@apple.com> Co-authored-by: Kazuyuki Tanimura <ktanimura@apple.com> Co-authored-by: Steve Vaughan Jr <s_vaughan@apple.com> Co-authored-by: Huaxin Gao <huaxin_gao@apple.com> Co-authored-by: Parth Chandra <parthc@apple.com> Co-authored-by: Oleksandr Voievodin <ovoievodin@apple.com> --- .gitignore | 10 + .scalafix.conf | 27 + DEBUGGING.md | 96 + DEVELOPMENT.md | 65 + EXPRESSIONS.md | 96 + Makefile | 68 + README.md | 61 +- bin/comet-spark-shell | 84 + common/pom.xml | 200 + .../org/apache/arrow/c/ArrowImporter.java | 61 + .../apache/comet/CometRuntimeException.java | 31 + .../java/org/apache/comet/NativeBase.java | 278 ++ .../apache/comet/ParquetRuntimeException.java | 31 + .../comet/parquet/AbstractColumnReader.java | 116 + .../org/apache/comet/parquet/BatchReader.java | 620 +++ .../comet/parquet/BloomFilterReader.java | 253 ++ .../comet/parquet/ColumnIndexReader.java | 230 ++ .../comet/parquet/ColumnPageReader.java | 252 ++ .../apache/comet/parquet/ColumnReader.java | 314 ++ .../apache/comet/parquet/CometInputFile.java | 157 + .../comet/parquet/ConstantColumnReader.java | 126 + .../comet/parquet/DictionaryPageReader.java | 190 + .../org/apache/comet/parquet/FileReader.java | 1151 ++++++ .../apache/comet/parquet/FooterReader.java | 56 + .../org/apache/comet/parquet/IndexFilter.java | 145 + .../comet/parquet/LazyColumnReader.java | 186 + .../comet/parquet/MetadataColumnReader.java | 80 + .../java/org/apache/comet/parquet/Native.java | 235 ++ .../org/apache/comet/parquet/ReadOptions.java | 197 + .../apache/comet/parquet/RowGroupFilter.java | 113 + .../apache/comet/parquet/RowGroupReader.java | 72 + .../comet/parquet/RowIndexColumnReader.java | 49 + .../apache/comet/parquet/SupportsComet.java | 24 + .../org/apache/comet/parquet/TypeUtil.java | 279 ++ .../java/org/apache/comet/parquet/Utils.java | 238 ++ .../comet/vector/CometDecodedVector.java | 106 + .../comet/vector/CometDelegateVector.java | 169 + .../apache/comet/vector/CometDictionary.java | 178 + .../comet/vector/CometDictionaryVector.java | 135 + .../apache/comet/vector/CometLazyVector.java | 82 + .../apache/comet/vector/CometListVector.java | 57 + .../apache/comet/vector/CometPlainVector.java | 172 + .../comet/vector/CometStructVector.java | 61 + .../org/apache/comet/vector/CometVector.java | 232 ++ common/src/main/resources/log4j2.properties | 40 + .../scala/org/apache/comet/CometConf.scala | 454 ++ .../scala/org/apache/comet/Constants.scala | 25 + .../main/scala/org/apache/comet/package.scala | 87 + .../comet/parquet/CometParquetUtils.scala | 99 + .../comet/parquet/CometReaderThreadPool.scala | 65 + .../apache/comet/shims/ShimBatchReader.scala | 54 + .../apache/comet/shims/ShimFileFormat.scala | 35 + .../shims/ShimResolveDefaultColumns.scala | 38 + .../org/apache/comet/vector/NativeUtil.scala | 141 + .../apache/comet/vector/StreamReader.scala | 70 + .../parquet/CometParquetReadSupport.scala | 378 ++ .../CometSparkToParquetSchemaConverter.scala | 322 ++ .../apache/spark/sql/comet/util/Utils.scala | 164 + .../comet/parquet/TestColumnReader.java | 193 + .../comet/parquet/TestCometInputFile.java | 44 + .../apache/comet/parquet/TestFileReader.java | 829 ++++ common/src/test/resources/log4j.properties | 36 + common/src/test/resources/log4j2.properties | 40 + conf/log4rs.yaml | 26 + core/.lldbinit | 37 + core/Cargo.lock | 3191 ++++++++++++++ core/Cargo.toml | 115 + core/benches/bit_util.rs | 182 + core/benches/common.rs | 83 + core/benches/hash.rs | 109 + core/benches/parquet_read.rs | 217 + core/benches/perf.rs | 61 + core/build.rs | 39 + core/rustfmt.toml | 27 + core/src/common/bit.rs | 1648 ++++++++ core/src/common/buffer.rs | 342 ++ core/src/common/mod.rs | 44 + core/src/common/mutable_vector.rs | 409 ++ core/src/common/vector.rs | 523 +++ core/src/data_type.rs | 241 ++ core/src/errors.rs | 820 ++++ .../execution/datafusion/expressions/avg.rs | 340 ++ .../datafusion/expressions/avg_decimal.rs | 534 +++ .../datafusion/expressions/bitwise_not.rs | 194 + .../execution/datafusion/expressions/cast.rs | 156 + .../datafusion/expressions/checkoverflow.rs | 190 + .../datafusion/expressions/if_expr.rs | 221 + .../execution/datafusion/expressions/mod.rs | 33 + .../datafusion/expressions/normalize_nan.rs | 167 + .../datafusion/expressions/scalar_funcs.rs | 490 +++ .../datafusion/expressions/strings.rs | 305 ++ .../datafusion/expressions/subquery.rs | 211 + .../datafusion/expressions/sum_decimal.rs | 455 ++ .../datafusion/expressions/temporal.rs | 518 +++ .../execution/datafusion/expressions/utils.rs | 268 ++ core/src/execution/datafusion/mod.rs | 23 + .../execution/datafusion/operators/expand.rs | 205 + .../src/execution/datafusion/operators/mod.rs | 18 + core/src/execution/datafusion/planner.rs | 1192 ++++++ core/src/execution/datafusion/spark_hash.rs | 473 +++ core/src/execution/jni_api.rs | 507 +++ core/src/execution/kernels/hash.rs | 171 + core/src/execution/kernels/mod.rs | 24 + core/src/execution/kernels/strings.rs | 243 ++ core/src/execution/kernels/temporal.rs | 438 ++ core/src/execution/metrics/mod.rs | 18 + core/src/execution/metrics/utils.rs | 69 + core/src/execution/mod.rs | 55 + core/src/execution/operators/copy.rs | 163 + core/src/execution/operators/mod.rs | 104 + core/src/execution/operators/scan.rs | 271 ++ core/src/execution/proto/expr.proto | 454 ++ core/src/execution/proto/operator.proto | 82 + core/src/execution/proto/partitioning.proto | 42 + core/src/execution/serde.rs | 147 + core/src/execution/timezone.rs | 143 + core/src/execution/utils.rs | 120 + core/src/jvm_bridge/comet_exec.rs | 101 + core/src/jvm_bridge/comet_metric_node.rs | 59 + core/src/jvm_bridge/mod.rs | 213 + core/src/lib.rs | 137 + core/src/parquet/compression.rs | 319 ++ core/src/parquet/data_type.rs | 111 + core/src/parquet/mod.rs | 562 +++ core/src/parquet/mutable_vector.rs | 251 ++ core/src/parquet/read/column.rs | 828 ++++ core/src/parquet/read/levels.rs | 230 ++ core/src/parquet/read/mod.rs | 114 + core/src/parquet/read/values.rs | 1008 +++++ core/src/parquet/util/bit_packing.rs | 3658 +++++++++++++++++ core/src/parquet/util/buffer.rs | 128 + core/src/parquet/util/hash_util.rs | 167 + core/src/parquet/util/jni.rs | 199 + core/src/parquet/util/jni_buffer.rs | 98 + core/src/parquet/util/memory.rs | 557 +++ core/src/parquet/util/mod.rs | 28 + .../src/parquet/util/test_common/file_util.rs | 53 + core/src/parquet/util/test_common/mod.rs | 24 + .../src/parquet/util/test_common/page_util.rs | 317 ++ core/src/parquet/util/test_common/rand_gen.rs | 57 + dev/cargo.config | 27 + dev/checkstyle-suppressions.xml | 34 + dev/ensure-jars-have-correct-contents.sh | 114 + dev/scalastyle-config.xml | 374 ++ doc/comet-overview.png | Bin 0 -> 172624 bytes doc/comet-plan.png | Bin 0 -> 310869 bytes pom.xml | 911 ++++ rust-toolchain | 1 + scalafmt.conf | 41 + spark-integration/pom.xml | 106 + spark/README.md | 22 + spark/pom.xml | 238 ++ .../filter2/predicate/SparkFilterApi.java | 54 + .../spark/sql/comet/CometScalarSubquery.java | 120 + spark/src/main/resources/log4j2.properties | 40 + .../org/apache/comet/CometExecIterator.scala | 230 ++ .../comet/CometSparkSessionExtensions.scala | 472 +++ .../org/apache/comet/MetricsSupport.scala | 56 + .../main/scala/org/apache/comet/Native.scala | 107 + .../parquet/CometParquetFileFormat.scala | 231 ++ .../CometParquetPartitionReaderFactory.scala | 231 ++ .../comet/parquet/CometParquetScan.scala | 84 + .../apache/comet/parquet/ParquetFilters.scala | 882 ++++ .../apache/comet/serde/QueryPlanSerde.scala | 1806 ++++++++ .../comet/shims/ShimCometBatchScanExec.scala | 45 + .../comet/shims/ShimCometScanExec.scala | 124 + .../ShimCometSparkSessionExtensions.scala | 35 + .../comet/shims/ShimQueryPlanSerde.scala | 47 + .../org/apache/comet/shims/ShimSQLConf.scala | 42 + .../main/scala/org/apache/spark/Plugins.scala | 115 + .../spark/sql/comet/CometBatchScanExec.scala | 154 + .../spark/sql/comet/CometCoalesceExec.scala | 86 + .../spark/sql/comet/CometMetricNode.scala | 96 + .../apache/spark/sql/comet/CometPlan.scala | 25 + .../spark/sql/comet/CometScanExec.scala | 458 +++ .../spark/sql/comet/DataSourceHelpers.scala | 31 + .../spark/sql/comet/DecimalPrecision.scala | 117 + .../apache/spark/sql/comet/operators.scala | 473 +++ .../apache/comet/IntegrationTestSuite.java | 36 + spark/src/test/resources/log4j.properties | 36 + spark/src/test/resources/log4j2.properties | 40 + .../before_1582_date_v2_4_5.snappy.parquet | Bin 0 -> 660 bytes .../before_1582_date_v2_4_6.snappy.parquet | Bin 0 -> 694 bytes .../before_1582_date_v3_2_0.snappy.parquet | Bin 0 -> 822 bytes ...timestamp_int96_dict_v2_4_5.snappy.parquet | Bin 0 -> 737 bytes ...timestamp_int96_dict_v2_4_6.snappy.parquet | Bin 0 -> 771 bytes ...timestamp_int96_dict_v3_2_0.snappy.parquet | Bin 0 -> 842 bytes ...imestamp_int96_plain_v2_4_5.snappy.parquet | Bin 0 -> 693 bytes ...imestamp_int96_plain_v2_4_6.snappy.parquet | Bin 0 -> 727 bytes ...imestamp_int96_plain_v3_2_0.snappy.parquet | Bin 0 -> 779 bytes ...582_timestamp_micros_v2_4_5.snappy.parquet | Bin 0 -> 767 bytes ...582_timestamp_micros_v2_4_6.snappy.parquet | Bin 0 -> 801 bytes ...582_timestamp_micros_v3_2_0.snappy.parquet | Bin 0 -> 929 bytes ...582_timestamp_millis_v2_4_5.snappy.parquet | Bin 0 -> 761 bytes ...582_timestamp_millis_v2_4_6.snappy.parquet | Bin 0 -> 795 bytes ...582_timestamp_millis_v3_2_0.snappy.parquet | Bin 0 -> 922 bytes .../test-data/dec-in-fixed-len.parquet | Bin 0 -> 460 bytes ...al32-written-as-64-bit-dict.snappy.parquet | Bin 0 -> 6795 bytes ...decimal32-written-as-64-bit.snappy.parquet | Bin 0 -> 383 bytes spark/src/test/resources/tpch-extended/q1.sql | 9 + .../apache/comet/CometExpressionSuite.scala | 1262 ++++++ .../CometSparkSessionExtensionsSuite.scala | 145 + .../scala/org/apache/comet/TestUtils.scala | 44 + .../comet/exec/CometAggregateSuite.scala | 740 ++++ .../apache/comet/exec/CometExecSuite.scala | 804 ++++ .../comet/parquet/ParquetReadSuite.scala | 1351 ++++++ .../org/apache/spark/CometPluginsSuite.scala | 100 + .../spark/sql/CometTPCDSQueriesList.scala | 74 + .../spark/sql/CometTPCDSQuerySuite.scala | 163 + .../spark/sql/CometTPCHQueriesList.scala | 68 + .../spark/sql/CometTPCHQuerySuite.scala | 297 ++ .../apache/spark/sql/CometTPCQueryBase.scala | 126 + .../spark/sql/CometTPCQueryListBase.scala | 112 + .../org/apache/spark/sql/CometTestBase.scala | 597 +++ .../org/apache/spark/sql/GenTPCHData.scala | 272 ++ .../org/apache/spark/sql/TPCDSQueries.scala | 164 + .../scala/org/apache/spark/sql/TPCH.scala | 174 + .../scala/org/apache/spark/sql/Tables.scala | 238 ++ .../benchmark/CometAggregateBenchmark.scala | 246 ++ .../benchmark/CometArithmeticBenchmark.scala | 145 + .../sql/benchmark/CometBenchmarkBase.scala | 136 + .../CometConditionalExpressionBenchmark.scala | 111 + .../CometDatetimeExpressionBenchmark.scala | 103 + .../sql/benchmark/CometExecBenchmark.scala | 230 ++ .../CometPredicateExpressionBenchmark.scala | 77 + .../sql/benchmark/CometReadBenchmark.scala | 437 ++ .../CometStringExpressionBenchmark.scala | 632 +++ .../benchmark/CometTPCDSQueryBenchmark.scala | 83 + .../benchmark/CometTPCHQueryBenchmark.scala | 138 + .../CometTPCQueryBenchmarkBase.scala | 87 + .../sql/comet/CometPlanStabilitySuite.scala | 315 ++ .../comet/ParquetDatetimeRebaseSuite.scala | 145 + .../sql/comet/ParquetEncryptionITCase.scala | 189 + 233 files changed, 54828 insertions(+), 1 deletion(-) create mode 100644 .gitignore create mode 100644 .scalafix.conf create mode 100644 DEBUGGING.md create mode 100644 DEVELOPMENT.md create mode 100644 EXPRESSIONS.md create mode 100644 Makefile create mode 100755 bin/comet-spark-shell create mode 100644 common/pom.xml create mode 100644 common/src/main/java/org/apache/arrow/c/ArrowImporter.java create mode 100644 common/src/main/java/org/apache/comet/CometRuntimeException.java create mode 100644 common/src/main/java/org/apache/comet/NativeBase.java create mode 100644 common/src/main/java/org/apache/comet/ParquetRuntimeException.java create mode 100644 common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/BatchReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/ColumnReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/CometInputFile.java create mode 100644 common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/FileReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/FooterReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/IndexFilter.java create mode 100644 common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/Native.java create mode 100644 common/src/main/java/org/apache/comet/parquet/ReadOptions.java create mode 100644 common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java create mode 100644 common/src/main/java/org/apache/comet/parquet/RowGroupReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java create mode 100644 common/src/main/java/org/apache/comet/parquet/SupportsComet.java create mode 100644 common/src/main/java/org/apache/comet/parquet/TypeUtil.java create mode 100644 common/src/main/java/org/apache/comet/parquet/Utils.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometDecodedVector.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometDelegateVector.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometDictionary.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometLazyVector.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometListVector.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometPlainVector.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometStructVector.java create mode 100644 common/src/main/java/org/apache/comet/vector/CometVector.java create mode 100644 common/src/main/resources/log4j2.properties create mode 100644 common/src/main/scala/org/apache/comet/CometConf.scala create mode 100644 common/src/main/scala/org/apache/comet/Constants.scala create mode 100644 common/src/main/scala/org/apache/comet/package.scala create mode 100644 common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala create mode 100644 common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala create mode 100644 common/src/main/scala/org/apache/comet/shims/ShimBatchReader.scala create mode 100644 common/src/main/scala/org/apache/comet/shims/ShimFileFormat.scala create mode 100644 common/src/main/scala/org/apache/comet/shims/ShimResolveDefaultColumns.scala create mode 100644 common/src/main/scala/org/apache/comet/vector/NativeUtil.scala create mode 100644 common/src/main/scala/org/apache/comet/vector/StreamReader.scala create mode 100644 common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala create mode 100644 common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala create mode 100644 common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala create mode 100644 common/src/test/java/org/apache/comet/parquet/TestColumnReader.java create mode 100644 common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java create mode 100644 common/src/test/java/org/apache/comet/parquet/TestFileReader.java create mode 100644 common/src/test/resources/log4j.properties create mode 100644 common/src/test/resources/log4j2.properties create mode 100644 conf/log4rs.yaml create mode 100644 core/.lldbinit create mode 100644 core/Cargo.lock create mode 100644 core/Cargo.toml create mode 100644 core/benches/bit_util.rs create mode 100644 core/benches/common.rs create mode 100644 core/benches/hash.rs create mode 100644 core/benches/parquet_read.rs create mode 100644 core/benches/perf.rs create mode 100644 core/build.rs create mode 100644 core/rustfmt.toml create mode 100644 core/src/common/bit.rs create mode 100644 core/src/common/buffer.rs create mode 100644 core/src/common/mod.rs create mode 100644 core/src/common/mutable_vector.rs create mode 100644 core/src/common/vector.rs create mode 100644 core/src/data_type.rs create mode 100644 core/src/errors.rs create mode 100644 core/src/execution/datafusion/expressions/avg.rs create mode 100644 core/src/execution/datafusion/expressions/avg_decimal.rs create mode 100644 core/src/execution/datafusion/expressions/bitwise_not.rs create mode 100644 core/src/execution/datafusion/expressions/cast.rs create mode 100644 core/src/execution/datafusion/expressions/checkoverflow.rs create mode 100644 core/src/execution/datafusion/expressions/if_expr.rs create mode 100644 core/src/execution/datafusion/expressions/mod.rs create mode 100644 core/src/execution/datafusion/expressions/normalize_nan.rs create mode 100644 core/src/execution/datafusion/expressions/scalar_funcs.rs create mode 100644 core/src/execution/datafusion/expressions/strings.rs create mode 100644 core/src/execution/datafusion/expressions/subquery.rs create mode 100644 core/src/execution/datafusion/expressions/sum_decimal.rs create mode 100644 core/src/execution/datafusion/expressions/temporal.rs create mode 100644 core/src/execution/datafusion/expressions/utils.rs create mode 100644 core/src/execution/datafusion/mod.rs create mode 100644 core/src/execution/datafusion/operators/expand.rs create mode 100644 core/src/execution/datafusion/operators/mod.rs create mode 100644 core/src/execution/datafusion/planner.rs create mode 100644 core/src/execution/datafusion/spark_hash.rs create mode 100644 core/src/execution/jni_api.rs create mode 100644 core/src/execution/kernels/hash.rs create mode 100644 core/src/execution/kernels/mod.rs create mode 100644 core/src/execution/kernels/strings.rs create mode 100644 core/src/execution/kernels/temporal.rs create mode 100644 core/src/execution/metrics/mod.rs create mode 100644 core/src/execution/metrics/utils.rs create mode 100644 core/src/execution/mod.rs create mode 100644 core/src/execution/operators/copy.rs create mode 100644 core/src/execution/operators/mod.rs create mode 100644 core/src/execution/operators/scan.rs create mode 100644 core/src/execution/proto/expr.proto create mode 100644 core/src/execution/proto/operator.proto create mode 100644 core/src/execution/proto/partitioning.proto create mode 100644 core/src/execution/serde.rs create mode 100644 core/src/execution/timezone.rs create mode 100644 core/src/execution/utils.rs create mode 100644 core/src/jvm_bridge/comet_exec.rs create mode 100644 core/src/jvm_bridge/comet_metric_node.rs create mode 100644 core/src/jvm_bridge/mod.rs create mode 100644 core/src/lib.rs create mode 100644 core/src/parquet/compression.rs create mode 100644 core/src/parquet/data_type.rs create mode 100644 core/src/parquet/mod.rs create mode 100644 core/src/parquet/mutable_vector.rs create mode 100644 core/src/parquet/read/column.rs create mode 100644 core/src/parquet/read/levels.rs create mode 100644 core/src/parquet/read/mod.rs create mode 100644 core/src/parquet/read/values.rs create mode 100644 core/src/parquet/util/bit_packing.rs create mode 100644 core/src/parquet/util/buffer.rs create mode 100644 core/src/parquet/util/hash_util.rs create mode 100644 core/src/parquet/util/jni.rs create mode 100644 core/src/parquet/util/jni_buffer.rs create mode 100644 core/src/parquet/util/memory.rs create mode 100644 core/src/parquet/util/mod.rs create mode 100644 core/src/parquet/util/test_common/file_util.rs create mode 100644 core/src/parquet/util/test_common/mod.rs create mode 100644 core/src/parquet/util/test_common/page_util.rs create mode 100644 core/src/parquet/util/test_common/rand_gen.rs create mode 100644 dev/cargo.config create mode 100644 dev/checkstyle-suppressions.xml create mode 100644 dev/ensure-jars-have-correct-contents.sh create mode 100644 dev/scalastyle-config.xml create mode 100644 doc/comet-overview.png create mode 100644 doc/comet-plan.png create mode 100644 pom.xml create mode 100644 rust-toolchain create mode 100644 scalafmt.conf create mode 100644 spark-integration/pom.xml create mode 100644 spark/README.md create mode 100644 spark/pom.xml create mode 100644 spark/src/main/java/org/apache/parquet/filter2/predicate/SparkFilterApi.java create mode 100644 spark/src/main/java/org/apache/spark/sql/comet/CometScalarSubquery.java create mode 100644 spark/src/main/resources/log4j2.properties create mode 100644 spark/src/main/scala/org/apache/comet/CometExecIterator.scala create mode 100644 spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala create mode 100644 spark/src/main/scala/org/apache/comet/MetricsSupport.scala create mode 100644 spark/src/main/scala/org/apache/comet/Native.scala create mode 100644 spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala create mode 100644 spark/src/main/scala/org/apache/comet/parquet/CometParquetPartitionReaderFactory.scala create mode 100644 spark/src/main/scala/org/apache/comet/parquet/CometParquetScan.scala create mode 100644 spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala create mode 100644 spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala create mode 100644 spark/src/main/scala/org/apache/comet/shims/ShimCometBatchScanExec.scala create mode 100644 spark/src/main/scala/org/apache/comet/shims/ShimCometScanExec.scala create mode 100644 spark/src/main/scala/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala create mode 100644 spark/src/main/scala/org/apache/comet/shims/ShimQueryPlanSerde.scala create mode 100644 spark/src/main/scala/org/apache/comet/shims/ShimSQLConf.scala create mode 100644 spark/src/main/scala/org/apache/spark/Plugins.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometPlan.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/DataSourceHelpers.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/DecimalPrecision.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/operators.scala create mode 100644 spark/src/test/java/org/apache/comet/IntegrationTestSuite.java create mode 100644 spark/src/test/resources/log4j.properties create mode 100644 spark/src/test/resources/log4j2.properties create mode 100644 spark/src/test/resources/test-data/before_1582_date_v2_4_5.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_date_v2_4_6.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_date_v3_2_0.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v2_4_6.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v3_2_0.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v2_4_5.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v2_4_6.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v3_2_0.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_5.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_6.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_micros_v3_2_0.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_5.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_6.snappy.parquet create mode 100644 spark/src/test/resources/test-data/before_1582_timestamp_millis_v3_2_0.snappy.parquet create mode 100644 spark/src/test/resources/test-data/dec-in-fixed-len.parquet create mode 100644 spark/src/test/resources/test-data/decimal32-written-as-64-bit-dict.snappy.parquet create mode 100644 spark/src/test/resources/test-data/decimal32-written-as-64-bit.snappy.parquet create mode 100644 spark/src/test/resources/tpch-extended/q1.sql create mode 100644 spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala create mode 100644 spark/src/test/scala/org/apache/comet/CometSparkSessionExtensionsSuite.scala create mode 100644 spark/src/test/scala/org/apache/comet/TestUtils.scala create mode 100644 spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala create mode 100644 spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala create mode 100644 spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala create mode 100644 spark/src/test/scala/org/apache/spark/CometPluginsSuite.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueriesList.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometTPCHQueriesList.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometTPCQueryBase.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometTPCQueryListBase.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/GenTPCHData.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/TPCDSQueries.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/TPCH.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/Tables.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometAggregateBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometArithmeticBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometConditionalExpressionBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometDatetimeExpressionBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometPredicateExpressionBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometStringExpressionBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSQueryBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCHQueryBenchmark.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCQueryBenchmarkBase.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000000..4eaf665136 --- /dev/null +++ b/.gitignore @@ -0,0 +1,10 @@ +target +.idea +*.iml +derby.log +metastore_db/ +spark-warehouse/ +dependency-reduced-pom.xml +core/src/execution/generated +prebuild +.flattened-pom.xml diff --git a/.scalafix.conf b/.scalafix.conf new file mode 100644 index 0000000000..0f23f57cdb --- /dev/null +++ b/.scalafix.conf @@ -0,0 +1,27 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. +rules = [ + ExplicitResultTypes, + NoAutoTupling, + RemoveUnused, + + DisableSyntax, + LeakingImplicitClassVal, + NoValInForComprehension, + ProcedureSyntax, + RedundantSyntax +] diff --git a/DEBUGGING.md b/DEBUGGING.md new file mode 100644 index 0000000000..25a6e17d3c --- /dev/null +++ b/DEBUGGING.md @@ -0,0 +1,96 @@ +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +# Comet Debugging Guide + +This HOWTO describes how to debug JVM code and Native code concurrently. The guide assumes you have: +1. Intellij as the Java IDE +2. CLion as the Native IDE. For Rust code, the CLion Rust language plugin is required. Note that the +Intellij Rust plugin is not sufficient. +3. CLion/LLDB as the native debugger. CLion ships with a bundled LLDB and the Rust community has +its own packaging of LLDB (`lldb-rust`). Both provide a better display of Rust symbols than plain +LLDB or the LLDB that is bundled with XCode. We will use the LLDB packaged with CLion for this guide. +4. We will use a Comet _unit_ test as the canonical use case. + +_Caveat: The steps here have only been tested with JDK 11_ on Mac (M1) + +## Debugging for Advanced Developers + +Add a `.lldbinit` to comet/core. This is not strictly necessary but will be useful if you want to +use advanced `lldb` debugging. A sample `.lldbinit` is provided in the comet/core directory + +### In Intellij + +1. Set a breakpoint in `NativeBase.load()`, at a point _after_ the Comet library has been loaded. + +1. Add a Debug Configuration for the unit test + +1. In the Debug Configuration for that unit test add `-Xint` as a JVM parameter. This option is +undocumented *magic*. Without this, the LLDB debugger hits a EXC_BAD_ACCESS (or EXC_BAD_INSTRUCTION) from +which one cannot recover. + +1. Add a println to the unit test to print the PID of the JVM process. (jps can also be used but this is less error prone if you have multiple jvm processes running) + ``` JDK8 + println("Waiting for Debugger: PID - ", ManagementFactory.getRuntimeMXBean().getName()) + ``` + This will print something like : `PID@your_machine_name`. + + For JDK9 and newer + ```JDK9 + println("Waiting for Debugger: PID - ", ProcessHandle.current.pid) + ``` + + ==> Note the PID + +1. Debug-run the test in Intellij and wait for the breakpoint to be hit + +### In CLion + +1. After the breakpoint is hit in Intellij, in Clion (or LLDB from terminal or editor) - + + 1. Attach to the jvm process (make sure the PID matches). In CLion, this is `Run -> Atttach to process` + + 1. Put your breakpoint in the native code + +1. Go back to intellij and resume the process. + +1. Most debugging in CLion is similar to Intellij. For advanced LLDB based debugging the LLDB command line can be accessed from the LLDB tab in the Debugger view. Refer to the [LLDB manual](https://lldb.llvm.org/use/tutorial.html) for LLDB commands. + +### After your debugging is done, + +1. In CLion, detach from the process if not already detached + +2. In Intellij, the debugger might have lost track of the process. If so, the debugger tab + will show the process as running (even if the test/job is shown as completed). + +3. Close the debugger tab, and if the IDS asks whether it should terminate the process, + click Yes. + +4. In terminal, use jps to identify the process with the process id you were debugging. If + it shows up as running, kill -9 [pid]. If that doesn't remove the process, don't bother, + the process will be left behind as a zombie and will consume no (significant) resources. + Eventually it will be cleaned up when you reboot possibly after a software update. + +### Additional Info + +OpenJDK mailing list on debugging the JDK on MacOS +https://mail.openjdk.org/pipermail/hotspot-dev/2019-September/039429.html + +Detecting the debugger +https://stackoverflow.com/questions/5393403/can-a-java-application-detect-that-a-debugger-is-attached#:~:text=No.,to%20let%20your%20app%20continue.&text=I%20know%20that%20those%20are,meant%20with%20my%20first%20phrase). diff --git a/DEVELOPMENT.md b/DEVELOPMENT.md new file mode 100644 index 0000000000..1793bb9e2e --- /dev/null +++ b/DEVELOPMENT.md @@ -0,0 +1,65 @@ +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +# Comet Development Guide + +## Project Layout + +``` +├── common <- common Java/Scala code +├── conf <- configuration files +├── core <- core native code, in Rust +├── spark <- Spark integration +``` + +## Development Setup + +1. Make sure `JAVA_HOME` is set and point to JDK 11 installation. +2. Install Rust toolchain. The easiest way is to use + [rustup](https://rustup.rs). + +## Build & Test + +A few common commands are specified in project's `Makefile`: + +- `make`: compile the entire project, but don't run tests +- `make test`: compile the project and run tests in both Rust and Java + side. +- `make release`: compile the project and creates a release build. This + is useful when you want to test Comet local installation in another project + such as Spark. +- `make clean`: clean up the workspace +- `bin/comet-spark-shell -d . -o spark/target/` run Comet spark shell for V1 datasources +- `bin/comet-spark-shell -d . -o spark/target/ --conf spark.sql.sources.useV1SourceList=""` run Comet spark shell for V2 datasources + +## Benchmark + +There's a `make` command to run micro benchmarks in the repo. For +instance: + +``` +make benchmark-org.apache.spark.sql.benchmark.CometReadBenchmark +``` + +To run TPC-H or TPC-DS micro benchmarks, please follow the instructions +in the respective source code, e.g., `CometTPCHQueryBenchmark`. + +## Debugging +Comet is a multi-language project with native code written in Rust and JVM code written in Java and Scala. +It is possible to debug both native and JVM code concurrently as described in the [DEBUGGING guide](DEBUGGING.md) diff --git a/EXPRESSIONS.md b/EXPRESSIONS.md new file mode 100644 index 0000000000..40e140abae --- /dev/null +++ b/EXPRESSIONS.md @@ -0,0 +1,96 @@ +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +# Expressions Supported by Comet + +The following Spark expressions are currently available: + ++ Literals ++ Arithmetic Operators + + UnaryMinus + + Add/Minus/Multiply/Divide/Remainder ++ Conditional functions + + Case When + + If ++ Cast ++ Coalesce ++ Boolean functions + + And + + Or + + Not + + EqualTo + + EqualNullSafe + + GreaterThan + + GreaterThanOrEqual + + LessThan + + LessThanOrEqual + + IsNull + + IsNotNull + + In ++ String functions + + Substring + + Coalesce + + StringSpace + + Like + + Contains + + Startswith + + Endswith + + Ascii + + Bit_length + + Octet_length + + Upper + + Lower + + Chr + + Initcap + + Trim/Btrim/Ltrim/Rtrim + + Concat_ws + + Repeat + + Length + + Reverse + + Instr + + Replace + + Translate ++ Bitwise functions + + Shiftright/Shiftleft ++ Date/Time functions + + Year/Hour/Minute/Second ++ Math functions + + Abs + + Acos + + Asin + + Atan + + Atan2 + + Cos + + Exp + + Ln + + Log10 + + Log2 + + Pow + + Round + + Signum + + Sin + + Sqrt + + Tan + + Ceil + + Floor ++ Aggregate functions + + Count + + Sum + + Max + + Min diff --git a/Makefile b/Makefile new file mode 100644 index 0000000000..10cb989724 --- /dev/null +++ b/Makefile @@ -0,0 +1,68 @@ +.PHONY: all core jvm test clean release-linux release bench + +all: core jvm + +core: + cd core && cargo build +jvm: + mvn clean package -DskipTests $(PROFILES) +test: + mvn clean + # We need to compile CometException so that the cargo test can pass + mvn compile -pl common -DskipTests $(PROFILES) + cd core && cargo build && \ + LD_LIBRARY_PATH=${LD_LIBRARY_PATH:+${LD_LIBRARY_PATH}:}${JAVA_HOME}/lib:${JAVA_HOME}/lib/server:${JAVA_HOME}/lib/jli && \ + DYLD_LIBRARY_PATH=${DYLD_LIBRARY_PATH:+${DYLD_LIBRARY_PATH}:}${JAVA_HOME}/lib:${JAVA_HOME}/lib/server:${JAVA_HOME}/lib/jli \ + RUST_BACKTRACE=1 cargo test + SPARK_HOME=`pwd` COMET_CONF_DIR=$(shell pwd)/conf RUST_BACKTRACE=1 mvn verify $(PROFILES) +clean: + cd core && cargo clean + mvn clean + rm -rf .dist +bench: + cd core && LD_LIBRARY_PATH=${LD_LIBRARY_PATH:+${LD_LIBRARY_PATH}:}${JAVA_HOME}/lib:${JAVA_HOME}/lib/server:${JAVA_HOME}/lib/jli && \ + DYLD_LIBRARY_PATH=${DYLD_LIBRARY_PATH:+${DYLD_LIBRARY_PATH}:}${JAVA_HOME}/lib:${JAVA_HOME}/lib/server:${JAVA_HOME}/lib/jli \ + RUSTFLAGS="-Ctarget-cpu=native" cargo bench $(filter-out $@,$(MAKECMDGOALS)) +format: + mvn compile test-compile scalafix:scalafix -Psemanticdb $(PROFILES) + mvn spotless:apply $(PROFILES) + +core-amd64: + rustup target add x86_64-apple-darwin + cd core && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --features nightly --release + mkdir -p common/target/classes/org/apache/comet/darwin/x86_64 + cp core/target/x86_64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/x86_64 + cd core && RUSTFLAGS="-Ctarget-cpu=haswell -Ctarget-feature=-prefer-256-bit" cargo build --features nightly --release + mkdir -p common/target/classes/org/apache/comet/linux/amd64 + cp core/target/release/libcomet.so common/target/classes/org/apache/comet/linux/amd64 + jar -cf common/target/comet-native-x86_64.jar \ + -C common/target/classes/org/apache/comet darwin \ + -C common/target/classes/org/apache/comet linux + ./dev/deploy-file common/target/comet-native-x86_64.jar comet-native-x86_64${COMET_CLASSIFIER} jar + +core-arm64: + rustup target add aarch64-apple-darwin + cd core && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --features nightly --release + mkdir -p common/target/classes/org/apache/comet/darwin/aarch64 + cp core/target/aarch64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/aarch64 + cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --features nightly --release + mkdir -p common/target/classes/org/apache/comet/linux/aarch64 + cp core/target/release/libcomet.so common/target/classes/org/apache/comet/linux/aarch64 + jar -cf common/target/comet-native-aarch64.jar \ + -C common/target/classes/org/apache/comet darwin \ + -C common/target/classes/org/apache/comet linux + ./dev/deploy-file common/target/comet-native-aarch64.jar comet-native-aarch64${COMET_CLASSIFIER} jar + +release-linux: clean + rustup target add aarch64-apple-darwin x86_64-apple-darwin + cd core && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --features nightly --release + cd core && RUSTFLAGS="-Ctarget-cpu=skylake -Ctarget-feature=-prefer-256-bit" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --features nightly --release + cd core && RUSTFLAGS="-Ctarget-cpu=native -Ctarget-feature=-prefer-256-bit" cargo build --features nightly --release + mvn install -Prelease -DskipTests $(PROFILES) +release: + cd core && RUSTFLAGS="-Ctarget-cpu=native" cargo build --features nightly --release + mvn install -Prelease -DskipTests $(PROFILES) +benchmark-%: clean release + cd spark && COMET_CONF_DIR=$(shell pwd)/conf MAVEN_OPTS='-Xmx20g' .mvn exec:java -Dexec.mainClass="$*" -Dexec.classpathScope="test" -Dexec.cleanupDaemonThreads="false" -Dexec.args="$(filter-out $@,$(MAKECMDGOALS))" $(PROFILES) +.DEFAULT: + @: # ignore arguments provided to benchmarks e.g. "make benchmark-foo -- --bar", we do not want to treat "--bar" as target diff --git a/README.md b/README.md index ba486a23f9..cab744874c 100644 --- a/README.md +++ b/README.md @@ -1 +1,60 @@ -# arrow-datafusion-comet +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +# Apache Arrow DataFusion Comet + +Comet is an Apache Spark plugin that uses [Apache Arrow DataFusion](https://arrow.apache.org/datafusion/) +as native runtime to achieve dramatic improvement in terms of query efficiency and query runtime. + +On a high level, Comet aims to support: +- a native Parquet implementation, including both reader and writer +- full implementation of Spark operators, including + Filter/Project/Aggregation/Join/Exchange etc. +- full implementation of Spark built-in expressions +- a UDF framework for users to migrate their existing UDF to native + +The following diagram illustrates the architecture of Comet: + +<a href="url"><img src="doc/comet-overview.png" align="center" height="600" width="750" ></a> + +## Current Status + +The project is currently integrated into Apache Spark 3.2, 3.3, and 3.4. + +## Feature Parity with Apache Spark + +The project strives to keep feature parity with Apache Spark, that is, +users should expect the same behavior (w.r.t features, configurations, +query results, etc) with Comet turned on or turned off in their Spark +jobs. In addition, Comet extension should automatically detect unsupported +features and fallback to Spark engine. + +To achieve this, besides unit tests within Comet itself, we also re-use +Spark SQL tests and make sure they all pass with Comet extension +enabled. + +## Supported Platforms + +Linux, Apple OSX (Intel and M1) + +## Requirements + +- Apache Spark 3.2, 3.3, or 3.4 +- JDK 8 and up +- GLIBC 2.17 (Centos 7) and up diff --git a/bin/comet-spark-shell b/bin/comet-spark-shell new file mode 100755 index 0000000000..9ae55a1b37 --- /dev/null +++ b/bin/comet-spark-shell @@ -0,0 +1,84 @@ +#!/bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + + +set -e + +if [[ -z ${SPARK_HOME} ]]; then + echo "ERROR: SPARK_HOME is not set: please set it to point to a Spark 3.4.0 distribution" + exit 1 +fi + +POSITIONAL=() +while [[ $# -gt 0 ]]; do + key="$1" + + case $key in + -r|--rebuild) + REBUILD=YES + shift + ;; + -o|--outdir) + COMET_OUTDIR=$2 + shift + shift + ;; + -d|--comet-dir) + COMET_DIR=$2 + shift + shift + ;; + *) + POSITIONAL+=("$1") + shift + ;; + esac +done + +set -- "${POSITIONAL[@]}" + +COMET_DIR="${COMET_DIR:-$HOME/git/comet}" +SPARK_VERSION_SHORT=3.4 +SCALA_BINARY_VERSION=2.12 +COMET_VERSION=0.1.0-SNAPSHOT +COMET_SPARK_JAR=comet-spark-spark${SPARK_VERSION_SHORT}_${SCALA_BINARY_VERSION}-${COMET_VERSION}.jar +COMET_OUTDIR="${COMET_OUTDIR:-/tmp}" + +if [[ ! -d $COMET_DIR ]]; then + echo "Comet repo: $COMET_DIR, doesn't exist" + exit 1 +fi + +if [[ ! -d $COMET_OUTDIR ]]; then + echo "Output directory for Comet Spark library: $COMET_OUTDIR, doesn't exist" + exit 1 +fi + +if [[ "X$REBUILD" == "XYES" ]]; then + cd $COMET_DIR && make release + cd $COMET_DIR/spark && cp target/${COMET_SPARK_JAR} $COMET_OUTDIR/${COMET_SPARK_JAR} +fi + +RUST_BACKTRACE=1 $SPARK_HOME/bin/spark-shell \ + --jars $COMET_OUTDIR/${COMET_SPARK_JAR} \ + --conf spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions \ + --conf spark.comet.enabled=true \ + --conf spark.comet.exec.enabled=true \ + --conf spark.comet.exec.all.enabled=true \ +$@ diff --git a/common/pom.xml b/common/pom.xml new file mode 100644 index 0000000000..f885c346df --- /dev/null +++ b/common/pom.xml @@ -0,0 +1,200 @@ +<?xml version="1.0" encoding="UTF-8"?> + +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + + +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.apache.comet</groupId> + <artifactId>comet-parent-spark${spark.version.short}_${scala.binary.version}</artifactId> + <version>0.1.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <artifactId>comet-common-spark${spark.version.short}_${scala.binary.version}</artifactId> + <name>comet-common</name> + + <properties> + <!-- Reverse default (skip installation), and then enable only for child modules --> + <maven.deploy.skip>false</maven.deploy.skip> + </properties> + + <dependencies> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-sql_${scala.binary.version}</artifactId> + </dependency> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-column</artifactId> + </dependency> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-hadoop</artifactId> + </dependency> + <dependency> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-vector</artifactId> + </dependency> + <dependency> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-memory-unsafe</artifactId> + </dependency> + <dependency> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-c-data</artifactId> + </dependency> + <dependency> + <groupId>junit</groupId> + <artifactId>junit</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + + <build> + <plugins> + <plugin> + <groupId>io.github.git-commit-id</groupId> + <artifactId>git-commit-id-maven-plugin</artifactId> + <version>5.0.0</version> + <executions> + <execution> + <id>get-the-git-infos</id> + <goals> + <goal>revision</goal> + </goals> + <phase>initialize</phase> + </execution> + </executions> + <configuration> + <generateGitPropertiesFile>true</generateGitPropertiesFile> + <generateGitPropertiesFilename>${project.build.outputDirectory}/comet-git-info.properties</generateGitPropertiesFilename> + <commitIdGenerationMode>full</commitIdGenerationMode> + <includeOnlyProperties> + <includeOnlyProperty>^git.branch$</includeOnlyProperty> + <includeOnlyProperty>^git.build.*$</includeOnlyProperty> + <includeOnlyProperty>^git.commit.id.(abbrev|full)$</includeOnlyProperty> + <includeOnlyProperty>^git.remote.*$</includeOnlyProperty> + </includeOnlyProperties> + </configuration> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <executions> + <execution> + <phase>package</phase> + <goals> + <goal>shade</goal> + </goals> + <configuration> + <createSourcesJar>true</createSourcesJar> + <shadeSourcesContent>true</shadeSourcesContent> + <shadedArtifactAttached>false</shadedArtifactAttached> + <createDependencyReducedPom>true</createDependencyReducedPom> + <artifactSet> + <includes> + <!-- We shade & relocation most of the Arrow classes, to prevent them from + conflicting with those in Spark --> + <include>org.apache.arrow:*</include> + </includes> + </artifactSet> + <filters> + <filter> + <artifact>*:*</artifact> + <excludes> + <exclude>**/*.thrift</exclude> + <exclude>git.properties</exclude> + <exclude>log4j.properties</exclude> + <exclude>log4j2.properties</exclude> + <exclude>arrow-git.properties</exclude> + </excludes> + </filter> + <filter> + <artifact>org.apache.arrow:arrow-vector</artifact> + <excludes> + <!-- Comet doesn't need codegen templates on Arrow --> + <exclude>codegen/**</exclude> + </excludes> + </filter> + </filters> + <relocations> + <relocation> + <pattern>org.apache.arrow</pattern> + <shadedPattern>${comet.shade.packageName}.arrow</shadedPattern> + <excludes> + <!-- We can't allocate Jni classes. These classes has no extra dependencies + so it should be OK to exclude --> + <exclude>org/apache/arrow/c/jni/JniWrapper</exclude> + <exclude>org/apache/arrow/c/jni/PrivateData</exclude> + <exclude>org/apache/arrow/c/jni/CDataJniException</exclude> + <!-- Also used by JNI: https://github.com/apache/arrow/blob/apache-arrow-11.0.0/java/c/src/main/cpp/jni_wrapper.cc#L341 + Note this class is not used by us, but required when loading the native lib --> + <exclude>org/apache/arrow/c/ArrayStreamExporter$ExportedArrayStreamPrivateData + </exclude> + </excludes> + </relocation> + </relocations> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>net.alchim31.maven</groupId> + <artifactId>scala-maven-plugin</artifactId> + </plugin> + </plugins> + <resources> + <resource> + <directory>${project.basedir}/src/main/resources</directory> + </resource> + <resource> + <directory>${project.basedir}/../core/target/x86_64-apple-darwin/release</directory> + <includes> + <include>libcomet.dylib</include> + </includes> + <targetPath>org/apache/comet/darwin/x86_64</targetPath> + </resource> + <resource> + <directory>${project.basedir}/../core/target/aarch64-apple-darwin/release</directory> + <includes> + <include>libcomet.dylib</include> + </includes> + <targetPath>org/apache/comet/darwin/aarch64</targetPath> + </resource> + <resource> + <directory>${jni.dir}</directory> + <includes> + <include>libcomet.dylib</include> + <include>libcomet.so</include> + </includes> + <targetPath>org/apache/comet/${platform}/${arch}</targetPath> + </resource> + </resources> + </build> + +</project> diff --git a/common/src/main/java/org/apache/arrow/c/ArrowImporter.java b/common/src/main/java/org/apache/arrow/c/ArrowImporter.java new file mode 100644 index 0000000000..90398cb72e --- /dev/null +++ b/common/src/main/java/org/apache/arrow/c/ArrowImporter.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.arrow.c; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.types.pojo.Field; + +/** + * This class is used to import Arrow schema and array from native execution/shuffle. We cannot use + * Arrow's Java API to import schema and array directly because Arrow's Java API `Data.importField` + * initiates a new `SchemaImporter` for each field. Each `SchemaImporter` maintains an internal + * dictionary id counter. So the dictionary ids for multiple dictionary columns will conflict with + * each other and cause data corruption. + */ +public class ArrowImporter { + private final SchemaImporter importer; + private final BufferAllocator allocator; + + public ArrowImporter(BufferAllocator allocator) { + this.allocator = allocator; + this.importer = new SchemaImporter(allocator); + } + + Field importField(ArrowSchema schema, CDataDictionaryProvider provider) { + Field var4; + try { + var4 = importer.importField(schema, provider); + } finally { + schema.release(); + schema.close(); + } + + return var4; + } + + public FieldVector importVector( + ArrowArray array, ArrowSchema schema, CDataDictionaryProvider provider) { + Field field = importField(schema, provider); + FieldVector vector = field.createVector(allocator); + Data.importIntoVector(allocator, array, vector, provider); + return vector; + } +} diff --git a/common/src/main/java/org/apache/comet/CometRuntimeException.java b/common/src/main/java/org/apache/comet/CometRuntimeException.java new file mode 100644 index 0000000000..b136ad1655 --- /dev/null +++ b/common/src/main/java/org/apache/comet/CometRuntimeException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet; + +/** The parent class for all Comet runtime exceptions */ +public class CometRuntimeException extends RuntimeException { + public CometRuntimeException(String message) { + super(message); + } + + public CometRuntimeException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/common/src/main/java/org/apache/comet/NativeBase.java b/common/src/main/java/org/apache/comet/NativeBase.java new file mode 100644 index 0000000000..42357b9bf7 --- /dev/null +++ b/common/src/main/java/org/apache/comet/NativeBase.java @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.file.Files; +import java.nio.file.StandardCopyOption; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.sql.comet.util.Utils; + +import static org.apache.comet.Constants.LOG_CONF_NAME; +import static org.apache.comet.Constants.LOG_CONF_PATH; + +/** Base class for JNI bindings. MUST be inherited by all classes that introduce JNI APIs. */ +public abstract class NativeBase { + static final String ARROW_UNSAFE_MEMORY_ACCESS = "arrow.enable_unsafe_memory_access"; + static final String ARROW_NULL_CHECK_FOR_GET = "arrow.enable_null_check_for_get"; + + private static final Logger LOG = LoggerFactory.getLogger(NativeBase.class); + private static final String NATIVE_LIB_NAME = "comet"; + + private static final String libraryToLoad = System.mapLibraryName(NATIVE_LIB_NAME); + private static boolean loaded = false; + private static final String searchPattern = "libcomet-"; + + static { + if (!isLoaded()) { + load(); + } + } + + public static synchronized boolean isLoaded() { + return loaded; + } + + // Only for testing + static synchronized void setLoaded(boolean b) { + loaded = b; + } + + static synchronized void load() { + if (loaded) { + return; + } + + cleanupOldTempLibs(); + + // Check if the arch used by JDK is the same as arch on the host machine, in particular, + // whether x86_64 JDK is used in arm64 Mac + if (!checkArch()) { + LOG.warn( + "Comet is disabled. JDK compiled for x86_64 is used in a Mac based on Apple Silicon. " + + "In order to use Comet, Please install a JDK version for ARM64 architecture"); + return; + } + + // Try to load Comet library from the java.library.path. + try { + System.loadLibrary(libraryToLoad); + loaded = true; + } catch (UnsatisfiedLinkError ex) { + // Doesn't exist, so proceed to loading bundled library. + bundleLoadLibrary(); + } + + initWithLogConf(); + // Only set the Arrow properties when debugging mode is off + if (!(boolean) CometConf.COMET_DEBUG_ENABLED().get()) { + setArrowProperties(); + } + } + + /** + * Use the bundled native libraries. Functionally equivalent to <code>System.loadLibrary</code>. + */ + private static void bundleLoadLibrary() { + String resourceName = resourceName(); + InputStream is = NativeBase.class.getResourceAsStream(resourceName); + if (is == null) { + throw new UnsupportedOperationException( + "Unsupported OS/arch, cannot find " + + resourceName + + ". Please try building from source."); + } + + File tempLib = null; + File tempLibLock = null; + try { + // Create the .lck file first to avoid a race condition + // with other concurrently running Java processes using Comet. + tempLibLock = File.createTempFile(searchPattern, "." + os().libExtension + ".lck"); + tempLib = new File(tempLibLock.getAbsolutePath().replaceFirst(".lck$", "")); + // copy to tempLib + Files.copy(is, tempLib.toPath(), StandardCopyOption.REPLACE_EXISTING); + System.load(tempLib.getAbsolutePath()); + loaded = true; + } catch (IOException e) { + throw new IllegalStateException("Cannot unpack libcomet: " + e); + } finally { + if (!loaded) { + if (tempLib != null && tempLib.exists()) { + if (!tempLib.delete()) { + LOG.error( + "Cannot unpack libcomet / cannot delete a temporary native library " + tempLib); + } + } + if (tempLibLock != null && tempLibLock.exists()) { + if (!tempLibLock.delete()) { + LOG.error( + "Cannot unpack libcomet / cannot delete a temporary lock file " + tempLibLock); + } + } + } else { + tempLib.deleteOnExit(); + tempLibLock.deleteOnExit(); + } + } + } + + private static void initWithLogConf() { + String logConfPath = System.getProperty(LOG_CONF_PATH(), Utils.getConfPath(LOG_CONF_NAME())); + + // If both the system property and the environmental variable failed to find a log + // configuration, then fall back to using the deployed default + if (logConfPath == null) { + LOG.info( + "Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. " + + "Using default log configuration which emits to stdout"); + logConfPath = ""; + } else { + LOG.info("Using {} for native library logging", logConfPath); + } + init(logConfPath); + } + + private static void cleanupOldTempLibs() { + String tempFolder = new File(System.getProperty("java.io.tmpdir")).getAbsolutePath(); + File dir = new File(tempFolder); + + File[] tempLibFiles = + dir.listFiles( + new FilenameFilter() { + public boolean accept(File dir, String name) { + return name.startsWith(searchPattern) && !name.endsWith(".lck"); + } + }); + + if (tempLibFiles != null) { + for (File tempLibFile : tempLibFiles) { + File lckFile = new File(tempLibFile.getAbsolutePath() + ".lck"); + if (!lckFile.exists()) { + try { + tempLibFile.delete(); + } catch (SecurityException e) { + LOG.error("Failed to delete old temp lib", e); + } + } + } + } + } + + // Set Arrow related properties upon initializing native, such as enabling unsafe memory access + // as well as disabling null check for get, for performance reasons. + private static void setArrowProperties() { + setPropertyIfNull(ARROW_UNSAFE_MEMORY_ACCESS, "true"); + setPropertyIfNull(ARROW_NULL_CHECK_FOR_GET, "false"); + } + + private static void setPropertyIfNull(String key, String value) { + if (System.getProperty(key) == null) { + LOG.info("Setting system property {} to {}", key, value); + System.setProperty(key, value); + } else { + LOG.info( + "Skip setting system property {} to {}, because it is already set to {}", + key, + value, + System.getProperty(key)); + } + } + + private enum OS { + // Even on Windows, the default compiler from cpptasks (gcc) uses .so as a shared lib extension + WINDOWS("win32", "so"), + LINUX("linux", "so"), + MAC("darwin", "dylib"), + SOLARIS("solaris", "so"); + public final String name, libExtension; + + OS(String name, String libExtension) { + this.name = name; + this.libExtension = libExtension; + } + } + + private static String arch() { + return System.getProperty("os.arch"); + } + + private static OS os() { + String osName = System.getProperty("os.name"); + if (osName.contains("Linux")) { + return OS.LINUX; + } else if (osName.contains("Mac")) { + return OS.MAC; + } else if (osName.contains("Windows")) { + return OS.WINDOWS; + } else if (osName.contains("Solaris") || osName.contains("SunOS")) { + return OS.SOLARIS; + } else { + throw new UnsupportedOperationException("Unsupported operating system: " + osName); + } + } + + // For some reason users will get JVM crash when running Comet that is compiled for `aarch64` + // using a JVM that is compiled against `amd64`. Here we check if that is the case and fallback + // to Spark accordingly. + private static boolean checkArch() { + if (os() == OS.MAC) { + try { + String javaArch = arch(); + Process process = Runtime.getRuntime().exec("uname -a"); + if (process.waitFor() == 0) { + BufferedReader in = new BufferedReader(new InputStreamReader(process.getInputStream())); + String line; + while ((line = in.readLine()) != null) { + if (javaArch.equals("x86_64") && line.contains("ARM64")) { + return false; + } + } + } + } catch (IOException | InterruptedException e) { + LOG.warn("Error parsing host architecture", e); + } + } + + return true; + } + + private static String resourceName() { + OS os = os(); + String packagePrefix = NativeBase.class.getPackage().getName().replace('.', '/'); + + return "/" + packagePrefix + "/" + os.name + "/" + arch() + "/" + libraryToLoad; + } + + /** + * Initialize the native library through JNI. + * + * @param logConfPath location to the native log configuration file + */ + static native void init(String logConfPath); +} diff --git a/common/src/main/java/org/apache/comet/ParquetRuntimeException.java b/common/src/main/java/org/apache/comet/ParquetRuntimeException.java new file mode 100644 index 0000000000..1f81587e76 --- /dev/null +++ b/common/src/main/java/org/apache/comet/ParquetRuntimeException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet; + +/** The parent class for the subset of Comet runtime exceptions related to Parquet. */ +public class ParquetRuntimeException extends CometRuntimeException { + public ParquetRuntimeException(String message) { + super(message); + } + + public ParquetRuntimeException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java new file mode 100644 index 0000000000..099c7b9733 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.TimestampNTZType$; + +import org.apache.comet.CometConf; +import org.apache.comet.vector.CometVector; + +/** Base class for Comet Parquet column reader implementations. */ +public abstract class AbstractColumnReader implements AutoCloseable { + protected static final Logger LOG = LoggerFactory.getLogger(AbstractColumnReader.class); + + /** The Spark data type. */ + protected final DataType type; + + /** Parquet column descriptor. */ + protected final ColumnDescriptor descriptor; + + /** + * Whether to always return 128 bit decimals, regardless of its precision. If false, this will + * return 32, 64 or 128 bit decimals depending on the precision. + */ + protected final boolean useDecimal128; + + /** + * Whether to return dates/timestamps that were written with legacy hybrid (Julian + Gregorian) + * calendar as it is. If this is true, Comet will return them as it is, instead of rebasing them + * to the new Proleptic Gregorian calendar. If this is false, Comet will throw exceptions when + * seeing these dates/timestamps. + */ + protected final boolean useLegacyDateTimestamp; + + /** The size of one batch, gets updated by 'readBatch' */ + protected int batchSize; + + /** A pointer to the native implementation of ColumnReader. */ + protected long nativeHandle; + + public AbstractColumnReader( + DataType type, + ColumnDescriptor descriptor, + boolean useDecimal128, + boolean useLegacyDateTimestamp) { + this.type = type; + this.descriptor = descriptor; + this.useDecimal128 = useDecimal128; + this.useLegacyDateTimestamp = useLegacyDateTimestamp; + TypeUtil.checkParquetType(descriptor, type); + } + + public ColumnDescriptor getDescriptor() { + return descriptor; + } + + /** + * Set the batch size of this reader to be 'batchSize'. Also initializes the native column reader. + */ + public void setBatchSize(int batchSize) { + assert nativeHandle == 0 + : "Native column reader shouldn't be initialized before " + "'setBatchSize' is called"; + this.batchSize = batchSize; + initNative(); + } + + /** + * Reads a batch of 'total' new rows. + * + * @param total the total number of rows to read + */ + public abstract void readBatch(int total); + + /** Returns the {@link CometVector} read by this reader. */ + public abstract CometVector currentBatch(); + + @Override + public void close() { + if (nativeHandle != 0) { + LOG.debug("Closing the column reader"); + Native.closeColumnReader(nativeHandle); + nativeHandle = 0; + } + } + + protected void initNative() { + LOG.debug("initializing the native column reader"); + DataType readType = (boolean) CometConf.COMET_SCHEMA_EVOLUTION_ENABLED().get() ? type : null; + boolean useLegacyDateTimestampOrNTZ = + useLegacyDateTimestamp || type == TimestampNTZType$.MODULE$; + nativeHandle = + Utils.initColumnReader( + descriptor, readType, batchSize, useDecimal128, useLegacyDateTimestampOrNTZ); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java new file mode 100644 index 0000000000..87302b3723 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -0,0 +1,620 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.Closeable; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; + +import scala.Option; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.Preconditions; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.comet.parquet.CometParquetReadSupport; +import org.apache.spark.sql.execution.datasources.PartitionedFile; +import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter; +import org.apache.spark.sql.execution.metric.SQLMetric; +import org.apache.spark.sql.types.*; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.util.AccumulatorV2; + +import org.apache.comet.CometConf; +import org.apache.comet.shims.ShimBatchReader; +import org.apache.comet.shims.ShimFileFormat; +import org.apache.comet.vector.CometVector; + +/** + * A vectorized Parquet reader that reads a Parquet file in a batched fashion. + * + * <p>Example of how to use this: + * + * <pre> + * BatchReader reader = new BatchReader(parquetFile, batchSize); + * try { + * reader.init(); + * while (reader.readBatch()) { + * ColumnarBatch batch = reader.currentBatch(); + * // consume the batch + * } + * } finally { // resources associated with the reader should be released + * reader.close(); + * } + * </pre> + */ +public class BatchReader extends RecordReader<Void, ColumnarBatch> implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); + + private Configuration conf; + private int capacity; + private boolean isCaseSensitive; + private boolean useFieldId; + private boolean ignoreMissingIds; + private StructType partitionSchema; + private InternalRow partitionValues; + private PartitionedFile file; + private final Map<String, SQLMetric> metrics; + + private long rowsRead; + private StructType sparkSchema; + private MessageType requestedSchema; + private CometVector[] vectors; + private AbstractColumnReader[] columnReaders; + private ColumnarBatch currentBatch; + private Future<Option<Throwable>> prefetchTask; + private LinkedBlockingQueue<Pair<PageReadStore, Long>> prefetchQueue; + private FileReader fileReader; + private boolean[] missingColumns; + private boolean isInitialized; + private ParquetMetadata footer; + + /** The total number of rows across all row groups of the input split. */ + private long totalRowCount; + + /** + * The total number of rows loaded so far, including all the rows from row groups that we've + * processed and the current row group. + */ + private long totalRowsLoaded; + + /** + * Whether the native scan should always return decimal represented by 128 bits, regardless of its + * precision. Normally, this should be true if native execution is enabled, since Arrow compute + * kernels doesn't support 32 and 64 bit decimals yet. + */ + private boolean useDecimal128; + + /** Whether to use the lazy materialization reader for reading columns. */ + private boolean useLazyMaterialization; + + /** + * Whether to return dates/timestamps that were written with legacy hybrid (Julian + Gregorian) + * calendar as it is. If this is true, Comet will return them as it is, instead of rebasing them + * to the new Proleptic Gregorian calendar. If this is false, Comet will throw exceptions when + * seeing these dates/timestamps. + */ + private boolean useLegacyDateTimestamp; + + /** The TaskContext object for executing this task. */ + private final TaskContext taskContext; + + // Only for testing + public BatchReader(String file, int capacity) { + this(file, capacity, null, null); + } + + // Only for testing + public BatchReader( + String file, int capacity, StructType partitionSchema, InternalRow partitionValues) { + this(new Configuration(), file, capacity, partitionSchema, partitionValues); + } + + // Only for testing + public BatchReader( + Configuration conf, + String file, + int capacity, + StructType partitionSchema, + InternalRow partitionValues) { + conf.set("spark.sql.parquet.binaryAsString", "false"); + conf.set("spark.sql.parquet.int96AsTimestamp", "false"); + conf.set("spark.sql.caseSensitive", "false"); + conf.set("spark.sql.parquet.inferTimestampNTZ.enabled", "true"); + conf.set("spark.sql.legacy.parquet.nanosAsLong", "false"); + + this.conf = conf; + this.capacity = capacity; + this.isCaseSensitive = false; + this.useFieldId = false; + this.ignoreMissingIds = false; + this.partitionSchema = partitionSchema; + this.partitionValues = partitionValues; + + this.file = ShimBatchReader.newPartitionedFile(partitionValues, file); + this.metrics = new HashMap<>(); + + this.taskContext = TaskContext$.MODULE$.get(); + } + + public BatchReader(AbstractColumnReader[] columnReaders) { + // Todo: set useDecimal128 and useLazyMaterialization + int numColumns = columnReaders.length; + this.columnReaders = new AbstractColumnReader[numColumns]; + vectors = new CometVector[numColumns]; + currentBatch = new ColumnarBatch(vectors); + // This constructor is used by Iceberg only. The columnReaders are + // initialized in Iceberg, so no need to call the init() + isInitialized = true; + this.taskContext = TaskContext$.MODULE$.get(); + this.metrics = new HashMap<>(); + } + + BatchReader( + Configuration conf, + PartitionedFile inputSplit, + ParquetMetadata footer, + int capacity, + StructType sparkSchema, + boolean isCaseSensitive, + boolean useFieldId, + boolean ignoreMissingIds, + boolean useLegacyDateTimestamp, + StructType partitionSchema, + InternalRow partitionValues, + Map<String, SQLMetric> metrics) { + this.conf = conf; + this.capacity = capacity; + this.sparkSchema = sparkSchema; + this.isCaseSensitive = isCaseSensitive; + this.useFieldId = useFieldId; + this.ignoreMissingIds = ignoreMissingIds; + this.useLegacyDateTimestamp = useLegacyDateTimestamp; + this.partitionSchema = partitionSchema; + this.partitionValues = partitionValues; + this.file = inputSplit; + this.footer = footer; + this.metrics = metrics; + this.taskContext = TaskContext$.MODULE$.get(); + } + + /** + * Initialize this reader. The reason we don't do it in the constructor is that we want to close + * any resource hold by this reader when error happens during the initialization. + */ + public void init() throws URISyntaxException, IOException { + useDecimal128 = + conf.getBoolean( + CometConf.COMET_USE_DECIMAL_128().key(), + (Boolean) CometConf.COMET_USE_DECIMAL_128().defaultValue().get()); + useLazyMaterialization = + conf.getBoolean( + CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), + (Boolean) CometConf.COMET_USE_LAZY_MATERIALIZATION().defaultValue().get()); + + long start = file.start(); + long length = file.length(); + String filePath = file.filePath().toString(); + + ParquetReadOptions.Builder builder = HadoopReadOptions.builder(conf, new Path(filePath)); + + if (start >= 0 && length >= 0) { + builder = builder.withRange(start, start + length); + } + ParquetReadOptions readOptions = builder.build(); + + // TODO: enable off-heap buffer when they are ready + ReadOptions cometReadOptions = ReadOptions.builder(conf).build(); + + Path path = new Path(new URI(filePath)); + fileReader = + new FileReader( + CometInputFile.fromPath(path, conf), footer, readOptions, cometReadOptions, metrics); + requestedSchema = fileReader.getFileMetaData().getSchema(); + MessageType fileSchema = requestedSchema; + + if (sparkSchema == null) { + sparkSchema = new ParquetToSparkSchemaConverter(conf).convert(requestedSchema); + } else { + requestedSchema = + CometParquetReadSupport.clipParquetSchema( + requestedSchema, sparkSchema, isCaseSensitive, useFieldId, ignoreMissingIds); + if (requestedSchema.getColumns().size() != sparkSchema.size()) { + throw new IllegalArgumentException( + String.format( + "Spark schema has %d columns while " + "Parquet schema has %d columns", + sparkSchema.size(), requestedSchema.getColumns().size())); + } + } + + totalRowCount = fileReader.getRecordCount(); + List<ColumnDescriptor> columns = requestedSchema.getColumns(); + int numColumns = columns.size(); + if (partitionSchema != null) numColumns += partitionSchema.size(); + columnReaders = new AbstractColumnReader[numColumns]; + + // Initialize missing columns and use null vectors for them + missingColumns = new boolean[columns.size()]; + List<String[]> paths = requestedSchema.getPaths(); + StructField[] nonPartitionFields = sparkSchema.fields(); + for (int i = 0; i < requestedSchema.getFieldCount(); i++) { + Type t = requestedSchema.getFields().get(i); + Preconditions.checkState( + t.isPrimitive() && !t.isRepetition(Type.Repetition.REPEATED), + "Complex type is not supported"); + String[] colPath = paths.get(i); + if (nonPartitionFields[i].name().equals(ShimFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME())) { + // Values of ROW_INDEX_TEMPORARY_COLUMN_NAME column are always populated with + // generated row indexes, rather than read from the file. + // TODO(SPARK-40059): Allow users to include columns named + // FileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME in their schemas. + long[] rowIndices = fileReader.getRowIndices(); + columnReaders[i] = new RowIndexColumnReader(nonPartitionFields[i], capacity, rowIndices); + missingColumns[i] = true; + } else if (fileSchema.containsPath(colPath)) { + ColumnDescriptor fd = fileSchema.getColumnDescription(colPath); + if (!fd.equals(columns.get(i))) { + throw new UnsupportedOperationException("Schema evolution is not supported"); + } + missingColumns[i] = false; + } else { + if (columns.get(i).getMaxDefinitionLevel() == 0) { + throw new IOException( + "Required column '" + + Arrays.toString(colPath) + + "' is missing" + + " in data file " + + filePath); + } + ConstantColumnReader reader = + new ConstantColumnReader(nonPartitionFields[i], capacity, useDecimal128); + columnReaders[i] = reader; + missingColumns[i] = true; + } + } + + // Initialize constant readers for partition columns + if (partitionSchema != null) { + StructField[] partitionFields = partitionSchema.fields(); + for (int i = columns.size(); i < columnReaders.length; i++) { + int fieldIndex = i - columns.size(); + StructField field = partitionFields[fieldIndex]; + ConstantColumnReader reader = + new ConstantColumnReader(field, capacity, partitionValues, fieldIndex, useDecimal128); + columnReaders[i] = reader; + } + } + + vectors = new CometVector[numColumns]; + currentBatch = new ColumnarBatch(vectors); + fileReader.setRequestedSchema(requestedSchema.getColumns()); + + // For test purpose only + // If the last external accumulator is `NumRowGroupsAccumulator`, the row group number to read + // will be updated to the accumulator. So we can check if the row groups are filtered or not + // in test case. + // Note that this tries to get thread local TaskContext object, if this is called at other + // thread, it won't update the accumulator. + if (taskContext != null) { + Option<AccumulatorV2<?, ?>> accu = taskContext.taskMetrics().externalAccums().lastOption(); + if (accu.isDefined() && accu.get().getClass().getSimpleName().equals("NumRowGroupsAcc")) { + @SuppressWarnings("unchecked") + AccumulatorV2<Integer, Integer> intAccum = (AccumulatorV2<Integer, Integer>) accu.get(); + intAccum.add(fileReader.getRowGroups().size()); + } + } + + // Pre-fetching + boolean preFetchEnabled = + conf.getBoolean( + CometConf.COMET_SCAN_PREFETCH_ENABLED().key(), + (boolean) CometConf.COMET_SCAN_PREFETCH_ENABLED().defaultValue().get()); + + if (preFetchEnabled) { + LOG.info("Prefetch enabled for BatchReader."); + this.prefetchQueue = new LinkedBlockingQueue<>(); + } + + isInitialized = true; + synchronized (this) { + // if prefetch is enabled, `init()` is called in separate thread. When + // `BatchReader.nextBatch()` is called asynchronously, it is possibly that + // `init()` is not called or finished. We need to hold on `nextBatch` until + // initialization of `BatchReader` is done. Once we are close to finish + // initialization, we notify the waiting thread of `nextBatch` to continue. + notifyAll(); + } + } + + public void setSparkSchema(StructType schema) { + this.sparkSchema = schema; + } + + public AbstractColumnReader[] getColumnReaders() { + return columnReaders; + } + + @Override + public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) + throws IOException, InterruptedException { + // Do nothing. The initialization work is done in 'init' already. + } + + @Override + public boolean nextKeyValue() throws IOException { + return nextBatch(); + } + + @Override + public Void getCurrentKey() { + return null; + } + + @Override + public ColumnarBatch getCurrentValue() { + return currentBatch(); + } + + @Override + public float getProgress() { + return (float) rowsRead / totalRowCount; + } + + /** + * Returns the current columnar batch being read. + * + * <p>Note that this must be called AFTER {@link BatchReader#nextBatch()}. + */ + public ColumnarBatch currentBatch() { + return currentBatch; + } + + // Only for testing + public Future<Option<Throwable>> getPrefetchTask() { + return this.prefetchTask; + } + + // Only for testing + public LinkedBlockingQueue<Pair<PageReadStore, Long>> getPrefetchQueue() { + return this.prefetchQueue; + } + + /** + * Loads the next batch of rows. + * + * @return true if there are no more rows to read, false otherwise. + */ + public boolean nextBatch() throws IOException { + if (this.prefetchTask == null) { + Preconditions.checkState(isInitialized, "init() should be called first!"); + } else { + // If prefetch is enabled, this reader will be initialized asynchronously from a + // different thread. Wait until it is initialized + while (!isInitialized) { + synchronized (this) { + try { + // Wait until initialization of current `BatchReader` is finished (i.e., `init()`), + // is done. It is possibly that `init()` is done after entering this while loop, + // so a short timeout is given. + wait(100); + + // Checks if prefetch task is finished. If so, tries to get exception if any. + if (prefetchTask.isDone()) { + Option<Throwable> exception = prefetchTask.get(); + if (exception.isDefined()) { + throw exception.get(); + } + } + } catch (RuntimeException e) { + // Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`. + throw e; + } catch (Throwable e) { + throw new IOException(e); + } + } + } + } + + if (rowsRead >= totalRowCount) return false; + boolean hasMore; + + try { + hasMore = loadNextRowGroupIfNecessary(); + } catch (RuntimeException e) { + // Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`. + throw e; + } catch (Throwable e) { + throw new IOException(e); + } + + if (!hasMore) return false; + int batchSize = (int) Math.min(capacity, totalRowsLoaded - rowsRead); + + return nextBatch(batchSize); + } + + public boolean nextBatch(int batchSize) { + long totalDecodeTime = 0, totalLoadTime = 0; + for (int i = 0; i < columnReaders.length; i++) { + AbstractColumnReader reader = columnReaders[i]; + long startNs = System.nanoTime(); + reader.readBatch(batchSize); + totalDecodeTime += System.nanoTime() - startNs; + startNs = System.nanoTime(); + vectors[i] = reader.currentBatch(); + totalLoadTime += System.nanoTime() - startNs; + } + + SQLMetric decodeMetric = metrics.get("ParquetNativeDecodeTime"); + if (decodeMetric != null) { + decodeMetric.add(totalDecodeTime); + } + SQLMetric loadMetric = metrics.get("ParquetNativeLoadTime"); + if (loadMetric != null) { + loadMetric.add(totalLoadTime); + } + + currentBatch.setNumRows(batchSize); + rowsRead += batchSize; + return true; + } + + @Override + public void close() throws IOException { + if (columnReaders != null) { + for (AbstractColumnReader reader : columnReaders) { + if (reader != null) { + reader.close(); + } + } + } + if (fileReader != null) { + fileReader.close(); + fileReader = null; + } + } + + private boolean loadNextRowGroupIfNecessary() throws Throwable { + // More rows can be read from loaded row group. No need to load next one. + if (rowsRead != totalRowsLoaded) return true; + + SQLMetric rowGroupTimeMetric = metrics.get("ParquetLoadRowGroupTime"); + SQLMetric numRowGroupsMetric = metrics.get("ParquetRowGroups"); + long startNs = System.nanoTime(); + + PageReadStore rowGroupReader = null; + if (prefetchTask != null && prefetchQueue != null) { + // Wait for pre-fetch task to finish. + Pair<PageReadStore, Long> rowGroupReaderPair = prefetchQueue.take(); + rowGroupReader = rowGroupReaderPair.getLeft(); + + // Update incremental byte read metric. Because this metric in Spark is maintained + // by thread local variable, we need to manually update it. + // TODO: We may expose metrics from `FileReader` and get from it directly. + long incBytesRead = rowGroupReaderPair.getRight(); + FileSystem.getAllStatistics().stream() + .forEach(statistic -> statistic.incrementBytesRead(incBytesRead)); + } else { + rowGroupReader = fileReader.readNextRowGroup(); + } + + if (rowGroupTimeMetric != null) { + rowGroupTimeMetric.add(System.nanoTime() - startNs); + } + if (rowGroupReader == null) { + return false; + } + if (numRowGroupsMetric != null) { + numRowGroupsMetric.add(1); + } + + List<ColumnDescriptor> columns = requestedSchema.getColumns(); + for (int i = 0; i < columns.size(); i++) { + if (missingColumns[i]) continue; + if (columnReaders[i] != null) columnReaders[i].close(); + // TODO: handle tz, datetime & int96 rebase + // TODO: consider passing page reader via ctor - however we need to fix the shading issue + // from Iceberg side. + DataType dataType = sparkSchema.fields()[i].dataType(); + ColumnReader reader = + Utils.getColumnReader( + dataType, + columns.get(i), + capacity, + useDecimal128, + useLazyMaterialization, + useLegacyDateTimestamp); + reader.setPageReader(rowGroupReader.getPageReader(columns.get(i))); + columnReaders[i] = reader; + } + totalRowsLoaded += rowGroupReader.getRowCount(); + return true; + } + + // Submits a prefetch task for this reader. + public void submitPrefetchTask(ExecutorService threadPool) { + this.prefetchTask = threadPool.submit(new PrefetchTask()); + } + + // A task for prefetching parquet row groups. + private class PrefetchTask implements Callable<Option<Throwable>> { + private long getBytesRead() { + return FileSystem.getAllStatistics().stream() + .mapToLong(s -> s.getThreadStatistics().getBytesRead()) + .sum(); + } + + @Override + public Option<Throwable> call() throws Exception { + // Gets the bytes read so far. + long baseline = getBytesRead(); + + try { + init(); + + while (true) { + PageReadStore rowGroupReader = fileReader.readNextRowGroup(); + + if (rowGroupReader == null) { + // Reaches the end of row groups. + return Option.empty(); + } else { + long incBytesRead = getBytesRead() - baseline; + + prefetchQueue.add(Pair.of(rowGroupReader, incBytesRead)); + } + } + } catch (Throwable e) { + // Returns exception thrown from the reader. The reader will re-throw it. + return Option.apply(e); + } finally { + if (fileReader != null) { + fileReader.closeStream(); + } + } + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java b/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java new file mode 100644 index 0000000000..a23216c7f5 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Operators; +import org.apache.parquet.filter2.predicate.UserDefinedPredicate; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.BloomFilterHeader; +import org.apache.parquet.format.Util; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.io.SeekableInputStream; + +public class BloomFilterReader implements FilterPredicate.Visitor<Boolean> { + private static final Logger LOG = LoggerFactory.getLogger(BloomFilterReader.class); + private static final boolean BLOCK_MIGHT_MATCH = false; + private static final boolean BLOCK_CANNOT_MATCH = true; + + private final Map<ColumnPath, ColumnChunkMetaData> columns; + private final Map<ColumnPath, BloomFilter> cache = new HashMap<>(); + private final InternalFileDecryptor fileDecryptor; + private final SeekableInputStream inputStream; + + BloomFilterReader( + BlockMetaData block, InternalFileDecryptor fileDecryptor, SeekableInputStream inputStream) { + this.columns = new HashMap<>(); + for (ColumnChunkMetaData column : block.getColumns()) { + columns.put(column.getPath(), column); + } + this.fileDecryptor = fileDecryptor; + this.inputStream = inputStream; + } + + @Override + public <T extends Comparable<T>> Boolean visit(Operators.Eq<T> eq) { + T value = eq.getValue(); + + if (value == null) { + // the bloom filter bitset contains only non-null values so isn't helpful. this + // could check the column stats, but the StatisticsFilter is responsible + return BLOCK_MIGHT_MATCH; + } + + Operators.Column<T> filterColumn = eq.getColumn(); + ColumnChunkMetaData meta = columns.get(filterColumn.getColumnPath()); + if (meta == null) { + // the column isn't in this file so all values are null, but the value + // must be non-null because of the above check. + return BLOCK_CANNOT_MATCH; + } + + try { + BloomFilter bloomFilter = readBloomFilter(meta); + if (bloomFilter != null && !bloomFilter.findHash(bloomFilter.hash(value))) { + return BLOCK_CANNOT_MATCH; + } + } catch (RuntimeException e) { + LOG.warn(e.getMessage()); + return BLOCK_MIGHT_MATCH; + } + + return BLOCK_MIGHT_MATCH; + } + + @Override + public <T extends Comparable<T>> Boolean visit(Operators.NotEq<T> notEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public <T extends Comparable<T>> Boolean visit(Operators.Lt<T> lt) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public <T extends Comparable<T>> Boolean visit(Operators.LtEq<T> ltEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public <T extends Comparable<T>> Boolean visit(Operators.Gt<T> gt) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public <T extends Comparable<T>> Boolean visit(Operators.GtEq<T> gtEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public Boolean visit(Operators.And and) { + return and.getLeft().accept(this) || and.getRight().accept(this); + } + + @Override + public Boolean visit(Operators.Or or) { + return or.getLeft().accept(this) && or.getRight().accept(this); + } + + @Override + public Boolean visit(Operators.Not not) { + throw new IllegalArgumentException( + "This predicate " + + not + + " contains a not! Did you forget" + + " to run this predicate through LogicalInverseRewriter?"); + } + + @Override + public <T extends Comparable<T>, U extends UserDefinedPredicate<T>> Boolean visit( + Operators.UserDefined<T, U> udp) { + return visit(udp, false); + } + + @Override + public <T extends Comparable<T>, U extends UserDefinedPredicate<T>> Boolean visit( + Operators.LogicalNotUserDefined<T, U> udp) { + return visit(udp.getUserDefined(), true); + } + + private <T extends Comparable<T>, U extends UserDefinedPredicate<T>> Boolean visit( + Operators.UserDefined<T, U> ud, boolean inverted) { + return BLOCK_MIGHT_MATCH; + } + + BloomFilter readBloomFilter(ColumnChunkMetaData meta) { + if (cache.containsKey(meta.getPath())) { + return cache.get(meta.getPath()); + } + try { + if (!cache.containsKey(meta.getPath())) { + BloomFilter bloomFilter = readBloomFilterInternal(meta); + if (bloomFilter == null) { + return null; + } + + cache.put(meta.getPath(), bloomFilter); + } + return cache.get(meta.getPath()); + } catch (IOException e) { + LOG.error("Failed to read Bloom filter data", e); + } + + return null; + } + + private BloomFilter readBloomFilterInternal(ColumnChunkMetaData meta) throws IOException { + long bloomFilterOffset = meta.getBloomFilterOffset(); + if (bloomFilterOffset < 0) { + return null; + } + + // Prepare to decrypt Bloom filter (for encrypted columns) + BlockCipher.Decryptor bloomFilterDecryptor = null; + byte[] bloomFilterHeaderAAD = null; + byte[] bloomFilterBitsetAAD = null; + if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { + InternalColumnDecryptionSetup columnDecryptionSetup = + fileDecryptor.getColumnSetup(meta.getPath()); + if (columnDecryptionSetup.isEncrypted()) { + bloomFilterDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); + bloomFilterHeaderAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.BloomFilterHeader, + meta.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + bloomFilterBitsetAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.BloomFilterBitset, + meta.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + } + } + + // Read Bloom filter data header. + inputStream.seek(bloomFilterOffset); + BloomFilterHeader bloomFilterHeader; + try { + bloomFilterHeader = + Util.readBloomFilterHeader(inputStream, bloomFilterDecryptor, bloomFilterHeaderAAD); + } catch (IOException e) { + LOG.warn("read no bloom filter"); + return null; + } + + int numBytes = bloomFilterHeader.getNumBytes(); + if (numBytes <= 0 || numBytes > BlockSplitBloomFilter.UPPER_BOUND_BYTES) { + LOG.warn("the read bloom filter size is wrong, size is {}", bloomFilterHeader.getNumBytes()); + return null; + } + + if (!bloomFilterHeader.getHash().isSetXXHASH() + || !bloomFilterHeader.getAlgorithm().isSetBLOCK() + || !bloomFilterHeader.getCompression().isSetUNCOMPRESSED()) { + LOG.warn( + "the read bloom filter is not supported yet, algorithm = {}, hash = {}, " + + "compression = {}", + bloomFilterHeader.getAlgorithm(), + bloomFilterHeader.getHash(), + bloomFilterHeader.getCompression()); + return null; + } + + byte[] bitset; + if (null == bloomFilterDecryptor) { + bitset = new byte[numBytes]; + inputStream.readFully(bitset); + } else { + bitset = bloomFilterDecryptor.decrypt(inputStream, bloomFilterBitsetAAD); + if (bitset.length != numBytes) { + throw new ParquetCryptoRuntimeException("Wrong length of decrypted bloom filter bitset"); + } + } + return new BlockSplitBloomFilter(bitset); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java new file mode 100644 index 0000000000..4e4f6ba0db --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.Util; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; +import org.apache.parquet.internal.hadoop.metadata.IndexReference; +import org.apache.parquet.io.SeekableInputStream; + +class ColumnIndexReader implements ColumnIndexStore { + private static final Logger LOG = LoggerFactory.getLogger(ColumnIndexReader.class); + + // Used for columns are not in this parquet file + private static final IndexStore MISSING_INDEX_STORE = + new IndexStore() { + @Override + public ColumnIndex getColumnIndex() { + return null; + } + + @Override + public OffsetIndex getOffsetIndex() { + return null; + } + }; + + private static final ColumnIndexReader EMPTY = + new ColumnIndexReader(new BlockMetaData(), Collections.emptySet(), null, null) { + @Override + public ColumnIndex getColumnIndex(ColumnPath column) { + return null; + } + + @Override + public OffsetIndex getOffsetIndex(ColumnPath column) { + throw new MissingOffsetIndexException(column); + } + }; + + private final InternalFileDecryptor fileDecryptor; + private final SeekableInputStream inputStream; + private final Map<ColumnPath, IndexStore> store; + + /** + * Creates a column index store which lazily reads column/offset indexes for the columns in paths. + * Paths are the set of columns used for the projection. + */ + static ColumnIndexReader create( + BlockMetaData block, + Set<ColumnPath> paths, + InternalFileDecryptor fileDecryptor, + SeekableInputStream inputStream) { + try { + return new ColumnIndexReader(block, paths, fileDecryptor, inputStream); + } catch (MissingOffsetIndexException e) { + return EMPTY; + } + } + + private ColumnIndexReader( + BlockMetaData block, + Set<ColumnPath> paths, + InternalFileDecryptor fileDecryptor, + SeekableInputStream inputStream) { + this.fileDecryptor = fileDecryptor; + this.inputStream = inputStream; + Map<ColumnPath, IndexStore> store = new HashMap<>(); + for (ColumnChunkMetaData column : block.getColumns()) { + ColumnPath path = column.getPath(); + if (paths.contains(path)) { + store.put(path, new IndexStoreImpl(column)); + } + } + this.store = store; + } + + @Override + public ColumnIndex getColumnIndex(ColumnPath column) { + return store.getOrDefault(column, MISSING_INDEX_STORE).getColumnIndex(); + } + + @Override + public OffsetIndex getOffsetIndex(ColumnPath column) { + return store.getOrDefault(column, MISSING_INDEX_STORE).getOffsetIndex(); + } + + private interface IndexStore { + ColumnIndex getColumnIndex(); + + OffsetIndex getOffsetIndex(); + } + + private class IndexStoreImpl implements IndexStore { + private final ColumnChunkMetaData meta; + private ColumnIndex columnIndex; + private boolean columnIndexRead; + private final OffsetIndex offsetIndex; + + IndexStoreImpl(ColumnChunkMetaData meta) { + this.meta = meta; + OffsetIndex oi; + try { + oi = readOffsetIndex(meta); + } catch (IOException e) { + // If the I/O issue still stands it will fail the reading later; + // otherwise we fail the filtering only with a missing offset index. + LOG.warn("Unable to read offset index for column {}", meta.getPath(), e); + oi = null; + } + if (oi == null) { + throw new MissingOffsetIndexException(meta.getPath()); + } + offsetIndex = oi; + } + + @Override + public ColumnIndex getColumnIndex() { + if (!columnIndexRead) { + try { + columnIndex = readColumnIndex(meta); + } catch (IOException e) { + // If the I/O issue still stands it will fail the reading later; + // otherwise we fail the filtering only with a missing column index. + LOG.warn("Unable to read column index for column {}", meta.getPath(), e); + } + columnIndexRead = true; + } + return columnIndex; + } + + @Override + public OffsetIndex getOffsetIndex() { + return offsetIndex; + } + } + + // Visible for testing + ColumnIndex readColumnIndex(ColumnChunkMetaData column) throws IOException { + IndexReference ref = column.getColumnIndexReference(); + if (ref == null) { + return null; + } + inputStream.seek(ref.getOffset()); + + BlockCipher.Decryptor columnIndexDecryptor = null; + byte[] columnIndexAAD = null; + if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { + InternalColumnDecryptionSetup columnDecryptionSetup = + fileDecryptor.getColumnSetup(column.getPath()); + if (columnDecryptionSetup.isEncrypted()) { + columnIndexDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); + columnIndexAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.ColumnIndex, + column.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + } + } + return ParquetMetadataConverter.fromParquetColumnIndex( + column.getPrimitiveType(), + Util.readColumnIndex(inputStream, columnIndexDecryptor, columnIndexAAD)); + } + + // Visible for testing + OffsetIndex readOffsetIndex(ColumnChunkMetaData column) throws IOException { + IndexReference ref = column.getOffsetIndexReference(); + if (ref == null) { + return null; + } + inputStream.seek(ref.getOffset()); + + BlockCipher.Decryptor offsetIndexDecryptor = null; + byte[] offsetIndexAAD = null; + if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { + InternalColumnDecryptionSetup columnDecryptionSetup = + fileDecryptor.getColumnSetup(column.getPath()); + if (columnDecryptionSetup.isEncrypted()) { + offsetIndexDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); + offsetIndexAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.OffsetIndex, + column.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + } + } + return ParquetMetadataConverter.fromParquetOffsetIndex( + Util.readOffsetIndex(inputStream, offsetIndexDecryptor, offsetIndexAAD)); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java new file mode 100644 index 0000000000..744d128304 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.List; +import java.util.Queue; + +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.io.ParquetDecodingException; + +public class ColumnPageReader implements PageReader { + private final CompressionCodecFactory.BytesInputDecompressor decompressor; + private final long valueCount; + private final Queue<DataPage> compressedPages; + private final DictionaryPage compressedDictionaryPage; + + private final OffsetIndex offsetIndex; + private final long rowCount; + private int pageIndex = 0; + + private final BlockCipher.Decryptor blockDecryptor; + private final byte[] dataPageAAD; + private final byte[] dictionaryPageAAD; + + ColumnPageReader( + CompressionCodecFactory.BytesInputDecompressor decompressor, + List<DataPage> compressedPages, + DictionaryPage compressedDictionaryPage, + OffsetIndex offsetIndex, + long rowCount, + BlockCipher.Decryptor blockDecryptor, + byte[] fileAAD, + int rowGroupOrdinal, + int columnOrdinal) { + this.decompressor = decompressor; + this.compressedPages = new ArrayDeque<>(compressedPages); + this.compressedDictionaryPage = compressedDictionaryPage; + long count = 0; + for (DataPage p : compressedPages) { + count += p.getValueCount(); + } + this.valueCount = count; + this.offsetIndex = offsetIndex; + this.rowCount = rowCount; + this.blockDecryptor = blockDecryptor; + + if (blockDecryptor != null) { + dataPageAAD = + AesCipher.createModuleAAD( + fileAAD, ModuleCipherFactory.ModuleType.DataPage, rowGroupOrdinal, columnOrdinal, 0); + dictionaryPageAAD = + AesCipher.createModuleAAD( + fileAAD, + ModuleCipherFactory.ModuleType.DictionaryPage, + rowGroupOrdinal, + columnOrdinal, + -1); + } else { + dataPageAAD = null; + dictionaryPageAAD = null; + } + } + + @Override + public long getTotalValueCount() { + return valueCount; + } + + /** Returns the total value count of the current page. */ + public int getPageValueCount() { + return compressedPages.element().getValueCount(); + } + + /** Skips the current page so it won't be returned by {@link #readPage()} */ + public void skipPage() { + compressedPages.poll(); + pageIndex++; + } + + @Override + public DataPage readPage() { + final DataPage compressedPage = compressedPages.poll(); + if (compressedPage == null) { + return null; + } + final int currentPageIndex = pageIndex++; + + if (null != blockDecryptor) { + AesCipher.quickUpdatePageAAD(dataPageAAD, getPageOrdinal(currentPageIndex)); + } + + return compressedPage.accept( + new DataPage.Visitor<DataPage>() { + @Override + public DataPage visit(DataPageV1 dataPageV1) { + try { + BytesInput bytes = dataPageV1.getBytes(); + if (null != blockDecryptor) { + bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dataPageAAD)); + } + BytesInput decompressed = + decompressor.decompress(bytes, dataPageV1.getUncompressedSize()); + + final DataPageV1 decompressedPage; + if (offsetIndex == null) { + decompressedPage = + new DataPageV1( + decompressed, + dataPageV1.getValueCount(), + dataPageV1.getUncompressedSize(), + dataPageV1.getStatistics(), + dataPageV1.getRlEncoding(), + dataPageV1.getDlEncoding(), + dataPageV1.getValueEncoding()); + } else { + long firstRowIndex = offsetIndex.getFirstRowIndex(currentPageIndex); + decompressedPage = + new DataPageV1( + decompressed, + dataPageV1.getValueCount(), + dataPageV1.getUncompressedSize(), + firstRowIndex, + Math.toIntExact( + offsetIndex.getLastRowIndex(currentPageIndex, rowCount) + - firstRowIndex + + 1), + dataPageV1.getStatistics(), + dataPageV1.getRlEncoding(), + dataPageV1.getDlEncoding(), + dataPageV1.getValueEncoding()); + } + if (dataPageV1.getCrc().isPresent()) { + decompressedPage.setCrc(dataPageV1.getCrc().getAsInt()); + } + return decompressedPage; + } catch (IOException e) { + throw new ParquetDecodingException("could not decompress page", e); + } + } + + @Override + public DataPage visit(DataPageV2 dataPageV2) { + if (!dataPageV2.isCompressed() && offsetIndex == null && null == blockDecryptor) { + return dataPageV2; + } + BytesInput pageBytes = dataPageV2.getData(); + + if (null != blockDecryptor) { + try { + pageBytes = + BytesInput.from(blockDecryptor.decrypt(pageBytes.toByteArray(), dataPageAAD)); + } catch (IOException e) { + throw new ParquetDecodingException( + "could not convert page ByteInput to byte array", e); + } + } + if (dataPageV2.isCompressed()) { + int uncompressedSize = + Math.toIntExact( + dataPageV2.getUncompressedSize() + - dataPageV2.getDefinitionLevels().size() + - dataPageV2.getRepetitionLevels().size()); + try { + pageBytes = decompressor.decompress(pageBytes, uncompressedSize); + } catch (IOException e) { + throw new ParquetDecodingException("could not decompress page", e); + } + } + + if (offsetIndex == null) { + return DataPageV2.uncompressed( + dataPageV2.getRowCount(), + dataPageV2.getNullCount(), + dataPageV2.getValueCount(), + dataPageV2.getRepetitionLevels(), + dataPageV2.getDefinitionLevels(), + dataPageV2.getDataEncoding(), + pageBytes, + dataPageV2.getStatistics()); + } else { + return DataPageV2.uncompressed( + dataPageV2.getRowCount(), + dataPageV2.getNullCount(), + dataPageV2.getValueCount(), + offsetIndex.getFirstRowIndex(currentPageIndex), + dataPageV2.getRepetitionLevels(), + dataPageV2.getDefinitionLevels(), + dataPageV2.getDataEncoding(), + pageBytes, + dataPageV2.getStatistics()); + } + } + }); + } + + @Override + public DictionaryPage readDictionaryPage() { + if (compressedDictionaryPage == null) { + return null; + } + try { + BytesInput bytes = compressedDictionaryPage.getBytes(); + if (null != blockDecryptor) { + bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dictionaryPageAAD)); + } + DictionaryPage decompressedPage = + new DictionaryPage( + decompressor.decompress(bytes, compressedDictionaryPage.getUncompressedSize()), + compressedDictionaryPage.getDictionarySize(), + compressedDictionaryPage.getEncoding()); + if (compressedDictionaryPage.getCrc().isPresent()) { + decompressedPage.setCrc(compressedDictionaryPage.getCrc().getAsInt()); + } + return decompressedPage; + } catch (IOException e) { + throw new ParquetDecodingException("Could not decompress dictionary page", e); + } + } + + private int getPageOrdinal(int currentPageIndex) { + return offsetIndex == null ? currentPageIndex : offsetIndex.getPageOrdinal(currentPageIndex); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java new file mode 100644 index 0000000000..7e45f4f9a2 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.memory.RootAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.types.pojo.DictionaryEncoding; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReader; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.CometConf; +import org.apache.comet.vector.CometDecodedVector; +import org.apache.comet.vector.CometDictionary; +import org.apache.comet.vector.CometDictionaryVector; +import org.apache.comet.vector.CometPlainVector; +import org.apache.comet.vector.CometVector; + +public class ColumnReader extends AbstractColumnReader { + protected static final Logger LOG = LoggerFactory.getLogger(ColumnReader.class); + protected static final BufferAllocator ALLOCATOR = new RootAllocator(); + + /** + * The current Comet vector holding all the values read by this column reader. Owned by this + * reader and MUST be closed after use. + */ + private CometDecodedVector currentVector; + + /** Dictionary values for this column. Only set if the column is using dictionary encoding. */ + protected CometDictionary dictionary; + + /** Reader for dictionary & data pages in the current column chunk. */ + protected PageReader pageReader; + + /** Whether the first data page has been loaded. */ + private boolean firstPageLoaded = false; + + /** + * The number of nulls in the current batch, used when we are skipping importing of Arrow vectors, + * in which case we'll simply update the null count of the existing vectors. + */ + int currentNumNulls; + + /** + * The number of values in the current batch, used when we are skipping importing of Arrow + * vectors, in which case we'll simply update the null count of the existing vectors. + */ + int currentNumValues; + + /** + * Whether the last loaded vector contains any null value. This is used to determine if we can + * skip vector reloading. If the flag is false, Arrow C API will skip to import the validity + * buffer, and therefore we cannot skip vector reloading. + */ + boolean hadNull; + + /** Dictionary provider for this column. */ + private final CDataDictionaryProvider dictionaryProvider = new CDataDictionaryProvider(); + + public ColumnReader( + DataType type, + ColumnDescriptor descriptor, + int batchSize, + boolean useDecimal128, + boolean useLegacyDateTimestamp) { + super(type, descriptor, useDecimal128, useLegacyDateTimestamp); + assert batchSize > 0 : "Batch size must be positive, found " + batchSize; + this.batchSize = batchSize; + initNative(); + } + + /** + * Set the page reader for a new column chunk to read. Expects to call `readBatch` after this. + * + * @param pageReader the page reader for the new column chunk + */ + public void setPageReader(PageReader pageReader) throws IOException { + this.pageReader = pageReader; + + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); + if (dictionaryPage != null) { + LOG.debug("dictionary page encoding = {}", dictionaryPage.getEncoding()); + Native.setDictionaryPage( + nativeHandle, + dictionaryPage.getDictionarySize(), + dictionaryPage.getBytes().toByteArray(), + dictionaryPage.getEncoding().ordinal()); + } + } + + @Override + public void readBatch(int total) { + LOG.debug("Start to batch of size = " + total); + + if (!firstPageLoaded) { + readPage(); + firstPageLoaded = true; + } + + // Now first reset the current columnar batch so that it can be used to fill in a new batch + // of values. Then, keep reading more data pages (via 'readBatch') until the current batch is + // full, or we have read 'total' number of values. + Native.resetBatch(nativeHandle); + + int left = total, nullsRead = 0; + while (left > 0) { + int[] array = Native.readBatch(nativeHandle, left); + int valuesRead = array[0]; + nullsRead += array[1]; + if (valuesRead < left) { + readPage(); + } + left -= valuesRead; + } + + this.currentNumValues = total; + this.currentNumNulls = nullsRead; + } + + /** Returns the {@link CometVector} read by this reader. */ + @Override + public CometVector currentBatch() { + return loadVector(); + } + + @Override + public void close() { + if (currentVector != null) { + currentVector.close(); + currentVector = null; + } + dictionaryProvider.close(); + super.close(); + } + + /** Returns a decoded {@link CometDecodedVector Comet vector}. */ + public CometDecodedVector loadVector() { + // Only re-use Comet vector iff: + // 1. if we're not using dictionary encoding, since with dictionary encoding, the native + // side may fallback to plain encoding and the underlying memory address for the vector + // will change as result. + // 2. if the column type is of fixed width, in other words, string/binary are not supported + // since the native side may resize the vector and therefore change memory address. + // 3. if the last loaded vector contains null values: if values of last vector are all not + // null, Arrow C data API will skip loading the native validity buffer, therefore we + // should not re-use the vector in that case. + // 4. if the last loaded vector doesn't contain any null value, but the current vector also + // are all not null, which means we can also re-use the loaded vector. + // 5. if the new number of value is the same or smaller + if ((hadNull || currentNumNulls == 0) + && currentVector != null + && dictionary == null + && currentVector.isFixedLength() + && currentVector.numValues() >= currentNumValues) { + currentVector.setNumNulls(currentNumNulls); + currentVector.setNumValues(currentNumValues); + return currentVector; + } + + LOG.debug("Reloading vector"); + + // Close the previous vector first to release struct memory allocated to import Arrow array & + // schema from native side, through the C data interface + if (currentVector != null) { + currentVector.close(); + } + + long[] addresses = Native.currentBatch(nativeHandle); + + try (ArrowArray array = ArrowArray.wrap(addresses[0]); + ArrowSchema schema = ArrowSchema.wrap(addresses[1])) { + FieldVector vector = Data.importVector(ALLOCATOR, array, schema, dictionaryProvider); + DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); + + CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); + + // Update whether the current vector contains any null values. This is used in the following + // batch(s) to determine whether we can skip loading the native vector. + hadNull = cometVector.hasNull(); + + if (dictionaryEncoding == null) { + if (dictionary != null) { + // This means the column was using dictionary encoding but now has fall-back to plain + // encoding, on the native side. Setting 'dictionary' to null here, so we can use it as + // a condition to check if we can re-use vector later. + dictionary = null; + } + // Either the column is not dictionary encoded, or it was using dictionary encoding but + // a new data page has switched back to use plain encoding. For both cases we should + // return plain vector. + currentVector = cometVector; + return currentVector; + } else if (dictionary == null) { + // There is dictionary from native side but the Java side dictionary hasn't been + // initialized yet. + Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); + CometPlainVector dictionaryVector = + new CometPlainVector(arrowDictionary.getVector(), useDecimal128); + dictionary = new CometDictionary(dictionaryVector); + } + + currentVector = + new CometDictionaryVector(cometVector, dictionary, dictionaryProvider, useDecimal128); + return currentVector; + } + } + + protected void readPage() { + DataPage page = pageReader.readPage(); + if (page == null) { + throw new RuntimeException("overreading: returned DataPage is null"); + } + ; + int pageValueCount = page.getValueCount(); + page.accept( + new DataPage.Visitor<Void>() { + @Override + public Void visit(DataPageV1 dataPageV1) { + LOG.debug("data page encoding = {}", dataPageV1.getValueEncoding()); + if (dataPageV1.getDlEncoding() != Encoding.RLE + && descriptor.getMaxDefinitionLevel() != 0) { + throw new UnsupportedOperationException( + "Unsupported encoding: " + dataPageV1.getDlEncoding()); + } + if (!isValidValueEncoding(dataPageV1.getValueEncoding())) { + throw new UnsupportedOperationException( + "Unsupported value encoding: " + dataPageV1.getValueEncoding()); + } + try { + boolean useDirectBuffer = + (Boolean) CometConf.COMET_PARQUET_ENABLE_DIRECT_BUFFER().get(); + if (useDirectBuffer) { + ByteBuffer buffer = dataPageV1.getBytes().toByteBuffer(); + Native.setPageBufferV1( + nativeHandle, pageValueCount, buffer, dataPageV1.getValueEncoding().ordinal()); + } else { + byte[] array = dataPageV1.getBytes().toByteArray(); + Native.setPageV1( + nativeHandle, pageValueCount, array, dataPageV1.getValueEncoding().ordinal()); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + return null; + } + + @Override + public Void visit(DataPageV2 dataPageV2) { + if (!isValidValueEncoding(dataPageV2.getDataEncoding())) { + throw new UnsupportedOperationException( + "Unsupported encoding: " + dataPageV2.getDataEncoding()); + } + try { + Native.setPageV2( + nativeHandle, + pageValueCount, + dataPageV2.getDefinitionLevels().toByteArray(), + dataPageV2.getRepetitionLevels().toByteArray(), + dataPageV2.getData().toByteArray(), + dataPageV2.getDataEncoding().ordinal()); + } catch (IOException e) { + throw new RuntimeException(e); + } + return null; + } + }); + } + + @SuppressWarnings("deprecation") + private boolean isValidValueEncoding(Encoding encoding) { + switch (encoding) { + case PLAIN: + case RLE_DICTIONARY: + case PLAIN_DICTIONARY: + return true; + default: + return false; + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/CometInputFile.java b/common/src/main/java/org/apache/comet/parquet/CometInputFile.java new file mode 100644 index 0000000000..eb54d1a724 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/CometInputFile.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.IOException; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FutureDataInputStreamBuilder; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.VersionInfo; +import org.apache.parquet.hadoop.util.HadoopStreams; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.SeekableInputStream; + +/** + * A Parquet {@link InputFile} implementation that's similar to {@link + * org.apache.parquet.hadoop.util.HadoopInputFile}, but with optimizations introduced in Hadoop 3.x, + * for S3 specifically. + */ +public class CometInputFile implements InputFile { + private static final String MAJOR_MINOR_REGEX = "^(\\d+)\\.(\\d+)(\\..*)?$"; + private static final Pattern VERSION_MATCHER = Pattern.compile(MAJOR_MINOR_REGEX); + + private final FileSystem fs; + private final FileStatus stat; + private final Configuration conf; + + public static CometInputFile fromPath(Path path, Configuration conf) throws IOException { + FileSystem fs = path.getFileSystem(conf); + return new CometInputFile(fs, fs.getFileStatus(path), conf); + } + + private CometInputFile(FileSystem fs, FileStatus stat, Configuration conf) { + this.fs = fs; + this.stat = stat; + this.conf = conf; + } + + @Override + public long getLength() { + return stat.getLen(); + } + + public Configuration getConf() { + return this.conf; + } + + public FileSystem getFileSystem() { + return this.fs; + } + + public Path getPath() { + return stat.getPath(); + } + + @Override + public SeekableInputStream newStream() throws IOException { + FSDataInputStream stream; + try { + if (isAtLeastHadoop33()) { + // If Hadoop version is >= 3.3.x, we'll use the 'openFile' API which can save a + // HEAD request from cloud storages like S3 + FutureDataInputStreamBuilder inputStreamBuilder = + fs.openFile(stat.getPath()).withFileStatus(stat); + + if (stat.getPath().toString().startsWith("s3a")) { + // Switch to random S3 input policy so that we don't do sequential read on the entire + // S3 object. By default, the policy is normal which does sequential read until a back + // seek happens, which in our case will never happen. + inputStreamBuilder = + inputStreamBuilder.opt("fs.s3a.experimental.input.fadvise", "random"); + } + stream = inputStreamBuilder.build().get(); + } else { + stream = fs.open(stat.getPath()); + } + } catch (Exception e) { + throw new IOException("Error when opening file " + stat.getPath(), e); + } + return HadoopStreams.wrap(stream); + } + + public SeekableInputStream newStream(long offset, long length) throws IOException { + try { + FSDataInputStream stream; + if (isAtLeastHadoop33()) { + FutureDataInputStreamBuilder inputStreamBuilder = + fs.openFile(stat.getPath()).withFileStatus(stat); + + if (stat.getPath().toString().startsWith("s3a")) { + // Switch to random S3 input policy so that we don't do sequential read on the entire + // S3 object. By default, the policy is normal which does sequential read until a back + // seek happens, which in our case will never happen. + // + // Also set read ahead length equal to the column chunk length so we don't have to open + // multiple S3 http connections. + inputStreamBuilder = + inputStreamBuilder + .opt("fs.s3a.experimental.input.fadvise", "random") + .opt("fs.s3a.readahead.range", Long.toString(length)); + } + + stream = inputStreamBuilder.build().get(); + } else { + stream = fs.open(stat.getPath()); + } + return HadoopStreams.wrap(stream); + } catch (Exception e) { + throw new IOException( + "Error when opening file " + stat.getPath() + ", offset=" + offset + ", length=" + length, + e); + } + } + + @Override + public String toString() { + return stat.getPath().toString(); + } + + private static boolean isAtLeastHadoop33() { + String version = VersionInfo.getVersion(); + return CometInputFile.isAtLeastHadoop33(version); + } + + static boolean isAtLeastHadoop33(String version) { + Matcher matcher = VERSION_MATCHER.matcher(version); + if (matcher.matches()) { + if (matcher.group(1).equals("3")) { + int minorVersion = Integer.parseInt(matcher.group(2)); + return minorVersion >= 3; + } + } + return false; + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java new file mode 100644 index 0000000000..8de2376f94 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.math.BigInteger; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.types.UTF8String; + +import org.apache.comet.shims.ShimResolveDefaultColumns; + +/** + * A column reader that always return constant vectors. Used for reading partition columns, for + * instance. + */ +public class ConstantColumnReader extends MetadataColumnReader { + /** Whether all the values in this constant column are nulls */ + private boolean isNull; + + /** The constant value in the format of Object that are used to initialize this column reader. */ + private Object value; + + public ConstantColumnReader(StructField field, int batchSize, boolean useDecimal128) { + this(field.dataType(), TypeUtil.convertToParquet(field), batchSize, useDecimal128); + this.value = ShimResolveDefaultColumns.getExistenceDefaultValue(field); + init(value); + } + + public ConstantColumnReader( + StructField field, int batchSize, InternalRow values, int index, boolean useDecimal128) { + this(field.dataType(), TypeUtil.convertToParquet(field), batchSize, useDecimal128); + init(values, index); + } + + public ConstantColumnReader( + DataType type, ColumnDescriptor descriptor, Object value, boolean useDecimal128) { + super(type, descriptor, useDecimal128); + this.value = value; + } + + ConstantColumnReader( + DataType type, ColumnDescriptor descriptor, int batchSize, boolean useDecimal128) { + super(type, descriptor, useDecimal128); + this.batchSize = batchSize; + initNative(); + } + + @Override + public void setBatchSize(int batchSize) { + super.setBatchSize(batchSize); + init(value); + } + + @Override + public void readBatch(int total) { + super.readBatch(total); + if (isNull) setNumNulls(total); + } + + private void init(InternalRow values, int index) { + Object value = values.get(index, type); + init(value); + } + + private void init(Object value) { + if (value == null) { + Native.setNull(nativeHandle); + isNull = true; + } else if (type == DataTypes.BooleanType) { + Native.setBoolean(nativeHandle, (boolean) value); + } else if (type == DataTypes.ByteType) { + Native.setByte(nativeHandle, (byte) value); + } else if (type == DataTypes.ShortType) { + Native.setShort(nativeHandle, (short) value); + } else if (type == DataTypes.IntegerType) { + Native.setInt(nativeHandle, (int) value); + } else if (type == DataTypes.LongType) { + Native.setLong(nativeHandle, (long) value); + } else if (type == DataTypes.FloatType) { + Native.setFloat(nativeHandle, (float) value); + } else if (type == DataTypes.DoubleType) { + Native.setDouble(nativeHandle, (double) value); + } else if (type == DataTypes.BinaryType) { + Native.setBinary(nativeHandle, (byte[]) value); + } else if (type == DataTypes.StringType) { + Native.setBinary(nativeHandle, ((UTF8String) value).getBytes()); + } else if (type == DataTypes.DateType) { + Native.setInt(nativeHandle, (int) value); + } else if (type == DataTypes.TimestampType || type == TimestampNTZType$.MODULE$) { + Native.setLong(nativeHandle, (long) value); + } else if (type instanceof DecimalType) { + DecimalType dt = (DecimalType) type; + Decimal d = (Decimal) value; + if (!useDecimal128 && dt.precision() <= Decimal.MAX_INT_DIGITS()) { + Native.setInt(nativeHandle, ((int) d.toUnscaledLong())); + } else if (!useDecimal128 && dt.precision() <= Decimal.MAX_LONG_DIGITS()) { + Native.setLong(nativeHandle, d.toUnscaledLong()); + } else { + final BigInteger integer = d.toJavaBigDecimal().unscaledValue(); + byte[] bytes = integer.toByteArray(); + Native.setDecimal(nativeHandle, bytes); + } + } else { + throw new UnsupportedOperationException("Unsupported Spark type: " + type); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java b/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java new file mode 100644 index 0000000000..b4b5a8fc96 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.DictionaryPageReadStore; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.DictionaryPageHeader; +import org.apache.parquet.format.PageHeader; +import org.apache.parquet.format.Util; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.io.SeekableInputStream; + +public class DictionaryPageReader implements DictionaryPageReadStore { + private final Map<String, Optional<DictionaryPage>> cache; + private final InternalFileDecryptor fileDecryptor; + private final SeekableInputStream inputStream; + private final ParquetReadOptions options; + private final Map<String, ColumnChunkMetaData> columns; + + DictionaryPageReader( + BlockMetaData block, + InternalFileDecryptor fileDecryptor, + SeekableInputStream inputStream, + ParquetReadOptions options) { + this.columns = new HashMap<>(); + this.cache = new ConcurrentHashMap<>(); + this.fileDecryptor = fileDecryptor; + this.inputStream = inputStream; + this.options = options; + + for (ColumnChunkMetaData column : block.getColumns()) { + columns.put(column.getPath().toDotString(), column); + } + } + + @Override + public DictionaryPage readDictionaryPage(ColumnDescriptor descriptor) { + String dotPath = String.join(".", descriptor.getPath()); + ColumnChunkMetaData column = columns.get(dotPath); + + if (column == null) { + throw new ParquetDecodingException("Failed to load dictionary, unknown column: " + dotPath); + } + + return cache + .computeIfAbsent( + dotPath, + key -> { + try { + final DictionaryPage dict = + column.hasDictionaryPage() ? readDictionary(column) : null; + + // Copy the dictionary to ensure it can be reused if it is returned + // more than once. This can happen when a DictionaryFilter has two or + // more predicates for the same column. Cache misses as well. + return (dict != null) ? Optional.of(reusableCopy(dict)) : Optional.empty(); + } catch (IOException e) { + throw new ParquetDecodingException("Failed to read dictionary", e); + } + }) + .orElse(null); + } + + DictionaryPage readDictionary(ColumnChunkMetaData meta) throws IOException { + if (!meta.hasDictionaryPage()) { + return null; + } + + if (inputStream.getPos() != meta.getStartingPos()) { + inputStream.seek(meta.getStartingPos()); + } + + boolean encryptedColumn = false; + InternalColumnDecryptionSetup columnDecryptionSetup = null; + byte[] dictionaryPageAAD = null; + BlockCipher.Decryptor pageDecryptor = null; + if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { + columnDecryptionSetup = fileDecryptor.getColumnSetup(meta.getPath()); + if (columnDecryptionSetup.isEncrypted()) { + encryptedColumn = true; + } + } + + PageHeader pageHeader; + if (!encryptedColumn) { + pageHeader = Util.readPageHeader(inputStream); + } else { + byte[] dictionaryPageHeaderAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.DictionaryPageHeader, + meta.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + pageHeader = + Util.readPageHeader( + inputStream, columnDecryptionSetup.getMetaDataDecryptor(), dictionaryPageHeaderAAD); + dictionaryPageAAD = + AesCipher.createModuleAAD( + fileDecryptor.getFileAAD(), + ModuleCipherFactory.ModuleType.DictionaryPage, + meta.getRowGroupOrdinal(), + columnDecryptionSetup.getOrdinal(), + -1); + pageDecryptor = columnDecryptionSetup.getDataDecryptor(); + } + + if (!pageHeader.isSetDictionary_page_header()) { + return null; + } + + DictionaryPage compressedPage = + readCompressedDictionary(pageHeader, inputStream, pageDecryptor, dictionaryPageAAD); + CompressionCodecFactory.BytesInputDecompressor decompressor = + options.getCodecFactory().getDecompressor(meta.getCodec()); + + return new DictionaryPage( + decompressor.decompress(compressedPage.getBytes(), compressedPage.getUncompressedSize()), + compressedPage.getDictionarySize(), + compressedPage.getEncoding()); + } + + private DictionaryPage readCompressedDictionary( + PageHeader pageHeader, + SeekableInputStream fin, + BlockCipher.Decryptor pageDecryptor, + byte[] dictionaryPageAAD) + throws IOException { + DictionaryPageHeader dictHeader = pageHeader.getDictionary_page_header(); + + int uncompressedPageSize = pageHeader.getUncompressed_page_size(); + int compressedPageSize = pageHeader.getCompressed_page_size(); + + byte[] dictPageBytes = new byte[compressedPageSize]; + fin.readFully(dictPageBytes); + + BytesInput bin = BytesInput.from(dictPageBytes); + + if (null != pageDecryptor) { + bin = BytesInput.from(pageDecryptor.decrypt(bin.toByteArray(), dictionaryPageAAD)); + } + + return new DictionaryPage( + bin, + uncompressedPageSize, + dictHeader.getNum_values(), + org.apache.parquet.column.Encoding.valueOf(dictHeader.getEncoding().name())); + } + + private static DictionaryPage reusableCopy(DictionaryPage dict) throws IOException { + return new DictionaryPage( + BytesInput.from(dict.getBytes().toByteArray()), + dict.getDictionarySize(), + dict.getEncoding()); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/FileReader.java b/common/src/main/java/org/apache/comet/parquet/FileReader.java new file mode 100644 index 0000000000..eddaf3f1ac --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/FileReader.java @@ -0,0 +1,1151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Method; +import java.net.URI; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.zip.CRC32; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.FileDecryptionProperties; +import org.apache.parquet.crypto.InternalColumnDecryptionSetup; +import org.apache.parquet.crypto.InternalFileDecryptor; +import org.apache.parquet.crypto.ModuleCipherFactory; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.DataPageHeader; +import org.apache.parquet.format.DataPageHeaderV2; +import org.apache.parquet.format.DictionaryPageHeader; +import org.apache.parquet.format.FileCryptoMetaData; +import org.apache.parquet.format.PageHeader; +import org.apache.parquet.format.Util; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.counters.BenchmarkCounter; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexFilter; +import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; +import org.apache.parquet.internal.filter2.columnindex.RowRanges; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.io.SeekableInputStream; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.spark.sql.execution.metric.SQLMetric; + +import static org.apache.parquet.hadoop.ParquetFileWriter.EFMAGIC; +import static org.apache.parquet.hadoop.ParquetFileWriter.MAGIC; + +import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.BLOOMFILTER; +import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.DICTIONARY; +import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.STATISTICS; + +/** + * A Parquet file reader. Mostly followed {@code ParquetFileReader} in {@code parquet-mr}, but with + * customizations & optimizations for Comet. + */ +public class FileReader implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); + + private final ParquetMetadataConverter converter; + protected final SeekableInputStream f; + private final InputFile file; + private final Map<String, SQLMetric> metrics; + private final Map<ColumnPath, ColumnDescriptor> paths = new HashMap<>(); + private final FileMetaData fileMetaData; // may be null + private final List<BlockMetaData> blocks; + private final List<ColumnIndexReader> blockIndexStores; + private final List<RowRanges> blockRowRanges; + private final CRC32 crc; + private final ParquetMetadata footer; + + /** + * Read configurations come from two options: - options: these are options defined & specified + * from 'parquet-mr' library - cometOptions: these are Comet-specific options, for the features + * introduced in Comet's Parquet implementation + */ + private final ParquetReadOptions options; + + private final ReadOptions cometOptions; + + private int currentBlock = 0; + private RowGroupReader currentRowGroup = null; + private InternalFileDecryptor fileDecryptor; + + public FileReader(InputFile file, ParquetReadOptions options, ReadOptions cometOptions) + throws IOException { + this(file, null, options, cometOptions, null); + } + + public FileReader( + InputFile file, + ParquetReadOptions options, + ReadOptions cometOptions, + Map<String, SQLMetric> metrics) + throws IOException { + this(file, null, options, cometOptions, metrics); + } + + public FileReader( + InputFile file, + ParquetMetadata footer, + ParquetReadOptions options, + ReadOptions cometOptions, + Map<String, SQLMetric> metrics) + throws IOException { + this.converter = new ParquetMetadataConverter(options); + this.file = file; + this.f = file.newStream(); + this.options = options; + this.cometOptions = cometOptions; + this.metrics = metrics; + if (footer == null) { + try { + footer = readFooter(file, options, f, converter); + } catch (Exception e) { + // In case that reading footer throws an exception in the constructor, the new stream + // should be closed. Otherwise, there's no way to close this outside. + f.close(); + throw e; + } + } + this.footer = footer; + this.fileMetaData = footer.getFileMetaData(); + this.fileDecryptor = fileMetaData.getFileDecryptor(); // must be called before filterRowGroups! + if (null != fileDecryptor && fileDecryptor.plaintextFile()) { + this.fileDecryptor = null; // Plaintext file. No need in decryptor + } + + this.blocks = filterRowGroups(footer.getBlocks()); + this.blockIndexStores = listWithNulls(this.blocks.size()); + this.blockRowRanges = listWithNulls(this.blocks.size()); + for (ColumnDescriptor col : footer.getFileMetaData().getSchema().getColumns()) { + paths.put(ColumnPath.get(col.getPath()), col); + } + this.crc = options.usePageChecksumVerification() ? new CRC32() : null; + } + + /** Returns the footer of the Parquet file being read. */ + public ParquetMetadata getFooter() { + return this.footer; + } + + /** Returns the metadata of the Parquet file being read. */ + public FileMetaData getFileMetaData() { + return this.fileMetaData; + } + + /** Returns the input stream of the Parquet file being read. */ + public SeekableInputStream getInputStream() { + return this.f; + } + + /** Returns the Parquet options for reading the file. */ + public ParquetReadOptions getOptions() { + return this.options; + } + + /** Returns all the row groups of this reader (after applying row group filtering). */ + public List<BlockMetaData> getRowGroups() { + return blocks; + } + + /** Sets the projected columns to be read later via {@link #readNextRowGroup()} */ + public void setRequestedSchema(List<ColumnDescriptor> projection) { + paths.clear(); + for (ColumnDescriptor col : projection) { + paths.put(ColumnPath.get(col.getPath()), col); + } + } + + /** + * Gets the total number of records across all row groups (after applying row group filtering). + */ + public long getRecordCount() { + long total = 0; + for (BlockMetaData block : blocks) { + total += block.getRowCount(); + } + return total; + } + + /** + * Gets the total number of records across all row groups (after applying both row group filtering + * and page-level column index filtering). + */ + public long getFilteredRecordCount() { + if (!options.useColumnIndexFilter() + || !FilterCompat.isFilteringRequired(options.getRecordFilter())) { + return getRecordCount(); + } + long total = 0; + for (int i = 0, n = blocks.size(); i < n; ++i) { + total += getRowRanges(i).rowCount(); + } + return total; + } + + /** Skips the next row group. Returns false if there's no row group to skip. Otherwise, true. */ + public boolean skipNextRowGroup() { + return advanceToNextBlock(); + } + + /** + * Returns the next row group to read (after applying row group filtering), or null if there's no + * more row group. + */ + public PageReadStore readNextRowGroup() throws IOException { + if (currentBlock == blocks.size()) { + return null; + } + BlockMetaData block = blocks.get(currentBlock); + if (block.getRowCount() == 0) { + throw new RuntimeException("Illegal row group of 0 rows"); + } + this.currentRowGroup = new RowGroupReader(block.getRowCount()); + // prepare the list of consecutive parts to read them in one scan + List<ConsecutivePartList> allParts = new ArrayList<>(); + ConsecutivePartList currentParts = null; + for (ColumnChunkMetaData mc : block.getColumns()) { + ColumnPath pathKey = mc.getPath(); + ColumnDescriptor columnDescriptor = paths.get(pathKey); + if (columnDescriptor != null) { + BenchmarkCounter.incrementTotalBytes(mc.getTotalSize()); + long startingPos = mc.getStartingPos(); + boolean mergeRanges = cometOptions.isIOMergeRangesEnabled(); + int mergeRangeDelta = cometOptions.getIOMergeRangesDelta(); + + // start a new list if - + // it is the first part or + // the part is consecutive or + // the part is not consecutive but within the merge range + if (currentParts == null + || (!mergeRanges && currentParts.endPos() != startingPos) + || (mergeRanges && startingPos - currentParts.endPos() > mergeRangeDelta)) { + currentParts = new ConsecutivePartList(startingPos); + allParts.add(currentParts); + } + // if we are in a consecutive part list and there is a gap in between the parts, + // we treat the gap as a skippable chunk + long delta = startingPos - currentParts.endPos(); + if (mergeRanges && delta > 0 && delta <= mergeRangeDelta) { + // add a chunk that will be skipped because it has no column descriptor + currentParts.addChunk(new ChunkDescriptor(null, null, startingPos, delta)); + } + currentParts.addChunk( + new ChunkDescriptor(columnDescriptor, mc, startingPos, mc.getTotalSize())); + } + } + // actually read all the chunks + return readChunks(block, allParts, new ChunkListBuilder()); + } + + /** + * Returns the next row group to read (after applying both row group filtering and page level + * column index filtering), or null if there's no more row group. + */ + public PageReadStore readNextFilteredRowGroup() throws IOException { + if (currentBlock == blocks.size()) { + return null; + } + if (!options.useColumnIndexFilter() + || !FilterCompat.isFilteringRequired(options.getRecordFilter())) { + return readNextRowGroup(); + } + BlockMetaData block = blocks.get(currentBlock); + if (block.getRowCount() == 0) { + throw new RuntimeException("Illegal row group of 0 rows"); + } + ColumnIndexStore ciStore = getColumnIndexReader(currentBlock); + RowRanges rowRanges = getRowRanges(currentBlock); + long rowCount = rowRanges.rowCount(); + if (rowCount == 0) { + // There are no matching rows -> skipping this row-group + advanceToNextBlock(); + return readNextFilteredRowGroup(); + } + if (rowCount == block.getRowCount()) { + // All rows are matching -> fall back to the non-filtering path + return readNextRowGroup(); + } + + this.currentRowGroup = new RowGroupReader(rowRanges); + // prepare the list of consecutive parts to read them in one scan + ChunkListBuilder builder = new ChunkListBuilder(); + List<ConsecutivePartList> allParts = new ArrayList<>(); + ConsecutivePartList currentParts = null; + for (ColumnChunkMetaData mc : block.getColumns()) { + ColumnPath pathKey = mc.getPath(); + ColumnDescriptor columnDescriptor = paths.get(pathKey); + if (columnDescriptor != null) { + OffsetIndex offsetIndex = ciStore.getOffsetIndex(mc.getPath()); + IndexFilter indexFilter = new IndexFilter(rowRanges, offsetIndex, block.getRowCount()); + OffsetIndex filteredOffsetIndex = indexFilter.filterOffsetIndex(); + for (IndexFilter.OffsetRange range : + indexFilter.calculateOffsetRanges(filteredOffsetIndex, mc)) { + BenchmarkCounter.incrementTotalBytes(range.length); + long startingPos = range.offset; + // first part or not consecutive => new list + if (currentParts == null || currentParts.endPos() != startingPos) { + currentParts = new ConsecutivePartList(startingPos); + allParts.add(currentParts); + } + ChunkDescriptor chunkDescriptor = + new ChunkDescriptor(columnDescriptor, mc, startingPos, range.length); + currentParts.addChunk(chunkDescriptor); + builder.setOffsetIndex(chunkDescriptor, filteredOffsetIndex); + } + } + } + // actually read all the chunks + return readChunks(block, allParts, builder); + } + + // Visible for testing + ColumnIndexReader getColumnIndexReader(int blockIndex) { + ColumnIndexReader ciStore = blockIndexStores.get(blockIndex); + if (ciStore == null) { + ciStore = ColumnIndexReader.create(blocks.get(blockIndex), paths.keySet(), fileDecryptor, f); + blockIndexStores.set(blockIndex, ciStore); + } + return ciStore; + } + + private PageReadStore readChunks( + BlockMetaData block, List<ConsecutivePartList> allParts, ChunkListBuilder builder) + throws IOException { + for (ConsecutivePartList consecutiveChunks : allParts) { + if (shouldReadParallel()) { + consecutiveChunks.readAllParallel(builder); + } else { + consecutiveChunks.readAll(f, builder); + } + } + for (Chunk chunk : builder.build()) { + readChunkPages(chunk, block); + } + + advanceToNextBlock(); + + return currentRowGroup; + } + + private boolean shouldReadParallel() { + if (file instanceof CometInputFile) { + URI uri = ((CometInputFile) file).getPath().toUri(); + return shouldReadParallel(cometOptions, uri.getScheme()); + } + + return false; + } + + static boolean shouldReadParallel(ReadOptions options, String scheme) { + return options.isParallelIOEnabled() && shouldReadParallelForScheme(scheme); + } + + private static boolean shouldReadParallelForScheme(String scheme) { + if (scheme == null) { + return false; + } + + switch (scheme) { + case "s3a": + // Only enable parallel read for S3, so far. + return true; + default: + return false; + } + } + + private void readChunkPages(Chunk chunk, BlockMetaData block) throws IOException { + if (fileDecryptor == null || fileDecryptor.plaintextFile()) { + currentRowGroup.addColumn(chunk.descriptor.col, chunk.readAllPages()); + return; + } + // Encrypted file + ColumnPath columnPath = ColumnPath.get(chunk.descriptor.col.getPath()); + InternalColumnDecryptionSetup columnDecryptionSetup = fileDecryptor.getColumnSetup(columnPath); + if (!columnDecryptionSetup.isEncrypted()) { // plaintext column + currentRowGroup.addColumn(chunk.descriptor.col, chunk.readAllPages()); + } else { // encrypted column + currentRowGroup.addColumn( + chunk.descriptor.col, + chunk.readAllPages( + columnDecryptionSetup.getMetaDataDecryptor(), + columnDecryptionSetup.getDataDecryptor(), + fileDecryptor.getFileAAD(), + block.getOrdinal(), + columnDecryptionSetup.getOrdinal())); + } + } + + private boolean advanceToNextBlock() { + if (currentBlock == blocks.size()) { + return false; + } + // update the current block and instantiate a dictionary reader for it + ++currentBlock; + return true; + } + + public long[] getRowIndices() { + long[] rowIndices = new long[blocks.size() * 2]; + for (int i = 0, n = blocks.size(); i < n; i++) { + BlockMetaData block = blocks.get(i); + rowIndices[i * 2] = getRowIndexOffset(block); + rowIndices[i * 2 + 1] = block.getRowCount(); + } + return rowIndices; + } + + // Uses reflection to get row index offset from a Parquet block metadata. + // + // The reason reflection is used here is that some Spark versions still depend on a + // Parquet version where the method `getRowIndexOffset` is not public. + private long getRowIndexOffset(BlockMetaData metaData) { + try { + Method method = BlockMetaData.class.getMethod("getRowIndexOffset"); + method.setAccessible(true); + return (long) method.invoke(metaData); + } catch (Exception e) { + throw new RuntimeException("Error when calling getRowIndexOffset", e); + } + } + + private RowRanges getRowRanges(int blockIndex) { + Preconditions.checkState( + FilterCompat.isFilteringRequired(options.getRecordFilter()), + "Should not be invoked if filter is null or NOOP"); + RowRanges rowRanges = blockRowRanges.get(blockIndex); + if (rowRanges == null) { + rowRanges = + ColumnIndexFilter.calculateRowRanges( + options.getRecordFilter(), + getColumnIndexReader(blockIndex), + paths.keySet(), + blocks.get(blockIndex).getRowCount()); + blockRowRanges.set(blockIndex, rowRanges); + } + return rowRanges; + } + + private static ParquetMetadata readFooter( + InputFile file, + ParquetReadOptions options, + SeekableInputStream f, + ParquetMetadataConverter converter) + throws IOException { + long fileLen = file.getLength(); + String filePath = file.toString(); + LOG.debug("File length {}", fileLen); + + int FOOTER_LENGTH_SIZE = 4; + + // MAGIC + data + footer + footerIndex + MAGIC + if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { + throw new RuntimeException( + filePath + " is not a Parquet file (length is too low: " + fileLen + ")"); + } + + // Read footer length and magic string - with a single seek + byte[] magic = new byte[MAGIC.length]; + long fileMetadataLengthIndex = fileLen - magic.length - FOOTER_LENGTH_SIZE; + LOG.debug("reading footer index at {}", fileMetadataLengthIndex); + f.seek(fileMetadataLengthIndex); + int fileMetadataLength = BytesUtils.readIntLittleEndian(f); + f.readFully(magic); + + boolean encryptedFooterMode; + if (Arrays.equals(MAGIC, magic)) { + encryptedFooterMode = false; + } else if (Arrays.equals(EFMAGIC, magic)) { + encryptedFooterMode = true; + } else { + throw new RuntimeException( + filePath + + " is not a Parquet file. Expected magic number " + + "at tail, but found " + + Arrays.toString(magic)); + } + + long fileMetadataIndex = fileMetadataLengthIndex - fileMetadataLength; + LOG.debug("read footer length: {}, footer index: {}", fileMetadataLength, fileMetadataIndex); + if (fileMetadataIndex < magic.length || fileMetadataIndex >= fileMetadataLengthIndex) { + throw new RuntimeException( + "corrupted file: the footer index is not within the file: " + fileMetadataIndex); + } + f.seek(fileMetadataIndex); + + FileDecryptionProperties fileDecryptionProperties = options.getDecryptionProperties(); + InternalFileDecryptor fileDecryptor = null; + if (null != fileDecryptionProperties) { + fileDecryptor = new InternalFileDecryptor(fileDecryptionProperties); + } + + // Read all the footer bytes in one time to avoid multiple read operations, + // since it can be pretty time consuming for a single read operation in HDFS. + byte[] footerBytes = new byte[fileMetadataLength]; + f.readFully(footerBytes); + ByteBuffer footerBytesBuffer = ByteBuffer.wrap(footerBytes); + LOG.debug("Finished to read all footer bytes."); + InputStream footerBytesStream = ByteBufferInputStream.wrap(footerBytesBuffer); + + // Regular file, or encrypted file with plaintext footer + if (!encryptedFooterMode) { + return converter.readParquetMetadata( + footerBytesStream, options.getMetadataFilter(), fileDecryptor, false, fileMetadataLength); + } + + // Encrypted file with encrypted footer + if (fileDecryptor == null) { + throw new ParquetCryptoRuntimeException( + "Trying to read file with encrypted footer. " + "No keys available"); + } + FileCryptoMetaData fileCryptoMetaData = Util.readFileCryptoMetaData(footerBytesStream); + fileDecryptor.setFileCryptoMetaData( + fileCryptoMetaData.getEncryption_algorithm(), true, fileCryptoMetaData.getKey_metadata()); + // footer length is required only for signed plaintext footers + return converter.readParquetMetadata( + footerBytesStream, options.getMetadataFilter(), fileDecryptor, true, 0); + } + + private List<BlockMetaData> filterRowGroups(List<BlockMetaData> blocks) { + FilterCompat.Filter recordFilter = options.getRecordFilter(); + if (FilterCompat.isFilteringRequired(recordFilter)) { + // set up data filters based on configured levels + List<RowGroupFilter.FilterLevel> levels = new ArrayList<>(); + + if (options.useStatsFilter()) { + levels.add(STATISTICS); + } + + if (options.useDictionaryFilter()) { + levels.add(DICTIONARY); + } + + if (options.useBloomFilter()) { + levels.add(BLOOMFILTER); + } + return RowGroupFilter.filterRowGroups(levels, recordFilter, blocks, this); + } + + return blocks; + } + + private static <T> List<T> listWithNulls(int size) { + return Stream.generate(() -> (T) null).limit(size).collect(Collectors.toList()); + } + + public void closeStream() throws IOException { + if (f != null) { + f.close(); + } + } + + @Override + public void close() throws IOException { + try { + if (f != null) { + f.close(); + } + } finally { + options.getCodecFactory().release(); + } + } + + /** + * Builder to concatenate the buffers of the discontinuous parts for the same column. These parts + * are generated as a result of the column-index based filtering when some pages might be skipped + * at reading. + */ + private class ChunkListBuilder { + private class ChunkData { + final List<ByteBuffer> buffers = new ArrayList<>(); + OffsetIndex offsetIndex; + } + + private final Map<ChunkDescriptor, ChunkData> map = new HashMap<>(); + + void add(ChunkDescriptor descriptor, List<ByteBuffer> buffers) { + ChunkListBuilder.ChunkData data = map.get(descriptor); + if (data == null) { + data = new ChunkData(); + map.put(descriptor, data); + } + data.buffers.addAll(buffers); + } + + void setOffsetIndex(ChunkDescriptor descriptor, OffsetIndex offsetIndex) { + ChunkData data = map.get(descriptor); + if (data == null) { + data = new ChunkData(); + map.put(descriptor, data); + } + data.offsetIndex = offsetIndex; + } + + List<Chunk> build() { + List<Chunk> chunks = new ArrayList<>(); + for (Map.Entry<ChunkDescriptor, ChunkListBuilder.ChunkData> entry : map.entrySet()) { + ChunkDescriptor descriptor = entry.getKey(); + ChunkData data = entry.getValue(); + chunks.add(new Chunk(descriptor, data.buffers, data.offsetIndex)); + } + return chunks; + } + } + + /** The data for a column chunk */ + private class Chunk { + private final ChunkDescriptor descriptor; + private final ByteBufferInputStream stream; + final OffsetIndex offsetIndex; + + /** + * @param descriptor descriptor for the chunk + * @param buffers ByteBuffers that contain the chunk + * @param offsetIndex the offset index for this column; might be null + */ + Chunk(ChunkDescriptor descriptor, List<ByteBuffer> buffers, OffsetIndex offsetIndex) { + this.descriptor = descriptor; + this.stream = ByteBufferInputStream.wrap(buffers); + this.offsetIndex = offsetIndex; + } + + protected PageHeader readPageHeader(BlockCipher.Decryptor blockDecryptor, byte[] pageHeaderAAD) + throws IOException { + return Util.readPageHeader(stream, blockDecryptor, pageHeaderAAD); + } + + /** + * Calculate checksum of input bytes, throw decoding exception if it does not match the provided + * reference crc + */ + private void verifyCrc(int referenceCrc, byte[] bytes, String exceptionMsg) { + crc.reset(); + crc.update(bytes); + if (crc.getValue() != ((long) referenceCrc & 0xffffffffL)) { + throw new ParquetDecodingException(exceptionMsg); + } + } + + private ColumnPageReader readAllPages() throws IOException { + return readAllPages(null, null, null, -1, -1); + } + + private ColumnPageReader readAllPages( + BlockCipher.Decryptor headerBlockDecryptor, + BlockCipher.Decryptor pageBlockDecryptor, + byte[] aadPrefix, + int rowGroupOrdinal, + int columnOrdinal) + throws IOException { + List<DataPage> pagesInChunk = new ArrayList<>(); + DictionaryPage dictionaryPage = null; + PrimitiveType type = + fileMetaData.getSchema().getType(descriptor.col.getPath()).asPrimitiveType(); + + long valuesCountReadSoFar = 0; + int dataPageCountReadSoFar = 0; + byte[] dataPageHeaderAAD = null; + if (null != headerBlockDecryptor) { + dataPageHeaderAAD = + AesCipher.createModuleAAD( + aadPrefix, + ModuleCipherFactory.ModuleType.DataPageHeader, + rowGroupOrdinal, + columnOrdinal, + getPageOrdinal(dataPageCountReadSoFar)); + } + while (hasMorePages(valuesCountReadSoFar, dataPageCountReadSoFar)) { + byte[] pageHeaderAAD = dataPageHeaderAAD; + if (null != headerBlockDecryptor) { + // Important: this verifies file integrity (makes sure dictionary page had not been + // removed) + if (null == dictionaryPage && descriptor.metadata.hasDictionaryPage()) { + pageHeaderAAD = + AesCipher.createModuleAAD( + aadPrefix, + ModuleCipherFactory.ModuleType.DictionaryPageHeader, + rowGroupOrdinal, + columnOrdinal, + -1); + } else { + int pageOrdinal = getPageOrdinal(dataPageCountReadSoFar); + AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal); + } + } + + PageHeader pageHeader = readPageHeader(headerBlockDecryptor, pageHeaderAAD); + int uncompressedPageSize = pageHeader.getUncompressed_page_size(); + int compressedPageSize = pageHeader.getCompressed_page_size(); + final BytesInput pageBytes; + switch (pageHeader.type) { + case DICTIONARY_PAGE: + // there is only one dictionary page per column chunk + if (dictionaryPage != null) { + throw new ParquetDecodingException( + "more than one dictionary page in column " + descriptor.col); + } + pageBytes = this.readAsBytesInput(compressedPageSize); + if (options.usePageChecksumVerification() && pageHeader.isSetCrc()) { + verifyCrc( + pageHeader.getCrc(), + pageBytes.toByteArray(), + "could not verify dictionary page integrity, CRC checksum verification failed"); + } + DictionaryPageHeader dicHeader = pageHeader.getDictionary_page_header(); + dictionaryPage = + new DictionaryPage( + pageBytes, + uncompressedPageSize, + dicHeader.getNum_values(), + converter.getEncoding(dicHeader.getEncoding())); + // Copy crc to new page, used for testing + if (pageHeader.isSetCrc()) { + dictionaryPage.setCrc(pageHeader.getCrc()); + } + break; + + case DATA_PAGE: + DataPageHeader dataHeaderV1 = pageHeader.getData_page_header(); + pageBytes = this.readAsBytesInput(compressedPageSize); + if (options.usePageChecksumVerification() && pageHeader.isSetCrc()) { + verifyCrc( + pageHeader.getCrc(), + pageBytes.toByteArray(), + "could not verify page integrity, CRC checksum verification failed"); + } + DataPageV1 dataPageV1 = + new DataPageV1( + pageBytes, + dataHeaderV1.getNum_values(), + uncompressedPageSize, + converter.fromParquetStatistics( + getFileMetaData().getCreatedBy(), dataHeaderV1.getStatistics(), type), + converter.getEncoding(dataHeaderV1.getRepetition_level_encoding()), + converter.getEncoding(dataHeaderV1.getDefinition_level_encoding()), + converter.getEncoding(dataHeaderV1.getEncoding())); + // Copy crc to new page, used for testing + if (pageHeader.isSetCrc()) { + dataPageV1.setCrc(pageHeader.getCrc()); + } + pagesInChunk.add(dataPageV1); + valuesCountReadSoFar += dataHeaderV1.getNum_values(); + ++dataPageCountReadSoFar; + break; + + case DATA_PAGE_V2: + DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); + int dataSize = + compressedPageSize + - dataHeaderV2.getRepetition_levels_byte_length() + - dataHeaderV2.getDefinition_levels_byte_length(); + pagesInChunk.add( + new DataPageV2( + dataHeaderV2.getNum_rows(), + dataHeaderV2.getNum_nulls(), + dataHeaderV2.getNum_values(), + this.readAsBytesInput(dataHeaderV2.getRepetition_levels_byte_length()), + this.readAsBytesInput(dataHeaderV2.getDefinition_levels_byte_length()), + converter.getEncoding(dataHeaderV2.getEncoding()), + this.readAsBytesInput(dataSize), + uncompressedPageSize, + converter.fromParquetStatistics( + getFileMetaData().getCreatedBy(), dataHeaderV2.getStatistics(), type), + dataHeaderV2.isIs_compressed())); + valuesCountReadSoFar += dataHeaderV2.getNum_values(); + ++dataPageCountReadSoFar; + break; + + default: + LOG.debug( + "skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize); + stream.skipFully(compressedPageSize); + break; + } + } + if (offsetIndex == null && valuesCountReadSoFar != descriptor.metadata.getValueCount()) { + // Would be nice to have a CorruptParquetFileException or something as a subclass? + throw new IOException( + "Expected " + + descriptor.metadata.getValueCount() + + " values in column chunk at " + + file + + " offset " + + descriptor.metadata.getFirstDataPageOffset() + + " but got " + + valuesCountReadSoFar + + " values instead over " + + pagesInChunk.size() + + " pages ending at file offset " + + (descriptor.fileOffset + stream.position())); + } + CompressionCodecFactory.BytesInputDecompressor decompressor = + options.getCodecFactory().getDecompressor(descriptor.metadata.getCodec()); + return new ColumnPageReader( + decompressor, + pagesInChunk, + dictionaryPage, + offsetIndex, + blocks.get(currentBlock).getRowCount(), + pageBlockDecryptor, + aadPrefix, + rowGroupOrdinal, + columnOrdinal); + } + + private boolean hasMorePages(long valuesCountReadSoFar, int dataPageCountReadSoFar) { + return offsetIndex == null + ? valuesCountReadSoFar < descriptor.metadata.getValueCount() + : dataPageCountReadSoFar < offsetIndex.getPageCount(); + } + + private int getPageOrdinal(int dataPageCountReadSoFar) { + if (null == offsetIndex) { + return dataPageCountReadSoFar; + } + + return offsetIndex.getPageOrdinal(dataPageCountReadSoFar); + } + + /** + * @param size the size of the page + * @return the page + * @throws IOException if there is an error while reading from the file stream + */ + public BytesInput readAsBytesInput(int size) throws IOException { + return BytesInput.from(stream.sliceBuffers(size)); + } + } + + /** + * Describes a list of consecutive parts to be read at once. A consecutive part may contain whole + * column chunks or only parts of them (some pages). + */ + private class ConsecutivePartList { + private final long offset; + private final List<ChunkDescriptor> chunks = new ArrayList<>(); + private long length; + private final SQLMetric fileReadTimeMetric; + private final SQLMetric fileReadSizeMetric; + private final SQLMetric readThroughput; + + /** + * @param offset where the first chunk starts + */ + ConsecutivePartList(long offset) { + if (metrics != null) { + this.fileReadTimeMetric = metrics.get("ParquetInputFileReadTime"); + this.fileReadSizeMetric = metrics.get("ParquetInputFileReadSize"); + this.readThroughput = metrics.get("ParquetInputFileReadThroughput"); + } else { + this.fileReadTimeMetric = null; + this.fileReadSizeMetric = null; + this.readThroughput = null; + } + this.offset = offset; + } + + /** + * Adds a chunk to the list. It must be consecutive to the previous chunk. + * + * @param descriptor a chunk descriptor + */ + public void addChunk(ChunkDescriptor descriptor) { + chunks.add(descriptor); + length += descriptor.size; + } + + private List<ByteBuffer> allocateReadBuffers() { + int fullAllocations = Math.toIntExact(length / options.getMaxAllocationSize()); + int lastAllocationSize = Math.toIntExact(length % options.getMaxAllocationSize()); + + int numAllocations = fullAllocations + (lastAllocationSize > 0 ? 1 : 0); + List<ByteBuffer> buffers = new ArrayList<>(numAllocations); + + for (int i = 0; i < fullAllocations; i += 1) { + buffers.add(options.getAllocator().allocate(options.getMaxAllocationSize())); + } + + if (lastAllocationSize > 0) { + buffers.add(options.getAllocator().allocate(lastAllocationSize)); + } + return buffers; + } + + /** + * @param f file to read the chunks from + * @param builder used to build chunk list to read the pages for the different columns + * @throws IOException if there is an error while reading from the stream + */ + public void readAll(SeekableInputStream f, ChunkListBuilder builder) throws IOException { + f.seek(offset); + + List<ByteBuffer> buffers = allocateReadBuffers(); + long startNs = System.nanoTime(); + + for (ByteBuffer buffer : buffers) { + f.readFully(buffer); + buffer.flip(); + } + setReadMetrics(startNs); + + // report in a counter the data we just scanned + BenchmarkCounter.incrementBytesRead(length); + ByteBufferInputStream stream = ByteBufferInputStream.wrap(buffers); + for (int i = 0; i < chunks.size(); i++) { + ChunkDescriptor descriptor = chunks.get(i); + if (descriptor.col != null) { + builder.add(descriptor, stream.sliceBuffers(descriptor.size)); + } else { + stream.skipFully(descriptor.size); + } + } + } + + /** + * Api to read a consecutive range from the Parquet file in parallel. This is identical to + * {@link #readAll(SeekableInputStream, ChunkListBuilder) readAll}, except that the consecutive + * range is split into multiple smaller ranges and read in parallel. The parallelism can be set + * by specifying the threadpool size via {@link + * ReadOptions.Builder#withParallelIOThreadPoolSize(int)}. + * + * @param builder used to build chunk list to read the pages for the different columns + * @throws IOException if there is an error while reading from the stream + */ + public void readAllParallel(ChunkListBuilder builder) throws IOException { + + List<ByteBuffer> buffers = allocateReadBuffers(); + long startNs = System.nanoTime(); + + int nThreads = cometOptions.parallelIOThreadPoolSize(); + ExecutorService threadPool = CometFileReaderThreadPool.getOrCreateThreadPool(nThreads); + List<Future<Void>> futures = new ArrayList<>(); + + long currentOffset = this.offset; + int buffersPerThread = buffers.size() / nThreads; + int remaining = buffers.size() % nThreads; + // offset in input file each thread seeks to before beginning read + long[] offsets = new long[nThreads]; + // index of buffer where each thread will start writing data + int[] bufferIndexes = new int[nThreads]; + // number of buffers for each thread to fill + int[] numBuffers = new int[nThreads]; + + int bufferNum = 0; + for (int i = 0; i < nThreads; i++) { + int nBuffers = 0; + offsets[i] = currentOffset; + bufferIndexes[i] = bufferNum; + nBuffers = buffersPerThread; + for (int j = 0; j < buffersPerThread; j++) { + currentOffset += buffers.get(bufferNum).capacity(); + bufferNum++; + } + if (remaining > 0) { + remaining--; + currentOffset += buffers.get(bufferNum).capacity(); + bufferNum++; + nBuffers++; + } + numBuffers[i] = nBuffers; + } + for (int n = 0; n < nThreads; n++) { + int threadIndex = n; + long pos = offsets[threadIndex]; + int bufferIndex = bufferIndexes[threadIndex]; + int nBuffers = numBuffers[threadIndex]; + if (nBuffers == 0) { + continue; + } + + // Find the total number of bytes to read for the current thread + long tmp = 0; + for (int i = 0; i < nBuffers; i++) { + int bufNo = bufferIndex + i; + if (bufNo >= buffers.size()) break; + tmp += buffers.get(bufNo).capacity(); + } + final long length = tmp; + + futures.add( + threadPool.submit( + () -> { + SeekableInputStream inputStream = null; + try { + if (file instanceof CometInputFile) { + inputStream = (((CometInputFile) file).newStream(pos, length)); + } else { + inputStream = file.newStream(); + } + + inputStream.seek(pos); + long curPos = pos; + for (int i = 0; i < nBuffers; i++) { + int bufNo = bufferIndex + i; + if (bufNo >= buffers.size()) { + break; + } + ByteBuffer buffer = buffers.get(bufNo); + LOG.debug( + "Thread: {} Offset: {} Buffer: {} Size: {}", + threadIndex, + curPos, + bufNo, + buffer.capacity()); + curPos += buffer.capacity(); + inputStream.readFully(buffer); + buffer.flip(); + } // for + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + return null; + })); + } + + for (Future<Void> future : futures) { + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + setReadMetrics(startNs); + + ByteBufferInputStream stream; + stream = ByteBufferInputStream.wrap(buffers); + // report in a counter the data we just scanned + BenchmarkCounter.incrementBytesRead(length); + for (int i = 0; i < chunks.size(); i++) { + ChunkDescriptor descriptor = chunks.get(i); + if (descriptor.col != null) { + builder.add(descriptor, stream.sliceBuffers(descriptor.size)); + } else { + stream.skipFully(descriptor.size); + } + } + } + + private void setReadMetrics(long startNs) { + long totalFileReadTimeNs = System.nanoTime() - startNs; + double sizeInMb = ((double) length) / (1024 * 1024); + double timeInSec = ((double) totalFileReadTimeNs) / 1000_0000_0000L; + double throughput = sizeInMb / timeInSec; + LOG.debug( + "Comet: File Read stats: Length: {} MB, Time: {} secs, throughput: {} MB/sec ", + sizeInMb, + timeInSec, + throughput); + if (fileReadTimeMetric != null) { + fileReadTimeMetric.add(totalFileReadTimeNs); + } + if (fileReadSizeMetric != null) { + fileReadSizeMetric.add(length); + } + if (readThroughput != null) { + readThroughput.set(throughput); + } + } + + /** + * @return the position following the last byte of these chunks + */ + public long endPos() { + return offset + length; + } + } + + /** Information needed to read a column chunk or a part of it. */ + private static class ChunkDescriptor { + private final ColumnDescriptor col; + private final ColumnChunkMetaData metadata; + private final long fileOffset; + private final long size; + + /** + * @param col column this chunk is part of + * @param metadata metadata for the column + * @param fileOffset offset in the file where this chunk starts + * @param size size of the chunk + */ + ChunkDescriptor( + ColumnDescriptor col, ColumnChunkMetaData metadata, long fileOffset, long size) { + this.col = col; + this.metadata = metadata; + this.fileOffset = fileOffset; + this.size = size; + } + + @Override + public int hashCode() { + return col.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (obj instanceof ChunkDescriptor) { + return col.equals(((ChunkDescriptor) obj).col); + } else { + return false; + } + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/FooterReader.java b/common/src/main/java/org/apache/comet/parquet/FooterReader.java new file mode 100644 index 0000000000..092eb442ce --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/FooterReader.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.spark.sql.execution.datasources.PartitionedFile; + +/** + * Copied from Spark's `ParquetFooterReader` in order to avoid shading issue around Parquet. + * + * <p>`FooterReader` is a util class which encapsulates the helper methods of reading parquet file + * footer. + */ +public class FooterReader { + public static ParquetMetadata readFooter(Configuration configuration, PartitionedFile file) + throws IOException, URISyntaxException { + long start = file.start(); + long length = file.length(); + Path filePath = new Path(new URI(file.filePath().toString())); + CometInputFile inputFile = CometInputFile.fromPath(filePath, configuration); + ParquetReadOptions readOptions = + HadoopReadOptions.builder(inputFile.getConf(), inputFile.getPath()) + .withRange(start, start + length) + .build(); + ReadOptions cometReadOptions = ReadOptions.builder(configuration).build(); + // Use try-with-resources to ensure fd is closed. + try (FileReader fileReader = new FileReader(inputFile, readOptions, cometReadOptions)) { + return fileReader.getFooter(); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/IndexFilter.java b/common/src/main/java/org/apache/comet/parquet/IndexFilter.java new file mode 100644 index 0000000000..afa5687cae --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/IndexFilter.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.internal.filter2.columnindex.RowRanges; + +public class IndexFilter { + private final RowRanges rowRanges; + private final OffsetIndex offsetIndex; + private final long totalRowCount; + + public IndexFilter(RowRanges rowRanges, OffsetIndex offsetIndex, long totalRowCount) { + this.rowRanges = rowRanges; + this.offsetIndex = offsetIndex; + this.totalRowCount = totalRowCount; + } + + OffsetIndex filterOffsetIndex() { + List<Integer> indexMap = new ArrayList<>(); + for (int i = 0, n = offsetIndex.getPageCount(); i < n; ++i) { + long from = offsetIndex.getFirstRowIndex(i); + if (rowRanges.isOverlapping(from, offsetIndex.getLastRowIndex(i, totalRowCount))) { + indexMap.add(i); + } + } + + int[] indexArray = new int[indexMap.size()]; + for (int i = 0; i < indexArray.length; i++) { + indexArray[i] = indexMap.get(i); + } + return new FilteredOffsetIndex(offsetIndex, indexArray); + } + + List<OffsetRange> calculateOffsetRanges(OffsetIndex filteredOffsetIndex, ColumnChunkMetaData cm) { + List<OffsetRange> ranges = new ArrayList<>(); + long firstPageOffset = offsetIndex.getOffset(0); + int n = filteredOffsetIndex.getPageCount(); + + if (n > 0) { + OffsetRange currentRange = null; + + // Add a range for the dictionary page if required + long rowGroupOffset = cm.getStartingPos(); + if (rowGroupOffset < firstPageOffset) { + currentRange = new OffsetRange(rowGroupOffset, (int) (firstPageOffset - rowGroupOffset)); + ranges.add(currentRange); + } + + for (int i = 0; i < n; ++i) { + long offset = filteredOffsetIndex.getOffset(i); + int length = filteredOffsetIndex.getCompressedPageSize(i); + if (currentRange == null || !currentRange.extend(offset, length)) { + currentRange = new OffsetRange(offset, length); + ranges.add(currentRange); + } + } + } + return ranges; + } + + private static class FilteredOffsetIndex implements OffsetIndex { + private final OffsetIndex offsetIndex; + private final int[] indexMap; + + private FilteredOffsetIndex(OffsetIndex offsetIndex, int[] indexMap) { + this.offsetIndex = offsetIndex; + this.indexMap = indexMap; + } + + @Override + public int getPageOrdinal(int pageIndex) { + return indexMap[pageIndex]; + } + + @Override + public int getPageCount() { + return indexMap.length; + } + + @Override + public long getOffset(int pageIndex) { + return offsetIndex.getOffset(indexMap[pageIndex]); + } + + @Override + public int getCompressedPageSize(int pageIndex) { + return offsetIndex.getCompressedPageSize(indexMap[pageIndex]); + } + + @Override + public long getFirstRowIndex(int pageIndex) { + return offsetIndex.getFirstRowIndex(indexMap[pageIndex]); + } + + @Override + public long getLastRowIndex(int pageIndex, long totalRowCount) { + int nextIndex = indexMap[pageIndex] + 1; + return (nextIndex >= offsetIndex.getPageCount() + ? totalRowCount + : offsetIndex.getFirstRowIndex(nextIndex)) + - 1; + } + } + + static class OffsetRange { + final long offset; + long length; + + private OffsetRange(long offset, int length) { + this.offset = offset; + this.length = length; + } + + private boolean extend(long offset, int length) { + if (this.offset + this.length == offset) { + this.length += length; + return true; + } else { + return false; + } + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java new file mode 100644 index 0000000000..a15d841925 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.IOException; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.vector.CometLazyVector; +import org.apache.comet.vector.CometVector; + +public class LazyColumnReader extends ColumnReader { + + // Remember the largest skipped index for sanity checking. + private int lastSkippedRowId = Integer.MAX_VALUE; + + // Track whether the underlying page is drained. + private boolean isPageDrained = true; + + // Leftover number of rows that did not skip in the previous batch. + private int numRowsToSkipFromPrevBatch; + + // The lazy vector being updated. + private final CometLazyVector vector; + + public LazyColumnReader( + DataType sparkReadType, + ColumnDescriptor descriptor, + int batchSize, + boolean useDecimal128, + boolean useLegacyDateTimestamp) { + super(sparkReadType, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + this.batchSize = 0; // the batch size is set later in `readBatch` + this.vector = new CometLazyVector(sparkReadType, this, useDecimal128); + } + + @Override + public void setPageReader(PageReader pageReader) throws IOException { + super.setPageReader(pageReader); + lastSkippedRowId = Integer.MAX_VALUE; + isPageDrained = true; + numRowsToSkipFromPrevBatch = 0; + currentNumValues = batchSize; + } + + /** + * Lazily read a batch of 'total' rows for this column. The includes: 1) Skip any unused rows from + * the previous batch 2) Reset the native columnar batch 3) Reset tracking variables + * + * @param total the number of rows in the batch. MUST be <= the number of rows available in this + * column chunk. + */ + @Override + public void readBatch(int total) { + // Before starting a new batch, take care of the remaining rows to skip from the previous batch. + tryPageSkip(batchSize); + numRowsToSkipFromPrevBatch += batchSize - currentNumValues; + + // Now first reset the current columnar batch so that it can be used to fill in a new batch + // of values. Then, keep reading more data pages (via 'readBatch') until the current batch is + // full, or we have read 'total' number of values. + Native.resetBatch(nativeHandle); + + batchSize = total; + currentNumValues = 0; + lastSkippedRowId = -1; + } + + @Override + public CometVector currentBatch() { + return vector; + } + + /** Read all rows up to the `batchSize`. Expects no rows are skipped so far. */ + public void readAllBatch() { + // All rows should be read without any skips so far + assert (lastSkippedRowId == -1); + + readBatch(batchSize - 1, 0); + } + + /** + * Read at least up to `rowId`. It may read beyond `rowId` if enough rows available in the page. + * It may skip reading rows before `rowId`. In case `rowId` is already read, return immediately. + * + * @param rowId the row index in the batch to read. + * @return true if `rowId` is newly materialized, or false if `rowId` is already materialized. + */ + public boolean materializeUpToIfNecessary(int rowId) { + // Not allowed reading rowId if it may have skipped previously. + assert (rowId > lastSkippedRowId); + + // If `rowId` is already materialized, return immediately. + if (rowId < currentNumValues) return false; + + int numRowsWholePageSkipped = tryPageSkip(rowId); + readBatch(rowId, numRowsWholePageSkipped); + return true; + } + + /** + * Read up to `rowId` (inclusive). If the whole pages are skipped previously in `tryPageSkip()`, + * pad the number of whole page skipped rows with nulls to the underlying vector before reading. + * + * @param rowId the row index in the batch to read. + * @param numNullRowsToPad the number of nulls to pad before reading. + */ + private void readBatch(int rowId, int numNullRowsToPad) { + if (numRowsToSkipFromPrevBatch > 0) { + // Reaches here only when starting a new batch and the page is previously drained + readPage(); + isPageDrained = false; + Native.skipBatch(nativeHandle, numRowsToSkipFromPrevBatch, true); + numRowsToSkipFromPrevBatch = 0; + } + while (rowId >= currentNumValues) { + int numRowsToRead = batchSize - currentNumValues; + if (isPageDrained) { + readPage(); + } + int[] array = Native.readBatch(nativeHandle, numRowsToRead, numNullRowsToPad); + int read = array[0]; + isPageDrained = read < numRowsToRead; + currentNumValues += read; + currentNumNulls += array[1]; + // No need to update numNullRowsToPad. numNullRowsToPad > 0 means there were whole page skips. + // That guarantees that the Native.readBatch can read up to rowId in the current page. + } + } + + /** + * Try to skip until `rowId` (exclusive). If possible, it skips whole underlying pages without + * decompressing. In that case, it returns early at the page end, so that the next iteration can + * lazily decide to `readPage()` or `tryPageSkip()` again. + * + * @param rowId the row index in the batch that it tries to skip up until (exclusive). + * @return the number of rows that the whole page skips were applied. + */ + private int tryPageSkip(int rowId) { + int total = rowId - currentNumValues; + int wholePageSkipped = 0; + if (total > 0) { + // First try to skip from the non-drained underlying page. + int skipped = isPageDrained ? 0 : Native.skipBatch(nativeHandle, total); + total -= skipped; + isPageDrained = total > 0; + if (isPageDrained) { + ColumnPageReader columnPageReader = (ColumnPageReader) pageReader; + // It is always `columnPageReader.getPageValueCount() > numRowsToSkipFromPriorBatch` + int pageValueCount = columnPageReader.getPageValueCount() - numRowsToSkipFromPrevBatch; + while (pageValueCount <= total) { + // skip the entire page if the next page is small enough + columnPageReader.skipPage(); + numRowsToSkipFromPrevBatch = 0; + total -= pageValueCount; + wholePageSkipped += pageValueCount; + pageValueCount = columnPageReader.getPageValueCount(); + } + } + + currentNumValues += skipped + wholePageSkipped; + currentNumNulls += skipped; + lastSkippedRowId = currentNumValues - 1; + } + return wholePageSkipped; + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java new file mode 100644 index 0000000000..b8722ca783 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import org.apache.arrow.c.ArrowArray; +import org.apache.arrow.c.ArrowSchema; +import org.apache.arrow.c.Data; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.vector.CometPlainVector; +import org.apache.comet.vector.CometVector; + +/** A metadata column reader that can be extended by {@link RowIndexColumnReader} etc. */ +public class MetadataColumnReader extends AbstractColumnReader { + private final BufferAllocator allocator = new RootAllocator(); + private CometVector vector; + + public MetadataColumnReader(DataType type, ColumnDescriptor descriptor, boolean useDecimal128) { + // TODO: should we handle legacy dates & timestamps for metadata columns? + super(type, descriptor, useDecimal128, false); + } + + @Override + public void setBatchSize(int batchSize) { + close(); + super.setBatchSize(batchSize); + } + + @Override + public void readBatch(int total) { + if (vector == null) { + long[] addresses = Native.currentBatch(nativeHandle); + try (ArrowArray array = ArrowArray.wrap(addresses[0]); + ArrowSchema schema = ArrowSchema.wrap(addresses[1])) { + FieldVector fieldVector = Data.importVector(allocator, array, schema, null); + vector = new CometPlainVector(fieldVector, useDecimal128); + } + } + vector.setNumValues(total); + } + + void setNumNulls(int total) { + vector.setNumNulls(total); + } + + @Override + public CometVector currentBatch() { + return vector; + } + + @Override + public void close() { + if (vector != null) { + vector.close(); + vector = null; + } + super.close(); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/Native.java b/common/src/main/java/org/apache/comet/parquet/Native.java new file mode 100644 index 0000000000..0887ae12f5 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/Native.java @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.nio.ByteBuffer; + +import org.apache.comet.NativeBase; + +public final class Native extends NativeBase { + public static int[] readBatch(long handle, int batchSize) { + return readBatch(handle, batchSize, 0); + } + + public static int skipBatch(long handle, int batchSize) { + return skipBatch(handle, batchSize, false); + } + + /** Native APIs * */ + + /** + * Creates a reader for a primitive Parquet column. + * + * @param physicalTypeId id for Parquet physical type + * @param logicalTypeId id for Parquet logical type + * @param expectedPhysicalTypeId id for Parquet physical type, converted from Spark read type. + * This is used for type promotion. + * @param path the path from the root schema to the column, derived from the method + * 'ColumnDescriptor#getPath()'. + * @param maxDl the maximum definition level of the primitive column + * @param maxRl the maximum repetition level of the primitive column + * @param bitWidth (only set when logical type is INT) the bit width for the integer type (INT8, + * INT16, INT32, etc) + * @param isSigned (only set when logical type is INT) whether it is signed or unsigned int. + * @param typeLength number of bytes required to store a value of the type, only set when the + * physical type is FIXED_LEN_BYTE_ARRAY, otherwise it's 0. + * @param precision (only set when logical type is DECIMAL) precision of the decimal type + * @param expectedPrecision (only set when logical type is DECIMAL) precision of the decimal type + * from Spark read schema. This is used for type promotion. + * @param scale (only set when logical type is DECIMAL) scale of the decimal type + * @param tu (only set when logical type is TIMESTAMP) unit for the timestamp + * @param isAdjustedUtc (only set when logical type is TIMESTAMP) whether the timestamp is + * adjusted to UTC or not + * @param batchSize the batch size for the columnar read + * @param useDecimal128 whether to always return 128 bit decimal regardless of precision + * @param useLegacyDateTimestampOrNTZ whether to read legacy dates/timestamps as it is + * @return a pointer to a native Parquet column reader created + */ + public static native long initColumnReader( + int physicalTypeId, + int logicalTypeId, + int expectedPhysicalTypeId, + String[] path, + int maxDl, + int maxRl, + int bitWidth, + boolean isSigned, + int typeLength, + int precision, + int expectedPrecision, + int scale, + int tu, + boolean isAdjustedUtc, + int batchSize, + boolean useDecimal128, + boolean useLegacyDateTimestampOrNTZ); + + /** + * Pass a Parquet dictionary page to the native column reader. Note this should only be called + * once per Parquet column chunk. Otherwise it'll panic. + * + * @param handle the handle to the native Parquet column reader + * @param dictionaryValueCount the number of values in this dictionary + * @param dictionaryData the actual dictionary page data, including repetition/definition levels + * as well as values + * @param encoding the encoding used by the dictionary + */ + public static native void setDictionaryPage( + long handle, int dictionaryValueCount, byte[] dictionaryData, int encoding); + + /** + * Passes a Parquet data page V1 to the native column reader. + * + * @param handle the handle to the native Parquet column reader + * @param pageValueCount the number of values in this data page + * @param pageData the actual page data, which should only contain PLAIN-encoded values. + * @param valueEncoding the encoding used by the values + */ + public static native void setPageV1( + long handle, int pageValueCount, byte[] pageData, int valueEncoding); + + /** + * Passes a Parquet data page V1 to the native column reader. + * + * @param handle the handle to the native Parquet column reader + * @param pageValueCount the number of values in this data page + * @param buffer the actual page data, represented by a DirectByteBuffer. + * @param valueEncoding the encoding used by the values + */ + public static native void setPageBufferV1( + long handle, int pageValueCount, ByteBuffer buffer, int valueEncoding); + + /** + * Passes a Parquet data page V2 to the native column reader. + * + * @param handle the handle to the native Parquet column reader + * @param pageValueCount the number of values in this data page + * @param defLevelData the data for definition levels + * @param repLevelData the data for repetition levels + * @param valueData the data for values + * @param valueEncoding the encoding used by the values + */ + public static native void setPageV2( + long handle, + int pageValueCount, + byte[] defLevelData, + byte[] repLevelData, + byte[] valueData, + int valueEncoding); + + /** + * Reset the current columnar batch. This will clear all the content of the batch as well as any + * internal state such as the current offset. + * + * @param handle the handle to the native Parquet column reader + */ + public static native void resetBatch(long handle); + + /** + * Reads at most 'batchSize' number of rows from the native Parquet column reader. Returns a tuple + * where the first element is the actual number of rows read (including both nulls and non-nulls), + * and the second element is the number of nulls read. + * + * <p>If the returned value is < 'batchSize' then it means the current page has been completely + * drained. In this case, the caller should call {@link Native#setPageV1} or {@link + * Native#setPageV2} before the next 'readBatch' call. + * + * <p>Note that the current page could also be drained if the returned value = 'batchSize', i.e., + * the remaining number of rows in the page is exactly equal to 'batchSize'. In this case, the + * next 'readBatch' call will return 0 and the caller should call {@link Native#setPageV1} or + * {@link Native#setPageV2} next. + * + * <p>If `nullPadSize` > 0, it pads nulls into the underlying vector before the values will be + * read into. + * + * @param handle the handle to the native Parquet column reader + * @param batchSize the number of rows to be read + * @param nullPadSize the number of nulls to pad before reading. + * @return a tuple: (the actual number of rows read, the number of nulls read) + */ + public static native int[] readBatch(long handle, int batchSize, int nullPadSize); + + /** + * Skips at most 'batchSize' number of rows from the native Parquet column reader, and returns the + * actual number of rows skipped. + * + * <p>If the returned value is < 'batchSize' then it means the current page has been completely + * drained. In this case, the caller should call {@link Native#setPageV1} or {@link + * Native#setPageV2} before the next 'skipBatch' call. + * + * <p>Note that the current page could also be drained if the returned value = 'batchSize', i.e., + * the remaining number of rows in the page is exactly equal to 'batchSize'. In this case, the + * next 'skipBatch' call will return 0 and the caller should call {@link Native#setPageV1} or + * {@link Native#setPageV2} next. + * + * @param handle the handle to the native Parquet column reader + * @param batchSize the number of rows to skip in the current page + * @param discard if true, discard read rows without padding nulls into the underlying vector + * @return the actual number of rows skipped + */ + public static native int skipBatch(long handle, int batchSize, boolean discard); + + /** + * Returns the current batch constructed via 'readBatch' + * + * @param handle the handle to the native Parquet column reader + * @return a long array with 2 elements, the first is the address to native Arrow array, and the + * second is the address to the Arrow schema. + */ + public static native long[] currentBatch(long handle); + + /** Set methods to set a constant value for the reader, so it'll return constant vectors */ + public static native void setNull(long handle); + + public static native void setBoolean(long handle, boolean value); + + public static native void setByte(long handle, byte value); + + public static native void setShort(long handle, short value); + + public static native void setInt(long handle, int value); + + public static native void setLong(long handle, long value); + + public static native void setFloat(long handle, float value); + + public static native void setDouble(long handle, double value); + + public static native void setBinary(long handle, byte[] value); + + /** Set decimal backed by FixedLengthByteArray */ + public static native void setDecimal(long handle, byte[] value); + + /** Set position of row index vector for Iceberg Metadata Column */ + public static native void setPosition(long handle, long value, int size); + + /** Set row index vector for Spark row index metadata column and return vector size */ + public static native int setIndices(long handle, long offset, int size, long[] indices); + + /** Set deleted info for Iceberg Metadata Column */ + public static native void setIsDeleted(long handle, boolean[] isDeleted); + + /** + * Closes the native Parquet column reader and releases all resources associated with it. + * + * @param handle the handle to the native Parquet column reader + */ + public static native void closeColumnReader(long handle); +} diff --git a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java new file mode 100644 index 0000000000..6754443e69 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.SparkEnv; +import org.apache.spark.launcher.SparkLauncher; + +/** + * Comet specific Parquet related read options. + * + * <p>TODO: merge this with {@link org.apache.parquet.HadoopReadOptions} once PARQUET-2203 is done. + */ +public class ReadOptions { + private static final Logger LOG = LoggerFactory.getLogger(ReadOptions.class); + public static final String COMET_PARQUET_PARALLEL_IO_ENABLED = + "comet.parquet.read.parallel.io.enabled"; + public static final boolean COMET_PARQUET_PARALLEL_IO_ENABLED_DEFAULT = true; + + public static final String COMET_PARQUET_PARALLEL_IO_THREADS = + "comet.parquet.read.parallel.io.thread-pool.size"; + public static final int COMET_PARQUET_PARALLEL_IO_THREADS_DEFAULT = 32; + + public static final String COMET_IO_MERGE_RANGES = "comet.parquet.read.io.mergeRanges"; + private static final boolean COMET_IO_MERGE_RANGES_DEFAULT = true; + + public static final String COMET_IO_MERGE_RANGES_DELTA = + "comet.parquet.read.io.mergeRanges.delta"; + private static final int COMET_IO_MERGE_RANGES_DELTA_DEFAULT = 1 << 23; // 8 MB + + // Max number of concurrent tasks we expect. Used to autoconfigure S3 client connections + public static final int S3A_MAX_EXPECTED_PARALLELISM = 32; + // defined in hadoop-aws - org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS + public static final String S3A_MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; + // default max connections in S3A - org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS + public static final int S3A_DEFAULT_MAX_HTTP_CONNECTIONS = 96; + + public static final String S3A_READAHEAD_RANGE = "fs.s3a.readahead.range"; + // Default read ahead range in Hadoop is 64K; we increase it to 1 MB + public static final long COMET_DEFAULT_READAHEAD_RANGE = 1 * 1024 * 1024; // 1 MB + + private final boolean parallelIOEnabled; + private final int parallelIOThreadPoolSize; + private final boolean ioMergeRanges; + private final int ioMergeRangesDelta; + + ReadOptions( + boolean parallelIOEnabled, + int parallelIOThreadPoolSize, + boolean ioMergeRanges, + int ioMergeRangesDelta) { + this.parallelIOEnabled = parallelIOEnabled; + this.parallelIOThreadPoolSize = parallelIOThreadPoolSize; + this.ioMergeRanges = ioMergeRanges; + this.ioMergeRangesDelta = ioMergeRangesDelta; + } + + public boolean isParallelIOEnabled() { + return this.parallelIOEnabled; + } + + public int parallelIOThreadPoolSize() { + return this.parallelIOThreadPoolSize; + } + + public boolean isIOMergeRangesEnabled() { + return ioMergeRanges; + } + + public int getIOMergeRangesDelta() { + return ioMergeRangesDelta; + } + + public static Builder builder(Configuration conf) { + return new Builder(conf); + } + + public static Builder builder() { + return builder(new Configuration()); + } + + public static class Builder { + private final Configuration conf; + + private boolean parallelIOEnabled; + private int parallelIOThreadPoolSize; + private boolean ioMergeRanges; + private int ioMergeRangesDelta; + + /** + * Whether to enable Parquet parallel IO when reading row groups. If true, Parquet reader will + * use multiple threads to read multiple chunks of data from the current row group in parallel. + */ + public Builder enableParallelIO(boolean b) { + this.parallelIOEnabled = b; + return this; + } + + /** + * Specify the number of threads to be used in parallel IO. + * + * <p><b>Note</b>: this will only be effective if parallel IO is enabled (e.g., via {@link + * #enableParallelIO(boolean)}). + */ + public Builder withParallelIOThreadPoolSize(int numThreads) { + this.parallelIOThreadPoolSize = numThreads; + return this; + } + + public Builder enableIOMergeRanges(boolean enableIOMergeRanges) { + this.ioMergeRanges = enableIOMergeRanges; + return this; + } + + public Builder withIOMergeRangesDelta(int ioMergeRangesDelta) { + this.ioMergeRangesDelta = ioMergeRangesDelta; + return this; + } + + public ReadOptions build() { + return new ReadOptions( + parallelIOEnabled, parallelIOThreadPoolSize, ioMergeRanges, ioMergeRangesDelta); + } + + public Builder(Configuration conf) { + this.conf = conf; + this.parallelIOEnabled = + conf.getBoolean( + COMET_PARQUET_PARALLEL_IO_ENABLED, COMET_PARQUET_PARALLEL_IO_ENABLED_DEFAULT); + this.parallelIOThreadPoolSize = + conf.getInt(COMET_PARQUET_PARALLEL_IO_THREADS, COMET_PARQUET_PARALLEL_IO_THREADS_DEFAULT); + this.ioMergeRanges = conf.getBoolean(COMET_IO_MERGE_RANGES, COMET_IO_MERGE_RANGES_DEFAULT); + this.ioMergeRangesDelta = + conf.getInt(COMET_IO_MERGE_RANGES_DELTA, COMET_IO_MERGE_RANGES_DELTA_DEFAULT); + // override some S3 defaults + setS3Config(); + } + + // For paths to S3, if the s3 connection pool max is less than twice the product of + // parallel reader threads * number of cores, then increase the connection pool max + private void setS3Config() { + int s3ConnectionsMax = S3A_DEFAULT_MAX_HTTP_CONNECTIONS; + SparkEnv env = SparkEnv.get(); + // Use a default number of cores in case we are using the FileReader outside the context + // of Spark. + int numExecutorCores = S3A_MAX_EXPECTED_PARALLELISM; + if (env != null) { + numExecutorCores = env.conf().getInt(SparkLauncher.EXECUTOR_CORES, numExecutorCores); + } + int parallelReaderThreads = this.parallelIOEnabled ? this.parallelIOThreadPoolSize : 1; + s3ConnectionsMax = Math.max(numExecutorCores * parallelReaderThreads * 2, s3ConnectionsMax); + + setS3ConfIfGreater(conf, S3A_MAXIMUM_CONNECTIONS, s3ConnectionsMax); + setS3ConfIfGreater(conf, S3A_READAHEAD_RANGE, COMET_DEFAULT_READAHEAD_RANGE); + } + + // Update the conf iff the new value is greater than the existing val + private void setS3ConfIfGreater(Configuration conf, String key, int newVal) { + int maxVal = newVal; + String curr = conf.get(key); + if (curr != null && !curr.isEmpty()) { + maxVal = Math.max(Integer.parseInt(curr), newVal); + } + LOG.info("File reader auto configured '{}={}'", key, maxVal); + conf.set(key, Integer.toString(maxVal)); + } + + // Update the conf iff the new value is greater than the existing val. This handles values that + // may have suffixes (K, M, G, T, P, E) indicating well known bytes size suffixes + private void setS3ConfIfGreater(Configuration conf, String key, long newVal) { + long maxVal = conf.getLongBytes(key, newVal); + maxVal = Math.max(maxVal, newVal); + LOG.info("File reader auto configured '{}={}'", key, maxVal); + conf.set(key, Long.toString(maxVal)); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java b/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java new file mode 100644 index 0000000000..40a61c05e0 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.compat.FilterCompat.Filter; +import org.apache.parquet.filter2.compat.FilterCompat.NoOpFilter; +import org.apache.parquet.filter2.compat.FilterCompat.Visitor; +import org.apache.parquet.filter2.dictionarylevel.DictionaryFilter; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator; +import org.apache.parquet.filter2.statisticslevel.StatisticsFilter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.schema.MessageType; + +public class RowGroupFilter implements Visitor<List<BlockMetaData>> { + private final List<BlockMetaData> blocks; + private final MessageType schema; + private final List<FilterLevel> levels; + private final FileReader reader; + + public enum FilterLevel { + STATISTICS, + DICTIONARY, + BLOOMFILTER + } + + public static List<BlockMetaData> filterRowGroups( + List<FilterLevel> levels, Filter filter, List<BlockMetaData> blocks, FileReader reader) { + return filter.accept(new RowGroupFilter(levels, blocks, reader)); + } + + private RowGroupFilter(List<FilterLevel> levels, List<BlockMetaData> blocks, FileReader reader) { + this.levels = levels; + this.blocks = blocks; + this.reader = reader; + this.schema = reader.getFileMetaData().getSchema(); + } + + @Override + public List<BlockMetaData> visit(FilterCompat.FilterPredicateCompat filterPredicateCompat) { + FilterPredicate filterPredicate = filterPredicateCompat.getFilterPredicate(); + + // check that the schema of the filter matches the schema of the file + SchemaCompatibilityValidator.validate(filterPredicate, schema); + + List<BlockMetaData> filteredBlocks = new ArrayList<>(); + + for (BlockMetaData block : blocks) { + boolean drop = false; + + if (levels.contains(FilterLevel.STATISTICS)) { + drop = StatisticsFilter.canDrop(filterPredicate, block.getColumns()); + } + + if (!drop && levels.contains(FilterLevel.DICTIONARY)) { + drop = + DictionaryFilter.canDrop( + filterPredicate, + block.getColumns(), + new DictionaryPageReader( + block, + reader.getFileMetaData().getFileDecryptor(), + reader.getInputStream(), + reader.getOptions())); + } + + if (!drop && levels.contains(FilterLevel.BLOOMFILTER)) { + drop = + filterPredicate.accept( + new BloomFilterReader( + block, reader.getFileMetaData().getFileDecryptor(), reader.getInputStream())); + } + + if (!drop) { + filteredBlocks.add(block); + } + } + + return filteredBlocks; + } + + @Override + public List<BlockMetaData> visit( + FilterCompat.UnboundRecordFilterCompat unboundRecordFilterCompat) { + return blocks; + } + + @Override + public List<BlockMetaData> visit(NoOpFilter noOpFilter) { + return blocks; + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java new file mode 100644 index 0000000000..d5d73b0783 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.PrimitiveIterator; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.internal.filter2.columnindex.RowRanges; + +public class RowGroupReader implements PageReadStore { + private final Map<ColumnDescriptor, PageReader> readers = new HashMap<>(); + private final long rowCount; + private final RowRanges rowRanges; + + public RowGroupReader(long rowCount) { + this.rowCount = rowCount; + this.rowRanges = null; + } + + RowGroupReader(RowRanges rowRanges) { + this.rowRanges = rowRanges; + this.rowCount = rowRanges.rowCount(); + } + + @Override + public long getRowCount() { + return rowCount; + } + + @Override + public PageReader getPageReader(ColumnDescriptor path) { + final PageReader pageReader = readers.get(path); + if (pageReader == null) { + throw new IllegalArgumentException( + path + " is not found: " + readers.keySet() + " " + rowCount); + } + return pageReader; + } + + @Override + public Optional<PrimitiveIterator.OfLong> getRowIndexes() { + return rowRanges == null ? Optional.empty() : Optional.of(rowRanges.iterator()); + } + + void addColumn(ColumnDescriptor path, ColumnPageReader reader) { + if (readers.put(path, reader) != null) { + throw new IllegalStateException(path + " was already added"); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java new file mode 100644 index 0000000000..8448318dbd --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/RowIndexColumnReader.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import org.apache.spark.sql.types.*; + +/** + * A column reader that returns the row index vector. Used for reading row index metadata column for + * Spark 3.4+. The row index can be accessed by {@code _tmp_metadata_row_index} column. + */ +public class RowIndexColumnReader extends MetadataColumnReader { + /** The row indices that are used to initialize this column reader. */ + private final long[] indices; + + /** The current number of indices to skip reading from {@code indices}. */ + private long offset; + + public RowIndexColumnReader(StructField field, int batchSize, long[] indices) { + super(field.dataType(), TypeUtil.convertToParquet(field), false); + this.indices = indices; + setBatchSize(batchSize); + } + + @Override + public void readBatch(int total) { + Native.resetBatch(nativeHandle); + int count = Native.setIndices(nativeHandle, offset, total, indices); + offset += count; + + super.readBatch(count); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/SupportsComet.java b/common/src/main/java/org/apache/comet/parquet/SupportsComet.java new file mode 100644 index 0000000000..f330a76819 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/SupportsComet.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +public interface SupportsComet { + boolean isCometEnabled(); +} diff --git a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java new file mode 100644 index 0000000000..340bf98231 --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java @@ -0,0 +1,279 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.util.Arrays; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.*; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; +import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException; +import org.apache.spark.sql.types.*; + +import org.apache.comet.CometConf; + +public class TypeUtil { + + /** Converts the input Spark 'field' into a Parquet column descriptor. */ + public static ColumnDescriptor convertToParquet(StructField field) { + Type.Repetition repetition; + int maxDefinitionLevel; + if (field.nullable()) { + repetition = Type.Repetition.OPTIONAL; + maxDefinitionLevel = 1; + } else { + repetition = Type.Repetition.REQUIRED; + maxDefinitionLevel = 0; + } + String[] path = new String[] {field.name()}; + + DataType type = field.dataType(); + + Types.PrimitiveBuilder<PrimitiveType> builder = null; + // Only partition column can be `NullType`, which also uses `ConstantColumnReader`. Here we + // piggy-back onto Parquet boolean type for constant vector of null values, we don't really + // care what Parquet type it is. + if (type == DataTypes.BooleanType || type == DataTypes.NullType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, repetition); + } else if (type == DataTypes.IntegerType || type instanceof YearMonthIntervalType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition); + } else if (type == DataTypes.DateType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(LogicalTypeAnnotation.dateType()); + } else if (type == DataTypes.ByteType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(LogicalTypeAnnotation.intType(8, true)); + } else if (type == DataTypes.ShortType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(LogicalTypeAnnotation.intType(16, true)); + } else if (type == DataTypes.LongType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition); + } else if (type == DataTypes.BinaryType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition); + } else if (type == DataTypes.StringType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) + .as(LogicalTypeAnnotation.stringType()); + } else if (type == DataTypes.FloatType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.FLOAT, repetition); + } else if (type == DataTypes.DoubleType) { + builder = Types.primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, repetition); + } else if (type == DataTypes.TimestampType) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)); + } else if (type == TimestampNTZType$.MODULE$) { + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(false, TimeUnit.MICROS)); + } else if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType) type; + builder = + Types.primitive(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, repetition) + .length(16) // always store as Decimal128 + .as(LogicalTypeAnnotation.decimalType(decimalType.scale(), decimalType.precision())); + } + if (builder == null) { + throw new UnsupportedOperationException("Unsupported input Spark type: " + type); + } + + return new ColumnDescriptor(path, builder.named(field.name()), 0, maxDefinitionLevel); + } + + /** + * Check whether the Parquet 'descriptor' and Spark read type 'sparkType' are compatible. If not, + * throw exception. + * + * <p>This mostly follows the logic in Spark's + * ParquetVectorUpdaterFactory#getUpdater(ColumnDescriptor, DataType) + * + * @param descriptor descriptor for a Parquet primitive column + * @param sparkType Spark read type + */ + public static void checkParquetType(ColumnDescriptor descriptor, DataType sparkType) { + PrimitiveType.PrimitiveTypeName typeName = descriptor.getPrimitiveType().getPrimitiveTypeName(); + LogicalTypeAnnotation logicalTypeAnnotation = + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + boolean allowTypePromotion = (boolean) CometConf.COMET_SCHEMA_EVOLUTION_ENABLED().get(); + + if (sparkType instanceof NullType) { + return; + } + + switch (typeName) { + case BOOLEAN: + if (sparkType == DataTypes.BooleanType) return; + break; + case INT32: + if (sparkType == DataTypes.IntegerType || canReadAsIntDecimal(descriptor, sparkType)) { + return; + } else if (sparkType == DataTypes.LongType + && isUnsignedIntTypeMatched(logicalTypeAnnotation, 32)) { + // In `ParquetToSparkSchemaConverter`, we map parquet UINT32 to our LongType. + // For unsigned int32, it stores as plain signed int32 in Parquet when dictionary + // fallbacks. We read them as long values. + return; + } else if (sparkType == DataTypes.LongType && allowTypePromotion) { + // In Comet we allow schema evolution from int to long, if + // `spark.comet.schemaEvolution.enabled` is enabled. + return; + } else if (sparkType == DataTypes.ByteType || sparkType == DataTypes.ShortType) { + return; + } else if (sparkType == DataTypes.DateType) { + // TODO: use dateTimeRebaseMode from Spark side + return; + } else if (sparkType instanceof YearMonthIntervalType) { + return; + } + break; + case INT64: + if (sparkType == DataTypes.LongType || canReadAsLongDecimal(descriptor, sparkType)) { + return; + } else if (isLongDecimal(sparkType) + && isUnsignedIntTypeMatched(logicalTypeAnnotation, 64)) { + // In `ParquetToSparkSchemaConverter`, we map parquet UINT64 to our Decimal(20, 0). + // For unsigned int64, it stores as plain signed int64 in Parquet when dictionary + // fallbacks. We read them as decimal values. + return; + } else if (isTimestampTypeMatched(logicalTypeAnnotation, TimeUnit.MICROS)) { + validateTimestampType(logicalTypeAnnotation, sparkType); + // TODO: use dateTimeRebaseMode from Spark side + return; + } else if (isTimestampTypeMatched(logicalTypeAnnotation, TimeUnit.MILLIS)) { + validateTimestampType(logicalTypeAnnotation, sparkType); + return; + } + break; + case INT96: + if (sparkType == TimestampNTZType$.MODULE$) { + convertErrorForTimestampNTZ(typeName.name()); + } else if (sparkType == DataTypes.TimestampType) { + return; + } + break; + case FLOAT: + if (sparkType == DataTypes.FloatType) return; + // In Comet we allow schema evolution from float to double, if + // `spark.comet.schemaEvolution.enabled` is enabled. + if (sparkType == DataTypes.DoubleType && allowTypePromotion) return; + break; + case DOUBLE: + if (sparkType == DataTypes.DoubleType) return; + break; + case BINARY: + if (sparkType == DataTypes.StringType + || sparkType == DataTypes.BinaryType + || canReadAsBinaryDecimal(descriptor, sparkType)) { + return; + } + break; + case FIXED_LEN_BYTE_ARRAY: + if (canReadAsIntDecimal(descriptor, sparkType) + || canReadAsLongDecimal(descriptor, sparkType) + || canReadAsBinaryDecimal(descriptor, sparkType) + || sparkType == DataTypes.BinaryType + // for uuid, since iceberg maps uuid to StringType + || sparkType == DataTypes.StringType) { + return; + } + break; + default: + break; + } + + throw new SchemaColumnConvertNotSupportedException( + Arrays.toString(descriptor.getPath()), + descriptor.getPrimitiveType().getPrimitiveTypeName().toString(), + sparkType.catalogString()); + } + + private static void validateTimestampType( + LogicalTypeAnnotation logicalTypeAnnotation, DataType sparkType) { + assert (logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation); + // Throw an exception if the Parquet type is TimestampLTZ and the Catalyst type is TimestampNTZ. + // This is to avoid mistakes in reading the timestamp values. + if (((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).isAdjustedToUTC() + && sparkType == TimestampNTZType$.MODULE$) { + convertErrorForTimestampNTZ("int64 time(" + logicalTypeAnnotation + ")"); + } + } + + private static void convertErrorForTimestampNTZ(String parquetType) { + throw new RuntimeException( + "Unable to create Parquet converter for data type " + + TimestampNTZType$.MODULE$.json() + + " whose Parquet type is " + + parquetType); + } + + private static boolean canReadAsIntDecimal(ColumnDescriptor descriptor, DataType dt) { + if (!DecimalType.is32BitDecimalType(dt)) return false; + return isDecimalTypeMatched(descriptor, dt); + } + + private static boolean canReadAsLongDecimal(ColumnDescriptor descriptor, DataType dt) { + if (!DecimalType.is64BitDecimalType(dt)) return false; + return isDecimalTypeMatched(descriptor, dt); + } + + private static boolean canReadAsBinaryDecimal(ColumnDescriptor descriptor, DataType dt) { + if (!DecimalType.isByteArrayDecimalType(dt)) return false; + return isDecimalTypeMatched(descriptor, dt); + } + + private static boolean isLongDecimal(DataType dt) { + if (dt instanceof DecimalType) { + DecimalType d = (DecimalType) dt; + return d.precision() == 20 && d.scale() == 0; + } + return false; + } + + private static boolean isDecimalTypeMatched(ColumnDescriptor descriptor, DataType dt) { + DecimalType d = (DecimalType) dt; + LogicalTypeAnnotation typeAnnotation = descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + if (typeAnnotation instanceof DecimalLogicalTypeAnnotation) { + DecimalLogicalTypeAnnotation decimalType = (DecimalLogicalTypeAnnotation) typeAnnotation; + // It's OK if the required decimal precision is larger than or equal to the physical decimal + // precision in the Parquet metadata, as long as the decimal scale is the same. + return decimalType.getPrecision() <= d.precision() && decimalType.getScale() == d.scale(); + } + return false; + } + + private static boolean isTimestampTypeMatched( + LogicalTypeAnnotation logicalTypeAnnotation, LogicalTypeAnnotation.TimeUnit unit) { + return logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation + && ((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).getUnit() == unit; + } + + private static boolean isUnsignedIntTypeMatched( + LogicalTypeAnnotation logicalTypeAnnotation, int bitWidth) { + return logicalTypeAnnotation instanceof IntLogicalTypeAnnotation + && !((IntLogicalTypeAnnotation) logicalTypeAnnotation).isSigned() + && ((IntLogicalTypeAnnotation) logicalTypeAnnotation).getBitWidth() == bitWidth; + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/Utils.java b/common/src/main/java/org/apache/comet/parquet/Utils.java new file mode 100644 index 0000000000..95ca06cdaf --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.spark.sql.types.*; + +public class Utils { + public static ColumnReader getColumnReader( + DataType type, + ColumnDescriptor descriptor, + int batchSize, + boolean useDecimal128, + boolean useLazyMaterialization) { + // TODO: support `useLegacyDateTimestamp` for Iceberg + return getColumnReader( + type, descriptor, batchSize, useDecimal128, useLazyMaterialization, true); + } + + public static ColumnReader getColumnReader( + DataType type, + ColumnDescriptor descriptor, + int batchSize, + boolean useDecimal128, + boolean useLazyMaterialization, + boolean useLegacyDateTimestamp) { + if (useLazyMaterialization && supportLazyMaterialization(type)) { + return new LazyColumnReader( + type, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + } else { + return new ColumnReader(type, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + } + } + + private static boolean supportLazyMaterialization(DataType type) { + return (type instanceof StringType || type instanceof BinaryType); + } + + /** + * Initialize the Comet native Parquet reader. + * + * @param descriptor the Parquet column descriptor for the column to be read + * @param readType the Spark read type used for type promotion. Null if promotion is not enabled. + * @param batchSize the batch size, i.e., maximum number of elements per record batch + * @param useDecimal128 whether to always represent decimals using 128 bits. If false, the native + * reader may represent decimals using 32 or 64 bits, depending on the precision. + * @param useLegacyDateTimestampOrNTZ whether to read dates/timestamps that were written in the + * legacy hybrid Julian + Gregorian calendar as it is. If false, throw exceptions instead. If + * the spark type is TimestampNTZ, this should be true. + */ + public static long initColumnReader( + ColumnDescriptor descriptor, + DataType readType, + int batchSize, + boolean useDecimal128, + boolean useLegacyDateTimestampOrNTZ) { + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + int primitiveTypeId = getPhysicalTypeId(primitiveType.getPrimitiveTypeName()); + LogicalTypeAnnotation annotation = primitiveType.getLogicalTypeAnnotation(); + + // Process logical type information + + int bitWidth = -1; + boolean isSigned = false; + if (annotation instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { + LogicalTypeAnnotation.IntLogicalTypeAnnotation intAnnotation = + (LogicalTypeAnnotation.IntLogicalTypeAnnotation) annotation; + bitWidth = intAnnotation.getBitWidth(); + isSigned = intAnnotation.isSigned(); + } + + int precision, scale; + precision = scale = -1; + if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalAnnotation = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) annotation; + precision = decimalAnnotation.getPrecision(); + scale = decimalAnnotation.getScale(); + } + + int tu = -1; + boolean isAdjustedUtc = false; + if (annotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampAnnotation = + (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) annotation; + tu = getTimeUnitId(timestampAnnotation.getUnit()); + isAdjustedUtc = timestampAnnotation.isAdjustedToUTC(); + } + + TypePromotionInfo promotionInfo; + if (readType != null) { + promotionInfo = new TypePromotionInfo(readType); + } else { + // If type promotion is not enable, we'll just use the Parquet primitive type and precision. + promotionInfo = new TypePromotionInfo(primitiveTypeId, precision); + } + + return Native.initColumnReader( + primitiveTypeId, + getLogicalTypeId(annotation), + promotionInfo.physicalTypeId, + descriptor.getPath(), + descriptor.getMaxDefinitionLevel(), + descriptor.getMaxRepetitionLevel(), + bitWidth, + isSigned, + primitiveType.getTypeLength(), + precision, + promotionInfo.precision, + scale, + tu, + isAdjustedUtc, + batchSize, + useDecimal128, + useLegacyDateTimestampOrNTZ); + } + + static class TypePromotionInfo { + // The Parquet physical type ID converted from the Spark read schema, or the original Parquet + // physical type ID if type promotion is not enabled. + int physicalTypeId; + // Decimal precision from the Spark read schema, or -1 if it's not decimal type. + int precision; + + TypePromotionInfo(int physicalTypeId, int precision) { + this.physicalTypeId = physicalTypeId; + this.precision = precision; + } + + TypePromotionInfo(DataType sparkReadType) { + // Create a dummy `StructField` from the input Spark type. We don't care about + // field name, nullability and metadata. + StructField f = new StructField("f", sparkReadType, false, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(f); + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + int physicalTypeId = getPhysicalTypeId(primitiveType.getPrimitiveTypeName()); + LogicalTypeAnnotation annotation = primitiveType.getLogicalTypeAnnotation(); + int precision = -1; + if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalAnnotation = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) annotation; + precision = decimalAnnotation.getPrecision(); + } + this.physicalTypeId = physicalTypeId; + this.precision = precision; + } + } + + /** + * Maps the input Parquet physical type 'typeName' to an integer representing it. This is used for + * serialization between the Java and native side. + * + * @param typeName enum for the Parquet physical type + * @return an integer representing the input physical type + */ + static int getPhysicalTypeId(PrimitiveType.PrimitiveTypeName typeName) { + switch (typeName) { + case BOOLEAN: + return 0; + case INT32: + return 1; + case INT64: + return 2; + case INT96: + return 3; + case FLOAT: + return 4; + case DOUBLE: + return 5; + case BINARY: + return 6; + case FIXED_LEN_BYTE_ARRAY: + return 7; + } + throw new IllegalArgumentException("Invalid Parquet physical type: " + typeName); + } + + /** + * Maps the input Parquet logical type 'annotation' to an integer representing it. This is used + * for serialization between the Java and native side. + * + * @param annotation the Parquet logical type annotation + * @return an integer representing the input logical type + */ + static int getLogicalTypeId(LogicalTypeAnnotation annotation) { + if (annotation == null) { + return -1; // No logical type associated + } else if (annotation instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { + return 0; + } else if (annotation instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { + return 1; + } else if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + return 2; + } else if (annotation instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { + return 3; + } else if (annotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + return 4; + } else if (annotation instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { + return 5; + } else if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { + return 6; + } + + throw new UnsupportedOperationException("Unsupported Parquet logical type " + annotation); + } + + static int getTimeUnitId(LogicalTypeAnnotation.TimeUnit tu) { + switch (tu) { + case MILLIS: + return 0; + case MICROS: + return 1; + case NANOS: + return 2; + default: + throw new UnsupportedOperationException("Unsupported TimeUnit " + tu); + } + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java b/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java new file mode 100644 index 0000000000..fec266bf11 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import org.apache.arrow.vector.BaseVariableWidthVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.spark.sql.comet.util.Utils; +import org.apache.spark.unsafe.Platform; + +/** A Comet vector whose elements are already decoded (i.e., materialized). */ +public abstract class CometDecodedVector extends CometVector { + /** + * The vector that stores all the values. For dictionary-backed vector, this is the vector of + * indices. + */ + protected final ValueVector valueVector; + + private boolean hasNull; + private int numNulls; + private int numValues; + private int validityByteCacheIndex = -1; + private byte validityByteCache; + + protected CometDecodedVector(ValueVector vector, Field valueField, boolean useDecimal128) { + super(Utils.fromArrowField(valueField), useDecimal128); + this.valueVector = vector; + this.numNulls = valueVector.getNullCount(); + this.numValues = valueVector.getValueCount(); + this.hasNull = numNulls != 0; + } + + @Override + ValueVector getValueVector() { + return valueVector; + } + + @Override + public void setNumNulls(int numNulls) { + // We don't need to update null count in 'valueVector' since 'ValueVector.getNullCount' will + // re-compute the null count from validity buffer. + this.numNulls = numNulls; + this.hasNull = numNulls != 0; + this.validityByteCacheIndex = -1; + } + + @Override + public void setNumValues(int numValues) { + this.numValues = numValues; + if (valueVector instanceof BaseVariableWidthVector) { + BaseVariableWidthVector bv = (BaseVariableWidthVector) valueVector; + // In case `lastSet` is smaller than `numValues`, `setValueCount` will set all the offsets + // within `[lastSet + 1, numValues)` to be empty, which is incorrect in our case. + // + // For instance, this can happen if one first call `setNumValues` with input 100, and then + // again `setNumValues` with 200. The first call will set `lastSet` to 99, while the second + // call will set all strings between indices `[100, 200)` to be empty. + bv.setLastSet(numValues); + } + valueVector.setValueCount(numValues); + } + + public int numValues() { + return numValues; + } + + @Override + public boolean hasNull() { + return hasNull; + } + + @Override + public int numNulls() { + return numNulls; + } + + @Override + public boolean isNullAt(int rowId) { + if (!hasNull) return false; + + int byteIndex = rowId >> 3; + if (byteIndex != validityByteCacheIndex) { + long validityBufferAddress = valueVector.getValidityBuffer().memoryAddress(); + validityByteCache = Platform.getByte(null, validityBufferAddress + byteIndex); + validityByteCacheIndex = byteIndex; + } + return ((validityByteCache >> (rowId & 7)) & 1) == 0; + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java b/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java new file mode 100644 index 0000000000..165af5631c --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +/** A special Comet vector that just delegate all calls */ +public class CometDelegateVector extends CometVector { + protected CometVector delegate; + + public CometDelegateVector(DataType dataType) { + this(dataType, null, false); + } + + public CometDelegateVector(DataType dataType, boolean useDecimal128) { + this(dataType, null, useDecimal128); + } + + public CometDelegateVector(DataType dataType, CometVector delegate, boolean useDecimal128) { + super(dataType, useDecimal128); + if (delegate instanceof CometDelegateVector) { + throw new IllegalArgumentException("cannot have nested delegation"); + } + this.delegate = delegate; + } + + public void setDelegate(CometVector delegate) { + this.delegate = delegate; + } + + @Override + public void setNumNulls(int numNulls) { + delegate.setNumNulls(numNulls); + } + + @Override + public void setNumValues(int numValues) { + delegate.setNumValues(numValues); + } + + @Override + public int numValues() { + return delegate.numValues(); + } + + @Override + public boolean hasNull() { + return delegate.hasNull(); + } + + @Override + public int numNulls() { + return delegate.numNulls(); + } + + @Override + public boolean isNullAt(int rowId) { + return delegate.isNullAt(rowId); + } + + @Override + public boolean getBoolean(int rowId) { + return delegate.getBoolean(rowId); + } + + @Override + public byte getByte(int rowId) { + return delegate.getByte(rowId); + } + + @Override + public short getShort(int rowId) { + return delegate.getShort(rowId); + } + + @Override + public int getInt(int rowId) { + return delegate.getInt(rowId); + } + + @Override + public long getLong(int rowId) { + return delegate.getLong(rowId); + } + + @Override + public float getFloat(int rowId) { + return delegate.getFloat(rowId); + } + + @Override + public double getDouble(int rowId) { + return delegate.getDouble(rowId); + } + + @Override + public Decimal getDecimal(int i, int precision, int scale) { + return delegate.getDecimal(i, precision, scale); + } + + @Override + byte[] getBinaryDecimal(int i) { + return delegate.getBinaryDecimal(i); + } + + @Override + public UTF8String getUTF8String(int rowId) { + return delegate.getUTF8String(rowId); + } + + @Override + public byte[] getBinary(int rowId) { + return delegate.getBinary(rowId); + } + + @Override + public ColumnarArray getArray(int i) { + return delegate.getArray(i); + } + + @Override + public ColumnarMap getMap(int i) { + return delegate.getMap(i); + } + + @Override + public ColumnVector getChild(int i) { + return delegate.getChild(i); + } + + @Override + ValueVector getValueVector() { + return delegate.getValueVector(); + } + + @Override + public CometVector slice(int offset, int length) { + return delegate.slice(offset, length); + } + + @Override + DictionaryProvider getDictionaryProvider() { + return delegate.getDictionaryProvider(); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionary.java b/common/src/main/java/org/apache/comet/vector/CometDictionary.java new file mode 100644 index 0000000000..8c70465853 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometDictionary.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import org.apache.arrow.vector.ValueVector; +import org.apache.spark.unsafe.types.UTF8String; + +/** A dictionary which maps indices (integers) to values. */ +public class CometDictionary implements AutoCloseable { + private static final int DECIMAL_BYTE_WIDTH = 16; + + private final CometPlainVector values; + private final int numValues; + + /** Decoded dictionary values. Only one of the following is set. */ + private byte[] bytes; + + private short[] shorts; + private int[] ints; + private long[] longs; + private float[] floats; + private double[] doubles; + private boolean[] booleans; + private ByteArrayWrapper[] binaries; + private UTF8String[] strings; + + public CometDictionary(CometPlainVector values) { + this.values = values; + this.numValues = values.numValues(); + initialize(); + } + + public ValueVector getValueVector() { + return values.getValueVector(); + } + + public boolean decodeToBoolean(int index) { + return booleans[index]; + } + + public byte decodeToByte(int index) { + return bytes[index]; + } + + public short decodeToShort(int index) { + return shorts[index]; + } + + public int decodeToInt(int index) { + return ints[index]; + } + + public long decodeToLong(int index) { + return longs[index]; + } + + public float decodeToFloat(int index) { + return floats[index]; + } + + public double decodeToDouble(int index) { + return doubles[index]; + } + + public byte[] decodeToBinary(int index) { + return binaries[index].bytes; + } + + public UTF8String decodeToUTF8String(int index) { + return strings[index]; + } + + @Override + public void close() { + values.close(); + } + + private void initialize() { + switch (values.getValueVector().getMinorType()) { + case BIT: + booleans = new boolean[numValues]; + for (int i = 0; i < numValues; i++) { + booleans[i] = values.getBoolean(i); + } + break; + case TINYINT: + bytes = new byte[numValues]; + for (int i = 0; i < numValues; i++) { + bytes[i] = values.getByte(i); + } + break; + case SMALLINT: + shorts = new short[numValues]; + for (int i = 0; i < numValues; i++) { + shorts[i] = values.getShort(i); + } + break; + case INT: + case DATEDAY: + ints = new int[numValues]; + for (int i = 0; i < numValues; i++) { + ints[i] = values.getInt(i); + } + break; + case BIGINT: + case TIMESTAMPMICRO: + case TIMESTAMPMICROTZ: + longs = new long[numValues]; + for (int i = 0; i < numValues; i++) { + longs[i] = values.getLong(i); + } + break; + case FLOAT4: + floats = new float[numValues]; + for (int i = 0; i < numValues; i++) { + floats[i] = values.getFloat(i); + } + break; + case FLOAT8: + doubles = new double[numValues]; + for (int i = 0; i < numValues; i++) { + doubles[i] = values.getDouble(i); + } + break; + case VARBINARY: + case FIXEDSIZEBINARY: + binaries = new ByteArrayWrapper[numValues]; + for (int i = 0; i < numValues; i++) { + binaries[i] = new ByteArrayWrapper(values.getBinary(i)); + } + break; + case VARCHAR: + strings = new UTF8String[numValues]; + for (int i = 0; i < numValues; i++) { + strings[i] = values.getUTF8String(i); + } + break; + case DECIMAL: + binaries = new ByteArrayWrapper[numValues]; + for (int i = 0; i < numValues; i++) { + // Need copying here since we re-use byte array for decimal + byte[] bytes = values.getBinaryDecimal(i); + byte[] copy = new byte[DECIMAL_BYTE_WIDTH]; + System.arraycopy(bytes, 0, copy, 0, DECIMAL_BYTE_WIDTH); + binaries[i] = new ByteArrayWrapper(copy); + } + break; + default: + throw new IllegalArgumentException( + "Invalid Arrow minor type: " + values.getValueVector().getMinorType()); + } + } + + private static class ByteArrayWrapper { + private final byte[] bytes; + + ByteArrayWrapper(byte[] bytes) { + this.bytes = bytes; + } + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java b/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java new file mode 100644 index 0000000000..225fcfc438 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.util.TransferPair; +import org.apache.parquet.Preconditions; +import org.apache.spark.unsafe.types.UTF8String; + +/** A column vector whose elements are dictionary-encoded. */ +public class CometDictionaryVector extends CometDecodedVector { + public final CometPlainVector indices; + public final CometDictionary values; + public final DictionaryProvider provider; + + /** Whether this vector is an alias sliced from another vector. */ + private final boolean isAlias; + + public CometDictionaryVector( + CometPlainVector indices, + CometDictionary values, + DictionaryProvider provider, + boolean useDecimal128) { + this(indices, values, provider, useDecimal128, false); + } + + public CometDictionaryVector( + CometPlainVector indices, + CometDictionary values, + DictionaryProvider provider, + boolean useDecimal128, + boolean isAlias) { + super(indices.valueVector, values.getValueVector().getField(), useDecimal128); + Preconditions.checkArgument( + indices.valueVector instanceof IntVector, "'indices' should be a IntVector"); + this.values = values; + this.indices = indices; + this.provider = provider; + this.isAlias = isAlias; + } + + @Override + DictionaryProvider getDictionaryProvider() { + return this.provider; + } + + @Override + public void close() { + super.close(); + // Only close the values vector if this is not a sliced vector. + if (!isAlias) { + values.close(); + } + } + + @Override + public boolean getBoolean(int i) { + return values.decodeToBoolean(indices.getInt(i)); + } + + @Override + public byte getByte(int i) { + return values.decodeToByte(indices.getInt(i)); + } + + @Override + public short getShort(int i) { + return values.decodeToShort(indices.getInt(i)); + } + + @Override + public int getInt(int i) { + return values.decodeToInt(indices.getInt(i)); + } + + @Override + public long getLong(int i) { + return values.decodeToLong(indices.getInt(i)); + } + + @Override + public float getFloat(int i) { + return values.decodeToFloat(indices.getInt(i)); + } + + @Override + public double getDouble(int i) { + return values.decodeToDouble(indices.getInt(i)); + } + + @Override + public UTF8String getUTF8String(int i) { + return values.decodeToUTF8String(indices.getInt(i)); + } + + @Override + public byte[] getBinary(int i) { + return values.decodeToBinary(indices.getInt(i)); + } + + @Override + byte[] getBinaryDecimal(int i) { + return values.decodeToBinary(indices.getInt(i)); + } + + @Override + public CometVector slice(int offset, int length) { + TransferPair tp = indices.valueVector.getTransferPair(indices.valueVector.getAllocator()); + tp.splitAndTransfer(offset, length); + CometPlainVector sliced = new CometPlainVector(tp.getTo(), useDecimal128); + + // Set the alias flag to true so that the sliced vector will not close the dictionary vector. + // Otherwise, if the dictionary is closed, the sliced vector will not be able to access the + // dictionary. + return new CometDictionaryVector(sliced, values, provider, useDecimal128, true); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometLazyVector.java b/common/src/main/java/org/apache/comet/vector/CometLazyVector.java new file mode 100644 index 0000000000..17b8d7e712 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometLazyVector.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import org.apache.arrow.vector.ValueVector; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.parquet.LazyColumnReader; + +public class CometLazyVector extends CometDelegateVector { + private final LazyColumnReader columnReader; + + public CometLazyVector(DataType type, LazyColumnReader columnReader, boolean useDecimal128) { + super(type, useDecimal128); + this.columnReader = columnReader; + } + + public CometDecodedVector getDecodedVector() { + return (CometDecodedVector) delegate; + } + + @Override + public ValueVector getValueVector() { + columnReader.readAllBatch(); + setDelegate(columnReader.loadVector()); + return super.getValueVector(); + } + + @Override + public void setNumNulls(int numNulls) { + throw new UnsupportedOperationException("CometLazyVector doesn't support 'setNumNulls'"); + } + + @Override + public void setNumValues(int numValues) { + throw new UnsupportedOperationException("CometLazyVector doesn't support 'setNumValues'"); + } + + @Override + public void close() { + // Do nothing. 'vector' is closed by 'columnReader' which owns it. + } + + @Override + public boolean hasNull() { + columnReader.readAllBatch(); + setDelegate(columnReader.loadVector()); + return super.hasNull(); + } + + @Override + public int numNulls() { + columnReader.readAllBatch(); + setDelegate(columnReader.loadVector()); + return super.numNulls(); + } + + @Override + public boolean isNullAt(int rowId) { + if (columnReader.materializeUpToIfNecessary(rowId)) { + setDelegate(columnReader.loadVector()); + } + return super.isNullAt(rowId); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometListVector.java b/common/src/main/java/org/apache/comet/vector/CometListVector.java new file mode 100644 index 0000000000..1c8f3e658c --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometListVector.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import org.apache.arrow.vector.*; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.util.TransferPair; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; + +/** A Comet column vector for list type. */ +public class CometListVector extends CometDecodedVector { + final ListVector listVector; + final ValueVector dataVector; + final ColumnVector dataColumnVector; + + public CometListVector(ValueVector vector, boolean useDecimal128) { + super(vector, vector.getField(), useDecimal128); + + this.listVector = ((ListVector) vector); + this.dataVector = listVector.getDataVector(); + this.dataColumnVector = getVector(dataVector, useDecimal128); + } + + @Override + public ColumnarArray getArray(int i) { + int start = listVector.getOffsetBuffer().getInt(i * ListVector.OFFSET_WIDTH); + int end = listVector.getOffsetBuffer().getInt((i + 1) * ListVector.OFFSET_WIDTH); + + return new ColumnarArray(dataColumnVector, start, end - start); + } + + @Override + public CometVector slice(int offset, int length) { + TransferPair tp = this.valueVector.getTransferPair(this.valueVector.getAllocator()); + tp.splitAndTransfer(offset, length); + + return new CometListVector(tp.getTo(), useDecimal128); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java new file mode 100644 index 0000000000..a7373590ab --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometPlainVector.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.UUID; + +import org.apache.arrow.c.CDataDictionaryProvider; +import org.apache.arrow.vector.*; +import org.apache.arrow.vector.util.TransferPair; +import org.apache.parquet.Preconditions; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.types.UTF8String; + +/** A column vector whose elements are plainly decoded. */ +public class CometPlainVector extends CometDecodedVector { + private final long valueBufferAddress; + private final boolean isBaseFixedWidthVector; + + private byte booleanByteCache; + private int booleanByteCacheIndex = -1; + + public CometPlainVector(ValueVector vector, boolean useDecimal128) { + super(vector, vector.getField(), useDecimal128); + // NullType doesn't have data buffer. + if (vector instanceof NullVector) { + this.valueBufferAddress = -1; + } else { + this.valueBufferAddress = vector.getDataBuffer().memoryAddress(); + } + + isBaseFixedWidthVector = valueVector instanceof BaseFixedWidthVector; + } + + @Override + public void setNumNulls(int numNulls) { + super.setNumNulls(numNulls); + this.booleanByteCacheIndex = -1; + } + + @Override + public boolean getBoolean(int rowId) { + int byteIndex = rowId >> 3; + if (byteIndex != booleanByteCacheIndex) { + booleanByteCache = getByte(byteIndex); + booleanByteCacheIndex = byteIndex; + } + return ((booleanByteCache >> (rowId & 7)) & 1) == 1; + } + + @Override + public byte getByte(int rowId) { + return Platform.getByte(null, valueBufferAddress + rowId); + } + + @Override + public short getShort(int rowId) { + return Platform.getShort(null, valueBufferAddress + rowId * 2L); + } + + @Override + public int getInt(int rowId) { + return Platform.getInt(null, valueBufferAddress + rowId * 4L); + } + + @Override + public long getLong(int rowId) { + return Platform.getLong(null, valueBufferAddress + rowId * 8L); + } + + @Override + public float getFloat(int rowId) { + return Platform.getFloat(null, valueBufferAddress + rowId * 4L); + } + + @Override + public double getDouble(int rowId) { + return Platform.getDouble(null, valueBufferAddress + rowId * 8L); + } + + @Override + public UTF8String getUTF8String(int rowId) { + if (!isBaseFixedWidthVector) { + BaseVariableWidthVector varWidthVector = (BaseVariableWidthVector) valueVector; + long offsetBufferAddress = varWidthVector.getOffsetBuffer().memoryAddress(); + int offset = Platform.getInt(null, offsetBufferAddress + rowId * 4L); + int length = Platform.getInt(null, offsetBufferAddress + (rowId + 1L) * 4L) - offset; + return UTF8String.fromAddress(null, valueBufferAddress + offset, length); + } else { + // Iceberg maps UUID to StringType. + // The data type here must be UUID because the only FLBA -> String mapping we have is UUID. + BaseFixedWidthVector fixedWidthVector = (BaseFixedWidthVector) valueVector; + int length = fixedWidthVector.getTypeWidth(); + int offset = rowId * length; + byte[] result = new byte[length]; + Platform.copyMemory( + null, valueBufferAddress + offset, result, Platform.BYTE_ARRAY_OFFSET, length); + return UTF8String.fromString(convertToUuid(result).toString()); + } + } + + @Override + public byte[] getBinary(int rowId) { + int offset; + int length; + if (valueVector instanceof BaseVariableWidthVector) { + BaseVariableWidthVector varWidthVector = (BaseVariableWidthVector) valueVector; + long offsetBufferAddress = varWidthVector.getOffsetBuffer().memoryAddress(); + offset = Platform.getInt(null, offsetBufferAddress + rowId * 4L); + length = Platform.getInt(null, offsetBufferAddress + (rowId + 1L) * 4L) - offset; + } else if (valueVector instanceof BaseFixedWidthVector) { + BaseFixedWidthVector fixedWidthVector = (BaseFixedWidthVector) valueVector; + length = fixedWidthVector.getTypeWidth(); + offset = rowId * length; + } else { + throw new RuntimeException("Unsupported binary vector type: " + valueVector.getName()); + } + byte[] result = new byte[length]; + Platform.copyMemory( + null, valueBufferAddress + offset, result, Platform.BYTE_ARRAY_OFFSET, length); + return result; + } + + @Override + CDataDictionaryProvider getDictionaryProvider() { + return null; + } + + @Override + public boolean isNullAt(int rowId) { + if (this.valueBufferAddress == -1) { + return true; + } else { + return super.isNullAt(rowId); + } + } + + @Override + public CometVector slice(int offset, int length) { + TransferPair tp = this.valueVector.getTransferPair(this.valueVector.getAllocator()); + tp.splitAndTransfer(offset, length); + + return new CometPlainVector(tp.getTo(), useDecimal128); + } + + private static UUID convertToUuid(byte[] buf) { + Preconditions.checkArgument(buf.length == 16, "UUID require 16 bytes"); + ByteBuffer bb = ByteBuffer.wrap(buf); + bb.order(ByteOrder.BIG_ENDIAN); + long mostSigBits = bb.getLong(); + long leastSigBits = bb.getLong(); + return new UUID(mostSigBits, leastSigBits); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometStructVector.java b/common/src/main/java/org/apache/comet/vector/CometStructVector.java new file mode 100644 index 0000000000..52dcd43260 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometStructVector.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.arrow.vector.*; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.util.TransferPair; +import org.apache.spark.sql.vectorized.ColumnVector; + +/** A Comet column vector for struct type. */ +public class CometStructVector extends CometDecodedVector { + final List<ColumnVector> children; + + public CometStructVector(ValueVector vector, boolean useDecimal128) { + super(vector, vector.getField(), useDecimal128); + + StructVector structVector = ((StructVector) vector); + + int size = structVector.size(); + List<ColumnVector> children = new ArrayList<>(); + + for (int i = 0; i < size; ++i) { + ValueVector value = structVector.getVectorById(i); + children.add(getVector(value, useDecimal128)); + } + this.children = children; + } + + @Override + public ColumnVector getChild(int i) { + return children.get(i); + } + + @Override + public CometVector slice(int offset, int length) { + TransferPair tp = this.valueVector.getTransferPair(this.valueVector.getAllocator()); + tp.splitAndTransfer(offset, length); + + return new CometStructVector(tp.getTo(), useDecimal128); + } +} diff --git a/common/src/main/java/org/apache/comet/vector/CometVector.java b/common/src/main/java/org/apache/comet/vector/CometVector.java new file mode 100644 index 0000000000..744f243958 --- /dev/null +++ b/common/src/main/java/org/apache/comet/vector/CometVector.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector; + +import java.math.BigDecimal; +import java.math.BigInteger; + +import org.apache.arrow.vector.FixedWidthVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.dictionary.Dictionary; +import org.apache.arrow.vector.dictionary.DictionaryProvider; +import org.apache.arrow.vector.types.pojo.DictionaryEncoding; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.types.UTF8String; + +/** Base class for all Comet column vector implementations. */ +public abstract class CometVector extends ColumnVector { + private static final int DECIMAL_BYTE_WIDTH = 16; + private final byte[] DECIMAL_BYTES = new byte[DECIMAL_BYTE_WIDTH]; + protected final boolean useDecimal128; + + protected CometVector(DataType type, boolean useDecimal128) { + super(type); + this.useDecimal128 = useDecimal128; + } + + /** + * Sets the number of nulls in this vector to be 'numNulls'. This is used when the vector is + * reused across batches. + */ + public abstract void setNumNulls(int numNulls); + + /** + * Sets the number of values (including both nulls and non-nulls) in this vector to be + * 'numValues'. This is used when the vector is reused across batches. + */ + public abstract void setNumValues(int numValues); + + /** Returns the number of values in this vector. */ + public abstract int numValues(); + + /** Whether the elements of this vector are of fixed length. */ + public boolean isFixedLength() { + return getValueVector() instanceof FixedWidthVector; + } + + @Override + public Decimal getDecimal(int i, int precision, int scale) { + if (!useDecimal128 && precision <= Decimal.MAX_INT_DIGITS() && type instanceof IntegerType) { + return Decimal.createUnsafe(getInt(i), precision, scale); + } else if (!useDecimal128 && precision <= Decimal.MAX_LONG_DIGITS()) { + return Decimal.createUnsafe(getLong(i), precision, scale); + } else { + byte[] bytes = getBinaryDecimal(i); + BigInteger bigInteger = new BigInteger(bytes); + BigDecimal javaDecimal = new BigDecimal(bigInteger, scale); + try { + return Decimal.apply(javaDecimal, precision, scale); + } catch (ArithmeticException e) { + throw new ArithmeticException( + "Cannot convert " + + javaDecimal + + " (bytes: " + + bytes + + ", integer: " + + bigInteger + + ") to decimal with precision: " + + precision + + " and scale: " + + scale); + } + } + } + + /** Reads a 16-byte byte array which are encoded big-endian for decimal128. */ + byte[] getBinaryDecimal(int i) { + long valueBufferAddress = getValueVector().getDataBuffer().memoryAddress(); + Platform.copyMemory( + null, + valueBufferAddress + (long) i * DECIMAL_BYTE_WIDTH, + DECIMAL_BYTES, + Platform.BYTE_ARRAY_OFFSET, + DECIMAL_BYTE_WIDTH); + // Decimal is stored little-endian in Arrow, so we need to reverse the bytes here + for (int j = 0, k = DECIMAL_BYTE_WIDTH - 1; j < DECIMAL_BYTE_WIDTH / 2; j++, k--) { + byte tmp = DECIMAL_BYTES[j]; + DECIMAL_BYTES[j] = DECIMAL_BYTES[k]; + DECIMAL_BYTES[k] = tmp; + } + return DECIMAL_BYTES; + } + + @Override + public boolean getBoolean(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public long getLong(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public ColumnarArray getArray(int i) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public ColumnarMap getMap(int i) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public ColumnVector getChild(int i) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @Override + public void close() { + getValueVector().close(); + } + + DictionaryProvider getDictionaryProvider() { + throw new UnsupportedOperationException("Not implemented"); + } + + abstract ValueVector getValueVector(); + + /** + * Returns a zero-copying new vector that contains the values from [offset, offset + length). + * + * @param offset the offset of the new vector + * @param length the length of the new vector + * @return the new vector + */ + public abstract CometVector slice(int offset, int length); + + /** + * Returns a corresponding `CometVector` implementation based on the given Arrow `ValueVector`. + * + * @param vector Arrow `ValueVector` + * @param useDecimal128 Whether to use Decimal128 for decimal column + * @return `CometVector` implementation + */ + protected static CometVector getVector( + ValueVector vector, boolean useDecimal128, DictionaryProvider dictionaryProvider) { + if (vector instanceof StructVector) { + return new CometStructVector(vector, useDecimal128); + } else if (vector instanceof ListVector) { + return new CometListVector(vector, useDecimal128); + } else { + DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); + CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); + + if (dictionaryEncoding == null) { + return cometVector; + } else { + Dictionary dictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); + CometPlainVector dictionaryVector = + new CometPlainVector(dictionary.getVector(), useDecimal128); + CometDictionary cometDictionary = new CometDictionary(dictionaryVector); + + return new CometDictionaryVector( + cometVector, cometDictionary, dictionaryProvider, useDecimal128); + } + } + } + + protected static CometVector getVector(ValueVector vector, boolean useDecimal128) { + return getVector(vector, useDecimal128, null); + } +} diff --git a/common/src/main/resources/log4j2.properties b/common/src/main/resources/log4j2.properties new file mode 100644 index 0000000000..04cdf85330 --- /dev/null +++ b/common/src/main/resources/log4j2.properties @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn + diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala new file mode 100644 index 0000000000..7f83d92962 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -0,0 +1,454 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.sql.comet.util.Utils +import org.apache.spark.sql.internal.SQLConf + +/** + * Configurations for a Comet application. Mostly inspired by [[SQLConf]] in Spark. + * + * To get the value of a Comet config key from a [[SQLConf]], you can do the following: + * + * {{{ + * CometConf.COMET_ENABLED.get + * }}} + * + * which retrieves the config value from the thread-local [[SQLConf]] object. Alternatively, you + * can also explicitly pass a [[SQLConf]] object to the `get` method. + */ +object CometConf { + def conf(key: String): ConfigBuilder = ConfigBuilder(key) + + val COMET_EXEC_CONFIG_PREFIX = "spark.comet.exec"; + + val COMET_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.enabled") + .doc( + "Whether to enable Comet extension for Spark. When this is turned on, Spark will use " + + "Comet to read Parquet data source. Note that to enable native vectorized execution, " + + "both this config and 'spark.comet.exec.enabled' need to be enabled. By default, this " + + "config is the value of the env var `ENABLE_COMET` if set, or true otherwise.") + .booleanConf + .createWithDefault(sys.env.getOrElse("ENABLE_COMET", "true").toBoolean) + + val COMET_SCAN_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.enabled") + .doc( + "Whether to enable Comet scan. When this is turned on, Spark will use Comet to read " + + "Parquet data source. Note that to enable native vectorized execution, both this " + + "config and 'spark.comet.exec.enabled' need to be enabled. By default, this config " + + "is true.") + .booleanConf + .createWithDefault(true) + + val COMET_EXEC_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.enabled") + .doc( + "Whether to enable Comet native vectorized execution for Spark. This controls whether " + + "Spark should convert operators into their Comet counterparts and execute them in " + + "native space. Note: each operator is associated with a separate config in the " + + "format of 'spark.comet.exec.<operator_name>.enabled' at the moment, and both the " + + "config and this need to be turned on, in order for the operator to be executed in " + + "native. By default, this config is false.") + .booleanConf + .createWithDefault(false) + + val COMET_MEMORY_OVERHEAD: OptionalConfigEntry[Long] = conf("spark.comet.memoryOverhead") + .doc( + "The amount of additional memory to be allocated per executor process for Comet, in MiB. " + + "This config is optional. If this is not specified, it will be set to " + + "`spark.comet.memory.overhead.factor` * `spark.executor.memory`. " + + "This is memory that accounts for things like Comet native execution, etc.") + .bytesConf(ByteUnit.MiB) + .createOptional + + val COMET_MEMORY_OVERHEAD_FACTOR: ConfigEntry[Double] = conf( + "spark.comet.memory.overhead.factor") + .doc( + "Fraction of executor memory to be allocated as additional non-heap memory per executor " + + "process for Comet. Default value is 0.2.") + .doubleConf + .checkValue( + factor => factor > 0, + "Ensure that Comet memory overhead factor is a double greater than 0") + .createWithDefault(0.2) + + val COMET_MEMORY_OVERHEAD_MIN_MIB: ConfigEntry[Long] = conf("spark.comet.memory.overhead.min") + .doc("Minimum amount of additional memory to be allocated per executor process for Comet, " + + "in MiB.") + .bytesConf(ByteUnit.MiB) + .checkValue( + _ >= 0, + "Ensure that Comet memory overhead min is a long greater than or equal to 0") + .createWithDefault(384) + + val COMET_EXEC_ALL_OPERATOR_ENABLED: ConfigEntry[Boolean] = conf( + s"$COMET_EXEC_CONFIG_PREFIX.all.enabled") + .doc( + "Whether to enable all Comet operators. By default, this config is false. Note that " + + "this config precedes all separate config 'spark.comet.exec.<operator_name>.enabled'. " + + "That being said, if this config is enabled, separate configs are ignored.") + .booleanConf + .createWithDefault(false) + + val COMET_EXEC_ALL_EXPR_ENABLED: ConfigEntry[Boolean] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.all.expr.enabled") + .doc( + "Whether to enable all Comet exprs. By default, this config is false. Note that " + + "this config precedes all separate config 'spark.comet.exec.<expr_name>.enabled'. " + + "That being said, if this config is enabled, separate configs are ignored.") + .booleanConf + .createWithDefault(false) + + val COMET_DEBUG_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.debug.enabled") + .doc( + "Whether to enable debug mode for Comet. By default, this config is false. " + + "When enabled, Comet will do additional checks for debugging purpose. For example, " + + "validating array when importing arrays from JVM at native side. Note that these " + + "checks may be expensive in performance and should only be enabled for debugging " + + "purpose.") + .booleanConf + .createWithDefault(false) + + val COMET_BATCH_SIZE: ConfigEntry[Int] = conf("spark.comet.batchSize") + .doc("The columnar batch size, i.e., the maximum number of rows that a batch can contain.") + .intConf + .createWithDefault(8192) + + val COMET_EXEC_MEMORY_FRACTION: ConfigEntry[Double] = conf("spark.comet.exec.memoryFraction") + .doc( + "The fraction of memory from Comet memory overhead that the native memory " + + "manager can use for execution. The purpose of this config is to set aside memory for " + + "untracked data structures, as well as imprecise size estimation during memory " + + "acquisition. Default value is 0.7.") + .doubleConf + .createWithDefault(0.7) + + val COMET_PARQUET_ENABLE_DIRECT_BUFFER: ConfigEntry[Boolean] = conf( + "spark.comet.parquet.enable.directBuffer") + .doc("Whether to use Java direct byte buffer when reading Parquet. By default, this is false") + .booleanConf + .createWithDefault(false) + + val COMET_SCAN_PREFETCH_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.scan.preFetch.enabled") + .doc("Whether to enable pre-fetching feature of CometScan. By default is disabled.") + .booleanConf + .createWithDefault(false) + + val COMET_SCAN_PREFETCH_THREAD_NUM: ConfigEntry[Int] = + conf("spark.comet.scan.preFetch.threadNum") + .doc( + "The number of threads running pre-fetching for CometScan. Effective if " + + s"${COMET_SCAN_PREFETCH_ENABLED.key} is enabled. By default it is 2. Note that more " + + "pre-fetching threads means more memory requirement to store pre-fetched row groups.") + .intConf + .createWithDefault(2) + + val COMET_NATIVE_LOAD_REQUIRED: ConfigEntry[Boolean] = conf("spark.comet.nativeLoadRequired") + .doc( + "Whether to require Comet native library to load successfully when Comet is enabled. " + + "If not, Comet will silently fallback to Spark when it fails to load the native lib. " + + "Otherwise, an error will be thrown and the Spark job will be aborted.") + .booleanConf + .createWithDefault(false) + + val COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP: ConfigEntry[Boolean] = + conf("spark.comet.exceptionOnDatetimeRebase") + .doc("Whether to throw exception when seeing dates/timestamps from the legacy hybrid " + + "(Julian + Gregorian) calendar. Since Spark 3, dates/timestamps were written according " + + "to the Proleptic Gregorian calendar. When this is true, Comet will " + + "throw exceptions when seeing these dates/timestamps that were written by Spark version " + + "before 3.0. If this is false, these dates/timestamps will be read as if they were " + + "written to the Proleptic Gregorian calendar and will not be rebased.") + .booleanConf + .createWithDefault(false) + + val COMET_USE_DECIMAL_128: ConfigEntry[Boolean] = conf("spark.comet.use.decimal128") + .internal() + .doc("If true, Comet will always use 128 bits to represent a decimal value, regardless of " + + "its precision. If false, Comet will use 32, 64 and 128 bits respectively depending on " + + "the precision. N.B. this is NOT a user-facing config but should be inferred and set by " + + "Comet itself.") + .booleanConf + .createWithDefault(false) + + val COMET_USE_LAZY_MATERIALIZATION: ConfigEntry[Boolean] = conf( + "spark.comet.use.lazyMaterialization") + .internal() + .doc( + "Whether to enable lazy materialization for Comet. When this is turned on, Comet will " + + "read Parquet data source lazily for string and binary columns. For filter operations, " + + "lazy materialization will improve read performance by skipping unused pages.") + .booleanConf + .createWithDefault(true) + + val COMET_SCHEMA_EVOLUTION_ENABLED: ConfigEntry[Boolean] = conf( + "spark.comet.schemaEvolution.enabled") + .internal() + .doc( + "Whether to enable schema evolution in Comet. For instance, promoting a integer " + + "column to a long column, a float column to a double column, etc. This is automatically" + + "enabled when reading from Iceberg tables.") + .booleanConf + .createWithDefault(false) +} + +object ConfigHelpers { + def toNumber[T](s: String, converter: String => T, key: String, configType: String): T = { + try { + converter(s.trim) + } catch { + case _: NumberFormatException => + throw new IllegalArgumentException(s"$key should be $configType, but was $s") + } + } + + def toBoolean(s: String, key: String): Boolean = { + try { + s.trim.toBoolean + } catch { + case _: IllegalArgumentException => + throw new IllegalArgumentException(s"$key should be boolean, but was $s") + } + } + + def stringToSeq[T](str: String, converter: String => T): Seq[T] = { + Utils.stringToSeq(str).map(converter) + } + + def seqToString[T](v: Seq[T], stringConverter: T => String): String = { + v.map(stringConverter).mkString(",") + } + + def timeFromString(str: String, unit: TimeUnit): Long = JavaUtils.timeStringAs(str, unit) + + def timeToString(v: Long, unit: TimeUnit): String = + TimeUnit.MILLISECONDS.convert(v, unit) + "ms" + + def byteFromString(str: String, unit: ByteUnit): Long = { + val (input, multiplier) = + if (str.nonEmpty && str.charAt(0) == '-') { + (str.substring(1), -1) + } else { + (str, 1) + } + multiplier * JavaUtils.byteStringAs(input, unit) + } + + def byteToString(v: Long, unit: ByteUnit): String = unit.convertTo(v, ByteUnit.BYTE) + "b" +} + +private class TypedConfigBuilder[T]( + val parent: ConfigBuilder, + val converter: String => T, + val stringConverter: T => String) { + + import ConfigHelpers._ + + def this(parent: ConfigBuilder, converter: String => T) = { + this(parent, converter, Option(_).map(_.toString).orNull) + } + + /** Apply a transformation to the user-provided values of the config entry. */ + def transform(fn: T => T): TypedConfigBuilder[T] = { + new TypedConfigBuilder(parent, s => fn(converter(s)), stringConverter) + } + + /** Checks if the user-provided value for the config matches the validator. */ + def checkValue(validator: T => Boolean, errorMsg: String): TypedConfigBuilder[T] = { + transform { v => + if (!validator(v)) { + throw new IllegalArgumentException(s"'$v' in ${parent.key} is invalid. $errorMsg") + } + v + } + } + + /** Check that user-provided values for the config match a pre-defined set. */ + def checkValues(validValues: Set[T]): TypedConfigBuilder[T] = { + transform { v => + if (!validValues.contains(v)) { + throw new IllegalArgumentException( + s"The value of ${parent.key} should be one of ${validValues.mkString(", ")}, but was $v") + } + v + } + } + + /** Turns the config entry into a sequence of values of the underlying type. */ + def toSequence: TypedConfigBuilder[Seq[T]] = { + new TypedConfigBuilder(parent, stringToSeq(_, converter), seqToString(_, stringConverter)) + } + + /** Creates a [[ConfigEntry]] that does not have a default value. */ + def createOptional: OptionalConfigEntry[T] = { + new OptionalConfigEntry[T]( + parent.key, + converter, + stringConverter, + parent._doc, + parent._public, + parent._version) + } + + /** Creates a [[ConfigEntry]] that has a default value. */ + def createWithDefault(default: T): ConfigEntry[T] = { + val transformedDefault = converter(stringConverter(default)) + new ConfigEntryWithDefault[T]( + parent.key, + transformedDefault, + converter, + stringConverter, + parent._doc, + parent._public, + parent._version) + } +} + +private[comet] abstract class ConfigEntry[T]( + val key: String, + val valueConverter: String => T, + val stringConverter: T => String, + val doc: String, + val isPublic: Boolean, + val version: String) { + + /** + * Retrieves the config value from the given [[SQLConf]]. + */ + def get(conf: SQLConf): T + + /** + * Retrieves the config value from the current thread-local [[SQLConf]] + * @return + */ + def get(): T = get(SQLConf.get) + + def defaultValue: Option[T] = None + def defaultValueString: String + + override def toString: String = { + s"ConfigEntry(key=$key, defaultValue=$defaultValueString, doc=$doc, " + + s"public=$isPublic, version=$version)" + } +} + +private[comet] class ConfigEntryWithDefault[T]( + key: String, + _defaultValue: T, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean, + version: String) + extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic, version) { + override def defaultValue: Option[T] = Some(_defaultValue) + override def defaultValueString: String = stringConverter(_defaultValue) + + def get(conf: SQLConf): T = { + val tmp = conf.getConfString(key, null) + if (tmp == null) { + _defaultValue + } else { + valueConverter(tmp) + } + } +} + +private[comet] class OptionalConfigEntry[T]( + key: String, + val rawValueConverter: String => T, + val rawStringConverter: T => String, + doc: String, + isPublic: Boolean, + version: String) + extends ConfigEntry[Option[T]]( + key, + s => Some(rawValueConverter(s)), + v => v.map(rawStringConverter).orNull, + doc, + isPublic, + version) { + + override def defaultValueString: String = ConfigEntry.UNDEFINED + + override def get(conf: SQLConf): Option[T] = { + Option(conf.getConfString(key, null)).map(rawValueConverter) + } +} + +private[comet] case class ConfigBuilder(key: String) { + import ConfigHelpers._ + + var _public = true + var _doc = "" + var _version = "" + + def internal(): ConfigBuilder = { + _public = false + this + } + + def doc(s: String): ConfigBuilder = { + _doc = s + this + } + + def version(v: String): ConfigBuilder = { + _version = v + this + } + + def intConf: TypedConfigBuilder[Int] = { + new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int")) + } + + def longConf: TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, toNumber(_, _.toLong, key, "long")) + } + + def doubleConf: TypedConfigBuilder[Double] = { + new TypedConfigBuilder(this, toNumber(_, _.toDouble, key, "double")) + } + + def booleanConf: TypedConfigBuilder[Boolean] = { + new TypedConfigBuilder(this, toBoolean(_, key)) + } + + def stringConf: TypedConfigBuilder[String] = { + new TypedConfigBuilder(this, v => v) + } + + def timeConf(unit: TimeUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, timeFromString(_, unit), timeToString(_, unit)) + } + + def bytesConf(unit: ByteUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, byteFromString(_, unit), byteToString(_, unit)) + } +} + +private object ConfigEntry { + val UNDEFINED = "<undefined>" +} diff --git a/common/src/main/scala/org/apache/comet/Constants.scala b/common/src/main/scala/org/apache/comet/Constants.scala new file mode 100644 index 0000000000..83b570fc3a --- /dev/null +++ b/common/src/main/scala/org/apache/comet/Constants.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +object Constants { + val LOG_CONF_PATH = "comet.log.file.path" + val LOG_CONF_NAME = "log4rs.yaml" +} diff --git a/common/src/main/scala/org/apache/comet/package.scala b/common/src/main/scala/org/apache/comet/package.scala new file mode 100644 index 0000000000..c9aca75382 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/package.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import java.util.Properties + +package object comet { + + /** + * Provides access to build information about the Comet libraries. This will be used by the + * benchmarking software to provide the source revision and repository. In addition, the build + * information is included to aid in future debugging efforts for releases. + */ + private object CometBuildInfo { + + val ( + cometVersion: String, + cometBranch: String, + cometRevision: String, + cometBuildUserName: String, + cometBuildUserEmail: String, + cometRepoUrl: String, + cometBuildTimestamp: String) = { + val resourceStream = Thread + .currentThread() + .getContextClassLoader + .getResourceAsStream("comet-git-info.properties") + if (resourceStream == null) { + throw new CometRuntimeException("Could not find comet-git-info.properties") + } + + try { + val unknownProp = "<unknown>" + val props = new Properties() + props.load(resourceStream) + ( + props.getProperty("git.build.version", unknownProp), + props.getProperty("git.branch", unknownProp), + props.getProperty("git.commit.id.full", unknownProp), + props.getProperty("git.build.user.name", unknownProp), + props.getProperty("git.build.user.email", unknownProp), + props.getProperty("git.remote.origin.url", unknownProp), + props.getProperty("git.build.time", unknownProp)) + } catch { + case e: Exception => + throw new CometRuntimeException( + "Error loading properties from comet-git-info.properties", + e) + } finally { + if (resourceStream != null) { + try { + resourceStream.close() + } catch { + case e: Exception => + throw new CometRuntimeException("Error closing Comet build info resource stream", e) + } + } + } + } + } + + val COMET_VERSION = CometBuildInfo.cometVersion + val COMET_BRANCH = CometBuildInfo.cometBranch + val COMET_REVISION = CometBuildInfo.cometRevision + val COMET_BUILD_USER_EMAIL = CometBuildInfo.cometBuildUserEmail + val COMET_BUILD_USER_NAME = CometBuildInfo.cometBuildUserName + val COMET_REPO_URL = CometBuildInfo.cometRepoUrl + val COMET_BUILD_TIMESTAMP = CometBuildInfo.cometBuildTimestamp + +} diff --git a/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala b/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala new file mode 100644 index 0000000000..d851067b54 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +object CometParquetUtils { + private val PARQUET_FIELD_ID_WRITE_ENABLED = "spark.sql.parquet.fieldId.write.enabled" + private val PARQUET_FIELD_ID_READ_ENABLED = "spark.sql.parquet.fieldId.read.enabled" + private val IGNORE_MISSING_PARQUET_FIELD_ID = "spark.sql.parquet.fieldId.read.ignoreMissing" + + def writeFieldId(conf: SQLConf): Boolean = + conf.getConfString(PARQUET_FIELD_ID_WRITE_ENABLED, "false").toBoolean + + def writeFieldId(conf: Configuration): Boolean = + conf.getBoolean(PARQUET_FIELD_ID_WRITE_ENABLED, false) + + def readFieldId(conf: SQLConf): Boolean = + conf.getConfString(PARQUET_FIELD_ID_READ_ENABLED, "false").toBoolean + + def ignoreMissingIds(conf: SQLConf): Boolean = + conf.getConfString(IGNORE_MISSING_PARQUET_FIELD_ID, "false").toBoolean + + // The following is copied from QueryExecutionErrors + // TODO: remove after dropping Spark 3.2.0 support and directly use + // QueryExecutionErrors.foundDuplicateFieldInFieldIdLookupModeError + def foundDuplicateFieldInFieldIdLookupModeError( + requiredId: Int, + matchedFields: String): Throwable = { + new RuntimeException(s""" + |Found duplicate field(s) "$requiredId": $matchedFields + |in id mapping mode + """.stripMargin.replaceAll("\n", " ")) + } + + // The followings are copied from org.apache.spark.sql.execution.datasources.parquet.ParquetUtils + // TODO: remove after dropping Spark 3.2.0 support and directly use ParquetUtils + /** + * A StructField metadata key used to set the field id of a column in the Parquet schema. + */ + val FIELD_ID_METADATA_KEY = "parquet.field.id" + + /** + * Whether there exists a field in the schema, whether inner or leaf, has the parquet field ID + * metadata. + */ + def hasFieldIds(schema: StructType): Boolean = { + def recursiveCheck(schema: DataType): Boolean = { + schema match { + case st: StructType => + st.exists(field => hasFieldId(field) || recursiveCheck(field.dataType)) + + case at: ArrayType => recursiveCheck(at.elementType) + + case mt: MapType => recursiveCheck(mt.keyType) || recursiveCheck(mt.valueType) + + case _ => + // No need to really check primitive types, just to terminate the recursion + false + } + } + if (schema.isEmpty) false else recursiveCheck(schema) + } + + def hasFieldId(field: StructField): Boolean = + field.metadata.contains(FIELD_ID_METADATA_KEY) + + def getFieldId(field: StructField): Int = { + require( + hasFieldId(field), + s"The key `$FIELD_ID_METADATA_KEY` doesn't exist in the metadata of " + field) + try { + Math.toIntExact(field.metadata.getLong(FIELD_ID_METADATA_KEY)) + } catch { + case _: ArithmeticException | _: ClassCastException => + throw new IllegalArgumentException( + s"The key `$FIELD_ID_METADATA_KEY` must be a 32-bit integer") + } + } +} diff --git a/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala b/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala new file mode 100644 index 0000000000..ca13bba0c4 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import java.util.concurrent.{Executors, ExecutorService, ThreadFactory} +import java.util.concurrent.atomic.AtomicLong + +abstract class CometReaderThreadPool { + private var threadPool: Option[ExecutorService] = None + + protected def threadNamePrefix: String + + private def initThreadPool(maxThreads: Int): ExecutorService = synchronized { + if (threadPool.isEmpty) { + val threadFactory: ThreadFactory = new ThreadFactory() { + private val defaultThreadFactory = Executors.defaultThreadFactory + val count = new AtomicLong(0) + + override def newThread(r: Runnable): Thread = { + val thread = defaultThreadFactory.newThread(r) + thread.setName(s"${threadNamePrefix}_${count.getAndIncrement()}") + thread.setDaemon(true) + thread + } + } + + val threadPoolExecutor = Executors.newFixedThreadPool(maxThreads, threadFactory) + threadPool = Some(threadPoolExecutor) + } + + threadPool.get + } + + def getOrCreateThreadPool(numThreads: Int): ExecutorService = { + threadPool.getOrElse(initThreadPool(numThreads)) + } + +} + +// A thread pool used for pre-fetching files. +object CometPrefetchThreadPool extends CometReaderThreadPool { + override def threadNamePrefix: String = "prefetch_thread" +} + +// Thread pool used by the Parquet parallel reader +object CometFileReaderThreadPool extends CometReaderThreadPool { + override def threadNamePrefix: String = "file_reader_thread" +} diff --git a/common/src/main/scala/org/apache/comet/shims/ShimBatchReader.scala b/common/src/main/scala/org/apache/comet/shims/ShimBatchReader.scala new file mode 100644 index 0000000000..ece4cfbe5e --- /dev/null +++ b/common/src/main/scala/org/apache/comet/shims/ShimBatchReader.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.shims + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile + +object ShimBatchReader { + + // TODO: remove after dropping Spark 3.2 & 3.3 support and directly call PartitionedFile + def newPartitionedFile(partitionValues: InternalRow, file: String): PartitionedFile = + classOf[PartitionedFile].getDeclaredConstructors + .map(c => + c.getParameterCount match { + case 5 => + c.newInstance( + partitionValues, + file, + Long.box(-1), // -1 means we read the entire file + Long.box(-1), + Array.empty[String]) + case 7 => + c.newInstance( + partitionValues, + c.getParameterTypes()(1) + .getConstructor(classOf[String]) + .newInstance(file) + .asInstanceOf[AnyRef], + Long.box(-1), // -1 means we read the entire file + Long.box(-1), + Array.empty[String], + Long.box(0), + Long.box(0)) + }) + .head + .asInstanceOf[PartitionedFile] +} diff --git a/common/src/main/scala/org/apache/comet/shims/ShimFileFormat.scala b/common/src/main/scala/org/apache/comet/shims/ShimFileFormat.scala new file mode 100644 index 0000000000..5ab7eaf4fa --- /dev/null +++ b/common/src/main/scala/org/apache/comet/shims/ShimFileFormat.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.shims + +object ShimFileFormat { + + // TODO: remove after dropping Spark 3.2 & 3.3 support and directly use FileFormat.ROW_INDEX + val ROW_INDEX = "row_index" + + // A name for a temporary column that holds row indexes computed by the file format reader + // until they can be placed in the _metadata struct. + // TODO: remove after dropping Spark 3.2 & 3.3 support and directly use + // FileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME + val ROW_INDEX_TEMPORARY_COLUMN_NAME: String = s"_tmp_metadata_$ROW_INDEX" + + // TODO: remove after dropping Spark 3.2 support and use FileFormat.OPTION_RETURNING_BATCH + val OPTION_RETURNING_BATCH = "returning_batch" +} diff --git a/common/src/main/scala/org/apache/comet/shims/ShimResolveDefaultColumns.scala b/common/src/main/scala/org/apache/comet/shims/ShimResolveDefaultColumns.scala new file mode 100644 index 0000000000..8a30c8e00b --- /dev/null +++ b/common/src/main/scala/org/apache/comet/shims/ShimResolveDefaultColumns.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.shims + +import scala.util.Try + +import org.apache.spark.sql.types.{StructField, StructType} + +object ShimResolveDefaultColumns { + // TODO: remove after dropping Spark 3.2 & 3.3 support and directly use ResolveDefaultColumns + def getExistenceDefaultValue(field: StructField): Any = + Try { + // scalastyle:off classforname + Class.forName("org.apache.spark.sql.catalyst.util.ResolveDefaultColumns$") + // scalastyle:on classforname + }.map { objClass => + val objInstance = objClass.getField("MODULE$").get(null) + val method = objClass.getMethod("getExistenceDefaultValues", classOf[StructType]) + method.invoke(objInstance, StructType(Seq(field))).asInstanceOf[Array[Any]].head + }.getOrElse(null) +} diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala new file mode 100644 index 0000000000..1e27ed8f05 --- /dev/null +++ b/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector + +import scala.collection.mutable + +import org.apache.arrow.c.{ArrowArray, ArrowImporter, ArrowSchema, CDataDictionaryProvider, Data} +import org.apache.arrow.memory.RootAllocator +import org.apache.arrow.vector._ +import org.apache.spark.SparkException +import org.apache.spark.sql.vectorized.ColumnarBatch + +class NativeUtil { + private val allocator = new RootAllocator(Long.MaxValue) + private val dictionaryProvider: CDataDictionaryProvider = new CDataDictionaryProvider + private val importer = new ArrowImporter(allocator) + + /** + * Exports a Comet `ColumnarBatch` into a list of memory addresses that can be consumed by the + * native execution. + * + * @param batch + * the input Comet columnar batch + * @return + * a list containing pairs of memory addresses in the format of (address of Arrow array, + * address of Arrow schema) + */ + def exportBatch(batch: ColumnarBatch): Array[Long] = { + val vectors = (0 until batch.numCols()).flatMap { index => + batch.column(index) match { + case a: CometVector => + val valueVector = a.getValueVector + + val provider = if (valueVector.getField.getDictionary != null) { + a.getDictionaryProvider + } else { + null + } + + val arrowSchema = ArrowSchema.allocateNew(allocator) + val arrowArray = ArrowArray.allocateNew(allocator) + Data.exportVector( + allocator, + getFieldVector(valueVector), + provider, + arrowArray, + arrowSchema) + + Seq((arrowArray, arrowSchema)) + case c => + throw new SparkException( + "Comet execution only takes Arrow Arrays, but got " + + s"${c.getClass}") + } + } + + vectors.flatMap { pair => + Seq(pair._1.memoryAddress(), pair._2.memoryAddress()) + }.toArray + } + + /** + * Imports a list of Arrow addresses from native execution, and return a list of Comet vectors. + * + * @param arrayAddress + * a list containing paris of Arrow addresses from the native, in the format of (address of + * Arrow array, address of Arrow schema) + * @return + * a list of Comet vectors + */ + def importVector(arrayAddress: Array[Long]): Seq[CometVector] = { + val arrayVectors = mutable.ArrayBuffer.empty[CometVector] + + for (i <- arrayAddress.indices by 2) { + val arrowSchema = ArrowSchema.wrap(arrayAddress(i + 1)) + val arrowArray = ArrowArray.wrap(arrayAddress(i)) + + // Native execution should always have 'useDecimal128' set to true since it doesn't support + // other cases. + arrayVectors += CometVector.getVector( + importer.importVector(arrowArray, arrowSchema, dictionaryProvider), + true, + dictionaryProvider) + + arrowArray.close() + arrowSchema.close() + } + arrayVectors.toSeq + } + + /** + * Takes zero-copy slices of the input batch with given start index and maximum number of rows. + * + * @param batch + * Input batch + * @param startIndex + * Start index of the slice + * @param maxNumRows + * Maximum number of rows in the slice + * @return + * A new batch with the sliced vectors + */ + def takeRows(batch: ColumnarBatch, startIndex: Int, maxNumRows: Int): ColumnarBatch = { + val arrayVectors = mutable.ArrayBuffer.empty[CometVector] + + for (i <- 0 until batch.numCols()) { + val column = batch.column(i).asInstanceOf[CometVector] + arrayVectors += column.slice(startIndex, maxNumRows) + } + + new ColumnarBatch(arrayVectors.toArray, maxNumRows) + } + + private def getFieldVector(valueVector: ValueVector): FieldVector = { + valueVector match { + case v @ (_: BitVector | _: TinyIntVector | _: SmallIntVector | _: IntVector | + _: BigIntVector | _: Float4Vector | _: Float8Vector | _: VarCharVector | + _: DecimalVector | _: DateDayVector | _: TimeStampMicroTZVector | _: VarBinaryVector | + _: FixedSizeBinaryVector | _: TimeStampMicroVector) => + v.asInstanceOf[FieldVector] + case _ => throw new SparkException(s"Unsupported Arrow Vector: ${valueVector.getClass}") + } + } +} diff --git a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala new file mode 100644 index 0000000000..9c4f99602f --- /dev/null +++ b/common/src/main/scala/org/apache/comet/vector/StreamReader.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.vector + +import java.nio.channels.ReadableByteChannel + +import scala.collection.JavaConverters.collectionAsScalaIterableConverter + +import org.apache.arrow.memory.RootAllocator +import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.vector.ipc.{ArrowStreamReader, ReadChannel} +import org.apache.arrow.vector.ipc.message.MessageChannelReader +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} + +/** + * A reader that consumes Arrow data from an input channel, and produces Comet batches. + */ +case class StreamReader(channel: ReadableByteChannel) extends AutoCloseable { + private var allocator = new RootAllocator(Long.MaxValue) + private val channelReader = new MessageChannelReader(new ReadChannel(channel), allocator) + private var arrowReader = new ArrowStreamReader(channelReader, allocator) + private var root = arrowReader.getVectorSchemaRoot + + def nextBatch(): Option[ColumnarBatch] = { + if (arrowReader.loadNextBatch()) { + Some(rootAsBatch(root)) + } else { + None + } + } + + private def rootAsBatch(root: VectorSchemaRoot): ColumnarBatch = { + val columns = root.getFieldVectors.asScala.map { vector => + // Native shuffle always uses decimal128. + CometVector.getVector(vector, true, arrowReader).asInstanceOf[ColumnVector] + }.toArray + val batch = new ColumnarBatch(columns) + batch.setNumRows(root.getRowCount) + batch + } + + override def close(): Unit = { + if (root != null) { + arrowReader.close() + root.close() + allocator.close() + + arrowReader = null + root = null + allocator = null + } + } +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala b/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala new file mode 100644 index 0000000000..0e8a190c2d --- /dev/null +++ b/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala @@ -0,0 +1,378 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import java.util.{Locale, UUID} + +import scala.collection.JavaConverters._ + +import org.apache.parquet.schema._ +import org.apache.parquet.schema.LogicalTypeAnnotation.ListLogicalTypeAnnotation +import org.apache.parquet.schema.Type.Repetition +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.types._ + +import org.apache.comet.parquet.CometParquetUtils + +/** + * This class is copied & slightly modified from [[ParquetReadSupport]] in Spark. Changes: + * - This doesn't extend from Parquet's `ReadSupport` class since that is used for row-based + * Parquet reader. Therefore, there is no `init`, `prepareForRead` as well as other methods + * that are unused. + */ +object CometParquetReadSupport { + val SPARK_PARQUET_SCHEMA_NAME = "spark_schema" + + val EMPTY_MESSAGE: MessageType = + Types.buildMessage().named(SPARK_PARQUET_SCHEMA_NAME) + + def generateFakeColumnName: String = s"_fake_name_${UUID.randomUUID()}" + + def clipParquetSchema( + parquetSchema: MessageType, + catalystSchema: StructType, + caseSensitive: Boolean, + useFieldId: Boolean, + ignoreMissingIds: Boolean): MessageType = { + if (!ignoreMissingIds && + !containsFieldIds(parquetSchema) && + CometParquetUtils.hasFieldIds(catalystSchema)) { + throw new RuntimeException( + "Spark read schema expects field Ids, " + + "but Parquet file schema doesn't contain any field Ids.\n" + + "Please remove the field ids from Spark schema or ignore missing ids by " + + "setting `spark.sql.parquet.fieldId.read.ignoreMissing = true`\n" + + s""" + |Spark read schema: + |${catalystSchema.prettyJson} + | + |Parquet file schema: + |${parquetSchema.toString} + |""".stripMargin) + } + clipParquetSchema(parquetSchema, catalystSchema, caseSensitive, useFieldId) + } + + /** + * Tailors `parquetSchema` according to `catalystSchema` by removing column paths don't exist in + * `catalystSchema`, and adding those only exist in `catalystSchema`. + */ + def clipParquetSchema( + parquetSchema: MessageType, + catalystSchema: StructType, + caseSensitive: Boolean, + useFieldId: Boolean): MessageType = { + val clippedParquetFields = clipParquetGroupFields( + parquetSchema.asGroupType(), + catalystSchema, + caseSensitive, + useFieldId) + if (clippedParquetFields.isEmpty) { + EMPTY_MESSAGE + } else { + Types + .buildMessage() + .addFields(clippedParquetFields: _*) + .named(SPARK_PARQUET_SCHEMA_NAME) + } + } + + private def clipParquetType( + parquetType: Type, + catalystType: DataType, + caseSensitive: Boolean, + useFieldId: Boolean): Type = { + val newParquetType = catalystType match { + case t: ArrayType if !isPrimitiveCatalystType(t.elementType) => + // Only clips array types with nested type as element type. + clipParquetListType(parquetType.asGroupType(), t.elementType, caseSensitive, useFieldId) + + case t: MapType + if !isPrimitiveCatalystType(t.keyType) || + !isPrimitiveCatalystType(t.valueType) => + // Only clips map types with nested key type or value type + clipParquetMapType( + parquetType.asGroupType(), + t.keyType, + t.valueType, + caseSensitive, + useFieldId) + + case t: StructType => + clipParquetGroup(parquetType.asGroupType(), t, caseSensitive, useFieldId) + + case _ => + // UDTs and primitive types are not clipped. For UDTs, a clipped version might not be able + // to be mapped to desired user-space types. So UDTs shouldn't participate schema merging. + parquetType + } + + if (useFieldId && parquetType.getId != null) { + newParquetType.withId(parquetType.getId.intValue()) + } else { + newParquetType + } + } + + /** + * Whether a Catalyst [[DataType]] is primitive. Primitive [[DataType]] is not equivalent to + * [[AtomicType]]. For example, [[CalendarIntervalType]] is primitive, but it's not an + * [[AtomicType]]. + */ + private def isPrimitiveCatalystType(dataType: DataType): Boolean = { + dataType match { + case _: ArrayType | _: MapType | _: StructType => false + case _ => true + } + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[ArrayType]]. The element type + * of the [[ArrayType]] should also be a nested type, namely an [[ArrayType]], a [[MapType]], or + * a [[StructType]]. + */ + private def clipParquetListType( + parquetList: GroupType, + elementType: DataType, + caseSensitive: Boolean, + useFieldId: Boolean): Type = { + // Precondition of this method, should only be called for lists with nested element types. + assert(!isPrimitiveCatalystType(elementType)) + + // Unannotated repeated group should be interpreted as required list of required element, so + // list element type is just the group itself. Clip it. + if (parquetList.getLogicalTypeAnnotation == null && + parquetList.isRepetition(Repetition.REPEATED)) { + clipParquetType(parquetList, elementType, caseSensitive, useFieldId) + } else { + assert( + parquetList.getLogicalTypeAnnotation.isInstanceOf[ListLogicalTypeAnnotation], + "Invalid Parquet schema. " + + "Logical type annotation of annotated Parquet lists must be ListLogicalTypeAnnotation: " + + parquetList.toString) + + assert( + parquetList.getFieldCount == 1 && parquetList + .getType(0) + .isRepetition(Repetition.REPEATED), + "Invalid Parquet schema. " + + "LIST-annotated group should only have exactly one repeated field: " + + parquetList) + + // Precondition of this method, should only be called for lists with nested element types. + assert(!parquetList.getType(0).isPrimitive) + + val repeatedGroup = parquetList.getType(0).asGroupType() + + // If the repeated field is a group with multiple fields, or the repeated field is a group + // with one field and is named either "array" or uses the LIST-annotated group's name with + // "_tuple" appended then the repeated type is the element type and elements are required. + // Build a new LIST-annotated group with clipped `repeatedGroup` as element type and the + // only field. + if (repeatedGroup.getFieldCount > 1 || + repeatedGroup.getName == "array" || + repeatedGroup.getName == parquetList.getName + "_tuple") { + Types + .buildGroup(parquetList.getRepetition) + .as(LogicalTypeAnnotation.listType()) + .addField(clipParquetType(repeatedGroup, elementType, caseSensitive, useFieldId)) + .named(parquetList.getName) + } else { + // Otherwise, the repeated field's type is the element type with the repeated field's + // repetition. + val newRepeatedGroup = Types + .repeatedGroup() + .addField( + clipParquetType(repeatedGroup.getType(0), elementType, caseSensitive, useFieldId)) + .named(repeatedGroup.getName) + + val newElementType = if (useFieldId && repeatedGroup.getId != null) { + newRepeatedGroup.withId(repeatedGroup.getId.intValue()) + } else { + newRepeatedGroup + } + + Types + .buildGroup(parquetList.getRepetition) + .as(LogicalTypeAnnotation.listType()) + .addField( + Types + .repeatedGroup() + .addField( + clipParquetType(repeatedGroup.getType(0), elementType, caseSensitive, useFieldId)) + .named(repeatedGroup.getName)) + .addField(newElementType) + .named(parquetList.getName) + } + } + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[MapType]]. Either key type or + * value type of the [[MapType]] must be a nested type, namely an [[ArrayType]], a [[MapType]], + * or a [[StructType]]. + */ + private def clipParquetMapType( + parquetMap: GroupType, + keyType: DataType, + valueType: DataType, + caseSensitive: Boolean, + useFieldId: Boolean): GroupType = { + // Precondition of this method, only handles maps with nested key types or value types. + assert(!isPrimitiveCatalystType(keyType) || !isPrimitiveCatalystType(valueType)) + + val repeatedGroup = parquetMap.getType(0).asGroupType() + val parquetKeyType = repeatedGroup.getType(0) + val parquetValueType = repeatedGroup.getType(1) + + val clippedRepeatedGroup = { + val newRepeatedGroup = Types + .repeatedGroup() + .as(repeatedGroup.getLogicalTypeAnnotation) + .addField(clipParquetType(parquetKeyType, keyType, caseSensitive, useFieldId)) + .addField(clipParquetType(parquetValueType, valueType, caseSensitive, useFieldId)) + .named(repeatedGroup.getName) + if (useFieldId && repeatedGroup.getId != null) { + newRepeatedGroup.withId(repeatedGroup.getId.intValue()) + } else { + newRepeatedGroup + } + } + + Types + .buildGroup(parquetMap.getRepetition) + .as(parquetMap.getLogicalTypeAnnotation) + .addField(clippedRepeatedGroup) + .named(parquetMap.getName) + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[StructType]]. + * + * @return + * A clipped [[GroupType]], which has at least one field. + * @note + * Parquet doesn't allow creating empty [[GroupType]] instances except for empty + * [[MessageType]]. Because it's legal to construct an empty requested schema for column + * pruning. + */ + private def clipParquetGroup( + parquetRecord: GroupType, + structType: StructType, + caseSensitive: Boolean, + useFieldId: Boolean): GroupType = { + val clippedParquetFields = + clipParquetGroupFields(parquetRecord, structType, caseSensitive, useFieldId) + Types + .buildGroup(parquetRecord.getRepetition) + .as(parquetRecord.getLogicalTypeAnnotation) + .addFields(clippedParquetFields: _*) + .named(parquetRecord.getName) + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[StructType]]. + * + * @return + * A list of clipped [[GroupType]] fields, which can be empty. + */ + private def clipParquetGroupFields( + parquetRecord: GroupType, + structType: StructType, + caseSensitive: Boolean, + useFieldId: Boolean): Seq[Type] = { + val toParquet = new CometSparkToParquetSchemaConverter( + writeLegacyParquetFormat = false, + useFieldId = useFieldId) + lazy val caseSensitiveParquetFieldMap = + parquetRecord.getFields.asScala.map(f => f.getName -> f).toMap + lazy val caseInsensitiveParquetFieldMap = + parquetRecord.getFields.asScala.groupBy(_.getName.toLowerCase(Locale.ROOT)) + lazy val idToParquetFieldMap = + parquetRecord.getFields.asScala.filter(_.getId != null).groupBy(f => f.getId.intValue()) + + def matchCaseSensitiveField(f: StructField): Type = { + caseSensitiveParquetFieldMap + .get(f.name) + .map(clipParquetType(_, f.dataType, caseSensitive, useFieldId)) + .getOrElse(toParquet.convertField(f)) + } + + def matchCaseInsensitiveField(f: StructField): Type = { + // Do case-insensitive resolution only if in case-insensitive mode + caseInsensitiveParquetFieldMap + .get(f.name.toLowerCase(Locale.ROOT)) + .map { parquetTypes => + if (parquetTypes.size > 1) { + // Need to fail if there is ambiguity, i.e. more than one field is matched + val parquetTypesString = parquetTypes.map(_.getName).mkString("[", ", ", "]") + throw QueryExecutionErrors.foundDuplicateFieldInCaseInsensitiveModeError( + f.name, + parquetTypesString) + } else { + clipParquetType(parquetTypes.head, f.dataType, caseSensitive, useFieldId) + } + } + .getOrElse(toParquet.convertField(f)) + } + + def matchIdField(f: StructField): Type = { + val fieldId = CometParquetUtils.getFieldId(f) + idToParquetFieldMap + .get(fieldId) + .map { parquetTypes => + if (parquetTypes.size > 1) { + // Need to fail if there is ambiguity, i.e. more than one field is matched + val parquetTypesString = parquetTypes.map(_.getName).mkString("[", ", ", "]") + throw CometParquetUtils.foundDuplicateFieldInFieldIdLookupModeError( + fieldId, + parquetTypesString) + } else { + clipParquetType(parquetTypes.head, f.dataType, caseSensitive, useFieldId) + } + } + .getOrElse { + // When there is no ID match, we use a fake name to avoid a name match by accident + // We need this name to be unique as well, otherwise there will be type conflicts + toParquet.convertField(f.copy(name = generateFakeColumnName)) + } + } + + val shouldMatchById = useFieldId && CometParquetUtils.hasFieldIds(structType) + structType.map { f => + if (shouldMatchById && CometParquetUtils.hasFieldId(f)) { + matchIdField(f) + } else if (caseSensitive) { + matchCaseSensitiveField(f) + } else { + matchCaseInsensitiveField(f) + } + } + } + + /** + * Whether the parquet schema contains any field IDs. + */ + private def containsFieldIds(schema: Type): Boolean = schema match { + case p: PrimitiveType => p.getId != null + // We don't require all fields to have IDs, so we use `exists` here. + case g: GroupType => g.getId != null || g.getFields.asScala.exists(containsFieldIds) + } +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala b/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala new file mode 100644 index 0000000000..2c8187e18f --- /dev/null +++ b/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala @@ -0,0 +1,322 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import org.apache.hadoop.conf.Configuration +import org.apache.parquet.schema._ +import org.apache.parquet.schema.LogicalTypeAnnotation._ +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ +import org.apache.parquet.schema.Type.Repetition._ +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import org.apache.comet.parquet.CometParquetUtils + +/** + * This class is copied & modified from Spark's [[SparkToParquetSchemaConverter]] class. + */ +class CometSparkToParquetSchemaConverter( + writeLegacyParquetFormat: Boolean = SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get, + outputTimestampType: SQLConf.ParquetOutputTimestampType.Value = + SQLConf.ParquetOutputTimestampType.INT96, + useFieldId: Boolean = CometParquetUtils.writeFieldId(new SQLConf)) { + + def this(conf: SQLConf) = this( + writeLegacyParquetFormat = conf.writeLegacyParquetFormat, + outputTimestampType = conf.parquetOutputTimestampType, + useFieldId = CometParquetUtils.writeFieldId(conf)) + + def this(conf: Configuration) = this( + writeLegacyParquetFormat = conf.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key).toBoolean, + outputTimestampType = SQLConf.ParquetOutputTimestampType.withName( + conf.get(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key)), + useFieldId = CometParquetUtils.writeFieldId(conf)) + + /** + * Converts a Spark SQL [[StructType]] to a Parquet [[MessageType]]. + */ + def convert(catalystSchema: StructType): MessageType = { + Types + .buildMessage() + .addFields(catalystSchema.map(convertField): _*) + .named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + } + + /** + * Converts a Spark SQL [[StructField]] to a Parquet [[Type]]. + */ + def convertField(field: StructField): Type = { + val converted = convertField(field, if (field.nullable) OPTIONAL else REQUIRED) + if (useFieldId && CometParquetUtils.hasFieldId(field)) { + converted.withId(CometParquetUtils.getFieldId(field)) + } else { + converted + } + } + + private def convertField(field: StructField, repetition: Type.Repetition): Type = { + + field.dataType match { + // =================== + // Simple atomic types + // =================== + + case BooleanType => + Types.primitive(BOOLEAN, repetition).named(field.name) + + case ByteType => + Types + .primitive(INT32, repetition) + .as(LogicalTypeAnnotation.intType(8, true)) + .named(field.name) + + case ShortType => + Types + .primitive(INT32, repetition) + .as(LogicalTypeAnnotation.intType(16, true)) + .named(field.name) + + case IntegerType => + Types.primitive(INT32, repetition).named(field.name) + + case LongType => + Types.primitive(INT64, repetition).named(field.name) + + case FloatType => + Types.primitive(FLOAT, repetition).named(field.name) + + case DoubleType => + Types.primitive(DOUBLE, repetition).named(field.name) + + case StringType => + Types + .primitive(BINARY, repetition) + .as(LogicalTypeAnnotation.stringType()) + .named(field.name) + + case DateType => + Types + .primitive(INT32, repetition) + .as(LogicalTypeAnnotation.dateType()) + .named(field.name) + + // NOTE: Spark SQL can write timestamp values to Parquet using INT96, TIMESTAMP_MICROS or + // TIMESTAMP_MILLIS. TIMESTAMP_MICROS is recommended but INT96 is the default to keep the + // behavior same as before. + // + // As stated in PARQUET-323, Parquet `INT96` was originally introduced to represent nanosecond + // timestamp in Impala for some historical reasons. It's not recommended to be used for any + // other types and will probably be deprecated in some future version of parquet-format spec. + // That's the reason why parquet-format spec only defines `TIMESTAMP_MILLIS` and + // `TIMESTAMP_MICROS` which are both logical types annotating `INT64`. + // + // Originally, Spark SQL uses the same nanosecond timestamp type as Impala and Hive. Starting + // from Spark 1.5.0, we resort to a timestamp type with microsecond precision so that we can + // store a timestamp into a `Long`. This design decision is subject to change though, for + // example, we may resort to nanosecond precision in the future. + case TimestampType => + outputTimestampType match { + case SQLConf.ParquetOutputTimestampType.INT96 => + Types.primitive(INT96, repetition).named(field.name) + case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS => + Types + .primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)) + .named(field.name) + case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS => + Types + .primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS)) + .named(field.name) + } + + case TimestampNTZType => + Types + .primitive(INT64, repetition) + .as(LogicalTypeAnnotation.timestampType(false, TimeUnit.MICROS)) + .named(field.name) + + case BinaryType => + Types.primitive(BINARY, repetition).named(field.name) + + // ====================== + // Decimals (legacy mode) + // ====================== + + // Spark 1.4.x and prior versions only support decimals with a maximum precision of 18 and + // always store decimals in fixed-length byte arrays. To keep compatibility with these older + // versions, here we convert decimals with all precisions to `FIXED_LEN_BYTE_ARRAY` annotated + // by `DECIMAL`. + case DecimalType.Fixed(precision, scale) if writeLegacyParquetFormat => + Types + .primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .as(LogicalTypeAnnotation.decimalType(scale, precision)) + .length(Decimal.minBytesForPrecision(precision)) + .named(field.name) + + // ======================== + // Decimals (standard mode) + // ======================== + + // Uses INT32 for 1 <= precision <= 9 + case DecimalType.Fixed(precision, scale) + if precision <= Decimal.MAX_INT_DIGITS && !writeLegacyParquetFormat => + Types + .primitive(INT32, repetition) + .as(LogicalTypeAnnotation.decimalType(scale, precision)) + .named(field.name) + + // Uses INT64 for 1 <= precision <= 18 + case DecimalType.Fixed(precision, scale) + if precision <= Decimal.MAX_LONG_DIGITS && !writeLegacyParquetFormat => + Types + .primitive(INT64, repetition) + .as(LogicalTypeAnnotation.decimalType(scale, precision)) + .named(field.name) + + // Uses FIXED_LEN_BYTE_ARRAY for all other precisions + case DecimalType.Fixed(precision, scale) if !writeLegacyParquetFormat => + Types + .primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .as(LogicalTypeAnnotation.decimalType(scale, precision)) + .length(Decimal.minBytesForPrecision(precision)) + .named(field.name) + + // =================================== + // ArrayType and MapType (legacy mode) + // =================================== + + // Spark 1.4.x and prior versions convert `ArrayType` with nullable elements into a 3-level + // `LIST` structure. This behavior is somewhat a hybrid of parquet-hive and parquet-avro + // (1.6.0rc3): the 3-level structure is similar to parquet-hive while the 3rd level element + // field name "array" is borrowed from parquet-avro. + case ArrayType(elementType, nullable @ true) if writeLegacyParquetFormat => + // <list-repetition> group <name> (LIST) { + // optional group bag { + // repeated <element-type> array; + // } + // } + + // This should not use `listOfElements` here because this new method checks if the + // element name is `element` in the `GroupType` and throws an exception if not. + // As mentioned above, Spark prior to 1.4.x writes `ArrayType` as `LIST` but with + // `array` as its element name as below. Therefore, we build manually + // the correct group type here via the builder. (See SPARK-16777) + Types + .buildGroup(repetition) + .as(LogicalTypeAnnotation.listType()) + .addField( + Types + .buildGroup(REPEATED) + // "array" is the name chosen by parquet-hive (1.7.0 and prior version) + .addField(convertField(StructField("array", elementType, nullable))) + .named("bag")) + .named(field.name) + + // Spark 1.4.x and prior versions convert ArrayType with non-nullable elements into a 2-level + // LIST structure. This behavior mimics parquet-avro (1.6.0rc3). Note that this case is + // covered by the backwards-compatibility rules implemented in `isElementType()`. + case ArrayType(elementType, nullable @ false) if writeLegacyParquetFormat => + // <list-repetition> group <name> (LIST) { + // repeated <element-type> element; + // } + + // Here too, we should not use `listOfElements`. (See SPARK-16777) + Types + .buildGroup(repetition) + .as(LogicalTypeAnnotation.listType()) + // "array" is the name chosen by parquet-avro (1.7.0 and prior version) + .addField(convertField(StructField("array", elementType, nullable), REPEATED)) + .named(field.name) + + // Spark 1.4.x and prior versions convert MapType into a 3-level group annotated by + // MAP_KEY_VALUE. This is covered by `convertGroupField(field: GroupType): DataType`. + case MapType(keyType, valueType, valueContainsNull) if writeLegacyParquetFormat => + // <map-repetition> group <name> (MAP) { + // repeated group map (MAP_KEY_VALUE) { + // required <key-type> key; + // <value-repetition> <value-type> value; + // } + // } + ConversionPatterns.mapType( + repetition, + field.name, + convertField(StructField("key", keyType, nullable = false)), + convertField(StructField("value", valueType, valueContainsNull))) + + // ===================================== + // ArrayType and MapType (standard mode) + // ===================================== + + case ArrayType(elementType, containsNull) if !writeLegacyParquetFormat => + // <list-repetition> group <name> (LIST) { + // repeated group list { + // <element-repetition> <element-type> element; + // } + // } + Types + .buildGroup(repetition) + .as(LogicalTypeAnnotation.listType()) + .addField( + Types + .repeatedGroup() + .addField(convertField(StructField("element", elementType, containsNull))) + .named("list")) + .named(field.name) + + case MapType(keyType, valueType, valueContainsNull) => + // <map-repetition> group <name> (MAP) { + // repeated group key_value { + // required <key-type> key; + // <value-repetition> <value-type> value; + // } + // } + Types + .buildGroup(repetition) + .as(LogicalTypeAnnotation.mapType()) + .addField( + Types + .repeatedGroup() + .addField(convertField(StructField("key", keyType, nullable = false))) + .addField(convertField(StructField("value", valueType, valueContainsNull))) + .named("key_value")) + .named(field.name) + + // =========== + // Other types + // =========== + + case StructType(fields) => + fields + .foldLeft(Types.buildGroup(repetition)) { (builder, field) => + builder.addField(convertField(field)) + } + .named(field.name) + + case udt: UserDefinedType[_] => + convertField(field.copy(dataType = udt.sqlType)) + + case _ => + throw QueryCompilationErrors.cannotConvertDataTypeToParquetTypeError(field) + } + } +} diff --git a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala new file mode 100644 index 0000000000..9e0541f44b --- /dev/null +++ b/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.util + +import java.io.File + +import scala.collection.JavaConverters._ + +import org.apache.arrow.vector.complex.MapVector +import org.apache.arrow.vector.types._ +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} +import org.apache.spark.sql.types._ + +object Utils { + def getConfPath(confFileName: String): String = { + sys.env + .get("COMET_CONF_DIR") + .map { t => new File(s"$t${File.separator}$confFileName") } + .filter(_.isFile) + .map(_.getAbsolutePath) + .orNull + } + + def stringToSeq(str: String): Seq[String] = { + str.split(",").map(_.trim()).filter(_.nonEmpty) + } + + def fromArrowField(field: Field): DataType = { + field.getType match { + case _: ArrowType.Map => + val elementField = field.getChildren.get(0) + val keyType = fromArrowField(elementField.getChildren.get(0)) + val valueType = fromArrowField(elementField.getChildren.get(1)) + MapType(keyType, valueType, elementField.getChildren.get(1).isNullable) + case ArrowType.List.INSTANCE => + val elementField = field.getChildren().get(0) + val elementType = fromArrowField(elementField) + ArrayType(elementType, containsNull = elementField.isNullable) + case ArrowType.Struct.INSTANCE => + val fields = field.getChildren().asScala.map { child => + val dt = fromArrowField(child) + StructField(child.getName, dt, child.isNullable) + } + StructType(fields.toSeq) + case arrowType => fromArrowType(arrowType) + } + } + + def fromArrowType(dt: ArrowType): DataType = dt match { + case ArrowType.Bool.INSTANCE => BooleanType + case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 => ByteType + case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 2 => ShortType + case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 4 => IntegerType + case int: ArrowType.Int if int.getIsSigned && int.getBitWidth == 8 * 8 => LongType + case float: ArrowType.FloatingPoint if float.getPrecision == FloatingPointPrecision.SINGLE => + FloatType + case float: ArrowType.FloatingPoint if float.getPrecision == FloatingPointPrecision.DOUBLE => + DoubleType + case ArrowType.Utf8.INSTANCE => StringType + case ArrowType.Binary.INSTANCE => BinaryType + case _: ArrowType.FixedSizeBinary => BinaryType + case d: ArrowType.Decimal => DecimalType(d.getPrecision, d.getScale) + case date: ArrowType.Date if date.getUnit == DateUnit.DAY => DateType + case ts: ArrowType.Timestamp if ts.getUnit == TimeUnit.MICROSECOND => TimestampType + case ArrowType.Null.INSTANCE => NullType + case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH => + YearMonthIntervalType() + case di: ArrowType.Interval if di.getUnit == IntervalUnit.DAY_TIME => DayTimeIntervalType() + case _ => throw new UnsupportedOperationException(s"Unsupported data type: ${dt.toString}") + } + + /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for TimestampTypes */ + def toArrowType(dt: DataType, timeZoneId: String): ArrowType = + dt match { + case BooleanType => ArrowType.Bool.INSTANCE + case ByteType => new ArrowType.Int(8, true) + case ShortType => new ArrowType.Int(8 * 2, true) + case IntegerType => new ArrowType.Int(8 * 4, true) + case LongType => new ArrowType.Int(8 * 8, true) + case FloatType => new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE) + case DoubleType => new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE) + case StringType => ArrowType.Utf8.INSTANCE + case BinaryType => ArrowType.Binary.INSTANCE + case DecimalType.Fixed(precision, scale) => new ArrowType.Decimal(precision, scale, 128) + case DateType => new ArrowType.Date(DateUnit.DAY) + case TimestampType => + if (timeZoneId == null) { + throw new UnsupportedOperationException( + s"${TimestampType.catalogString} must supply timeZoneId parameter") + } else { + new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId) + } + case TimestampNTZType => + new ArrowType.Timestamp(TimeUnit.MICROSECOND, null) + case _ => + throw new UnsupportedOperationException(s"Unsupported data type: ${dt.catalogString}") + } + + /** Maps field from Spark to Arrow. NOTE: timeZoneId required for TimestampType */ + def toArrowField(name: String, dt: DataType, nullable: Boolean, timeZoneId: String): Field = { + dt match { + case ArrayType(elementType, containsNull) => + val fieldType = new FieldType(nullable, ArrowType.List.INSTANCE, null) + new Field( + name, + fieldType, + Seq(toArrowField("element", elementType, containsNull, timeZoneId)).asJava) + case StructType(fields) => + val fieldType = new FieldType(nullable, ArrowType.Struct.INSTANCE, null) + new Field( + name, + fieldType, + fields + .map { field => + toArrowField(field.name, field.dataType, field.nullable, timeZoneId) + } + .toSeq + .asJava) + case MapType(keyType, valueType, valueContainsNull) => + val mapType = new FieldType(nullable, new ArrowType.Map(false), null) + // Note: Map Type struct can not be null, Struct Type key field can not be null + new Field( + name, + mapType, + Seq( + toArrowField( + MapVector.DATA_VECTOR_NAME, + new StructType() + .add(MapVector.KEY_NAME, keyType, nullable = false) + .add(MapVector.VALUE_NAME, valueType, nullable = valueContainsNull), + nullable = false, + timeZoneId)).asJava) + case dataType => + val fieldType = new FieldType(nullable, toArrowType(dataType, timeZoneId), null) + new Field(name, fieldType, Seq.empty[Field].asJava) + } + } + + /** + * Maps schema from Spark to Arrow. NOTE: timeZoneId required for TimestampType in StructType + */ + def toArrowSchema(schema: StructType, timeZoneId: String): Schema = { + new Schema(schema.map { field => + toArrowField(field.name, field.dataType, field.nullable, timeZoneId) + }.asJava) + } +} diff --git a/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java b/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java new file mode 100644 index 0000000000..d4e748a9b6 --- /dev/null +++ b/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.BiFunction; + +import scala.collection.JavaConverters; + +import org.junit.Test; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.FixedSizeBinaryVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.*; +import org.apache.spark.sql.vectorized.ColumnVector; + +import org.apache.comet.vector.CometPlainVector; +import org.apache.comet.vector.CometVector; + +import static org.apache.spark.sql.types.DataTypes.*; +import static org.junit.Assert.*; + +@SuppressWarnings("unchecked") +public class TestColumnReader { + private static final int BATCH_SIZE = 1024; + private static final List<DataType> TYPES = + Arrays.asList( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + BinaryType, + DecimalType.apply(5, 2), + DecimalType.apply(18, 10), + DecimalType.apply(19, 5)); + private static final List<Object> VALUES = + Arrays.asList( + true, + (byte) 42, + (short) 100, + 1000, + (long) 10000, + (float) 3.14, + 3.1415926, + new byte[] {1, 2, 3, 4, 5, 6, 7, 8}, + Decimal.apply("123.45"), + Decimal.apply("00.0123456789"), + Decimal.apply("-001234.56789")); + private static final List<BiFunction<CometVector, Integer, Object>> GETTERS = + Arrays.asList( + ColumnVector::getBoolean, + ColumnVector::getByte, + ColumnVector::getShort, + ColumnVector::getInt, + ColumnVector::getLong, + ColumnVector::getFloat, + ColumnVector::getDouble, + ColumnVector::getBinary, + (v, i) -> v.getDecimal(i, 5, 2), + (v, i) -> v.getDecimal(i, 18, 10), + (v, i) -> v.getDecimal(i, 19, 5)); + + @Test + public void testConstantVectors() { + for (int i = 0; i < TYPES.size(); i++) { + DataType type = TYPES.get(i); + StructField field = StructField.apply("f", type, false, null); + + List<Object> values = Collections.singletonList(VALUES.get(i)); + InternalRow row = GenericInternalRow.apply(JavaConverters.asScalaBuffer(values).toSeq()); + ConstantColumnReader reader = new ConstantColumnReader(field, BATCH_SIZE, row, 0, true); + reader.readBatch(BATCH_SIZE); + CometVector vector = reader.currentBatch(); + assertEquals(BATCH_SIZE, vector.numValues()); + assertEquals(0, vector.numNulls()); + for (int j = 0; j < BATCH_SIZE; j++) { + if (TYPES.get(i) == BinaryType || TYPES.get(i) == StringType) { + assertArrayEquals((byte[]) VALUES.get(i), (byte[]) GETTERS.get(i).apply(vector, j)); + } else { + assertEquals(VALUES.get(i), GETTERS.get(i).apply(vector, j)); + } + } + + // Test null values too + row.setNullAt(0); + reader = new ConstantColumnReader(field, BATCH_SIZE, row, 0, true); + reader.readBatch(BATCH_SIZE); + vector = reader.currentBatch(); + assertEquals(BATCH_SIZE, vector.numValues()); + assertEquals(BATCH_SIZE, vector.numNulls()); + for (int j = 0; j < BATCH_SIZE; j++) { + assertTrue(vector.isNullAt(j)); + } + } + + if (org.apache.spark.package$.MODULE$.SPARK_VERSION_SHORT().compareTo("3.4") >= 0) { + Metadata meta = new MetadataBuilder().putString("EXISTS_DEFAULT", "123").build(); + StructField field = StructField.apply("f", LongType, false, meta); + ConstantColumnReader reader = new ConstantColumnReader(field, BATCH_SIZE, true); + reader.readBatch(BATCH_SIZE); + CometVector vector = reader.currentBatch(); + + assertEquals(BATCH_SIZE, vector.numValues()); + assertEquals(0, vector.numNulls()); + for (int j = 0; j < BATCH_SIZE; j++) { + assertEquals(123, vector.getLong(j)); + } + } + } + + @Test + public void testRowIndexColumnVectors() { + StructField field = StructField.apply("f", LongType, false, null); + int bigBatchSize = BATCH_SIZE * 2; + int step = 4; + int batchSize = bigBatchSize / step; + long[] indices = new long[step * 2]; + List<Long> expected = new ArrayList<>(); + + long idx = 0, len = 0; + for (int i = 0; i < step; i++) { + idx = ThreadLocalRandom.current().nextLong(idx + len, Long.MAX_VALUE); + indices[i * 2] = idx; + len = ThreadLocalRandom.current().nextLong(Long.max(bigBatchSize - expected.size(), 0)); + indices[i * 2 + 1] = len; + for (int j = 0; j < len; j++) { + expected.add(idx + j); + } + } + + RowIndexColumnReader reader = new RowIndexColumnReader(field, BATCH_SIZE, indices); + for (int i = 0; i < step; i++) { + reader.readBatch(batchSize); + CometVector vector = reader.currentBatch(); + assertEquals( + Integer.min(batchSize, Integer.max(expected.size() - i * batchSize, 0)), + vector.numValues()); + assertEquals(0, vector.numNulls()); + for (int j = 0; j < vector.numValues(); j++) { + assertEquals((long) expected.get(i * batchSize + j), vector.getLong(j)); + } + } + + reader.close(); + } + + @Test + public void testIsFixedLength() { + BufferAllocator allocator = new RootAllocator(Integer.MAX_VALUE); + + ValueVector vv = new IntVector("v1", allocator); + CometVector vector = new CometPlainVector(vv, false); + assertTrue(vector.isFixedLength()); + + vv = new FixedSizeBinaryVector("v2", allocator, 12); + vector = new CometPlainVector(vv, false); + assertTrue(vector.isFixedLength()); + + vv = new VarBinaryVector("v3", allocator); + vector = new CometPlainVector(vv, false); + assertFalse(vector.isFixedLength()); + } +} diff --git a/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java b/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java new file mode 100644 index 0000000000..63bb65d5c8 --- /dev/null +++ b/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import org.junit.Assert; +import org.junit.Test; + +public class TestCometInputFile { + @Test + public void testIsAtLeastHadoop33() { + Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.3.0")); + Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.4.0-SNAPSHOT")); + Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.12.5")); + Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.20.6.4-xyz")); + + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7.2")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7.3-SNAPSHOT")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.2")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.0.2.5-abc")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.1.2-test")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3-SNAPSHOT")); + Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.2-SNAPSHOT")); + } +} diff --git a/common/src/test/java/org/apache/comet/parquet/TestFileReader.java b/common/src/test/java/org/apache/comet/parquet/TestFileReader.java new file mode 100644 index 0000000000..6e73f7510f --- /dev/null +++ b/common/src/test/java/org/apache/comet/parquet/TestFileReader.java @@ -0,0 +1,829 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.*; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.statistics.BinaryStatistics; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.filter2.predicate.FilterApi; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Operators; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.ParquetInputFormat; +import org.apache.parquet.hadoop.metadata.*; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.internal.column.columnindex.BoundaryOrder; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Types; + +import static org.apache.parquet.column.Encoding.*; +import static org.apache.parquet.format.converter.ParquetMetadataConverter.MAX_STATS_SIZE; +import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; + +@SuppressWarnings("deprecation") +public class TestFileReader { + private static final MessageType SCHEMA = + MessageTypeParser.parseMessageType( + "" + + "message m {" + + " required group a {" + + " required binary b;" + + " }" + + " required group c {" + + " required int64 d;" + + " }" + + "}"); + + private static final MessageType SCHEMA2 = + MessageTypeParser.parseMessageType( + "" + + "message root { " + + "required int32 id;" + + "required binary name(UTF8); " + + "required int32 num; " + + "required binary comment(UTF8);" + + "}"); + + private static final MessageType PROJECTED_SCHEMA2 = + MessageTypeParser.parseMessageType( + "" + + "message root { " + + "required int32 id;" + + "required binary name(UTF8); " + + "required binary comment(UTF8);" + + "}"); + + private static final String[] PATH1 = {"a", "b"}; + private static final ColumnDescriptor C1 = SCHEMA.getColumnDescription(PATH1); + private static final String[] PATH2 = {"c", "d"}; + private static final ColumnDescriptor C2 = SCHEMA.getColumnDescription(PATH2); + + private static final byte[] BYTES1 = {0, 1, 2, 3}; + private static final byte[] BYTES2 = {1, 2, 3, 4}; + private static final byte[] BYTES3 = {2, 3, 4, 5}; + private static final byte[] BYTES4 = {3, 4, 5, 6}; + private static final CompressionCodecName CODEC = CompressionCodecName.UNCOMPRESSED; + + private static final org.apache.parquet.column.statistics.Statistics<?> EMPTY_STATS = + org.apache.parquet.column.statistics.Statistics.getBuilderForReading( + Types.required(PrimitiveTypeName.BINARY).named("test_binary")) + .build(); + + @Rule public final TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testEnableReadParallel() { + Configuration configuration = new Configuration(); + ReadOptions options = ReadOptions.builder(configuration).build(); + + assertFalse(FileReader.shouldReadParallel(options, "hdfs")); + assertFalse(FileReader.shouldReadParallel(options, "file")); + assertFalse(FileReader.shouldReadParallel(options, null)); + assertTrue(FileReader.shouldReadParallel(options, "s3a")); + + options = ReadOptions.builder(configuration).enableParallelIO(false).build(); + assertFalse(FileReader.shouldReadParallel(options, "s3a")); + } + + @Test + public void testReadWrite() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + // Start a Parquet file with 2 row groups, each with 2 column chunks + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + w.start(); + w.startBlock(3); + w.startColumn(C1, 5, CODEC); + long c1Starts = w.getPos(); + long c1p1Starts = w.getPos(); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(BYTES1), EMPTY_STATS, 3, RLE, RLE, PLAIN); + w.endColumn(); + long c1Ends = w.getPos(); + w.startColumn(C2, 6, CODEC); + long c2Starts = w.getPos(); + w.writeDictionaryPage(new DictionaryPage(BytesInput.from(BYTES2), 4, RLE_DICTIONARY)); + long c2p1Starts = w.getPos(); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(BYTES2), EMPTY_STATS, 3, RLE, RLE, PLAIN); + w.writeDataPage(1, 4, BytesInput.from(BYTES2), EMPTY_STATS, 1, RLE, RLE, PLAIN); + w.endColumn(); + long c2Ends = w.getPos(); + w.endBlock(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, 7, RLE, RLE, PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, 8, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<>()); + + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader reader = new FileReader(file, options, cometOptions)) { + ParquetMetadata readFooter = reader.getFooter(); + assertEquals("footer: " + readFooter, 2, readFooter.getBlocks().size()); + BlockMetaData rowGroup = readFooter.getBlocks().get(0); + assertEquals(c1Ends - c1Starts, rowGroup.getColumns().get(0).getTotalSize()); + assertEquals(c2Ends - c2Starts, rowGroup.getColumns().get(1).getTotalSize()); + assertEquals(c2Ends - c1Starts, rowGroup.getTotalByteSize()); + + assertEquals(c1Starts, rowGroup.getColumns().get(0).getStartingPos()); + assertEquals(0, rowGroup.getColumns().get(0).getDictionaryPageOffset()); + assertEquals(c1p1Starts, rowGroup.getColumns().get(0).getFirstDataPageOffset()); + assertEquals(c2Starts, rowGroup.getColumns().get(1).getStartingPos()); + assertEquals(c2Starts, rowGroup.getColumns().get(1).getDictionaryPageOffset()); + assertEquals(c2p1Starts, rowGroup.getColumns().get(1).getFirstDataPageOffset()); + + HashSet<Encoding> expectedEncoding = new HashSet<>(); + expectedEncoding.add(PLAIN); + expectedEncoding.add(RLE); + assertEquals(expectedEncoding, rowGroup.getColumns().get(0).getEncodings()); + } + + // read first block of col #1 + try (FileReader r = new FileReader(file, options, cometOptions)) { + r.setRequestedSchema(Arrays.asList(SCHEMA.getColumnDescription(PATH1))); + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(pages, PATH1, 3, BytesInput.from(BYTES1)); + assertTrue(r.skipNextRowGroup()); + assertNull(r.readNextRowGroup()); + } + + // read all blocks of col #1 and #2 + try (FileReader r = new FileReader(file, options, cometOptions)) { + r.setRequestedSchema( + Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(pages, PATH1, 3, BytesInput.from(BYTES1)); + validateContains(pages, PATH2, 2, BytesInput.from(BYTES2)); + validateContains(pages, PATH2, 3, BytesInput.from(BYTES2)); + validateContains(pages, PATH2, 1, BytesInput.from(BYTES2)); + + pages = r.readNextRowGroup(); + assertEquals(4, pages.getRowCount()); + + validateContains(pages, PATH1, 7, BytesInput.from(BYTES3)); + validateContains(pages, PATH2, 8, BytesInput.from(BYTES4)); + + assertNull(r.readNextRowGroup()); + } + } + + @Test + public void testBloomFilterReadWrite() throws Exception { + MessageType schema = + MessageTypeParser.parseMessageType("message test { required binary foo; }"); + File testFile = temp.newFile(); + testFile.delete(); + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + configuration.set("parquet.bloom.filter.column.names", "foo"); + String[] colPath = {"foo"}; + + ColumnDescriptor col = schema.getColumnDescription(colPath); + BinaryStatistics stats1 = new BinaryStatistics(); + ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); + w.start(); + w.startBlock(3); + w.startColumn(col, 5, CODEC); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), stats1, 2, RLE, RLE, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(BYTES1), stats1, 2, RLE, RLE, PLAIN); + w.endColumn(); + BloomFilter blockSplitBloomFilter = new BlockSplitBloomFilter(0); + blockSplitBloomFilter.insertHash(blockSplitBloomFilter.hash(Binary.fromString("hello"))); + blockSplitBloomFilter.insertHash(blockSplitBloomFilter.hash(Binary.fromString("world"))); + addBloomFilter(w, "foo", blockSplitBloomFilter); + w.endBlock(); + w.end(new HashMap<>()); + + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader r = new FileReader(file, options, cometOptions)) { + ParquetMetadata footer = r.getFooter(); + r.setRequestedSchema(Arrays.asList(schema.getColumnDescription(colPath))); + BloomFilterReader bloomFilterReader = + new BloomFilterReader( + footer.getBlocks().get(0), + r.getFileMetaData().getFileDecryptor(), + r.getInputStream()); + BloomFilter bloomFilter = + bloomFilterReader.readBloomFilter(footer.getBlocks().get(0).getColumns().get(0)); + assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("hello")))); + assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("world")))); + } + } + + @Test + public void testReadWriteDataPageV2() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + w.start(); + w.startBlock(14); + + BytesInput repLevels = BytesInput.fromInt(2); + BytesInput defLevels = BytesInput.fromInt(1); + BytesInput data = BytesInput.fromInt(3); + BytesInput data2 = BytesInput.fromInt(10); + + org.apache.parquet.column.statistics.Statistics<?> statsC1P1 = createStatistics("s", "z", C1); + org.apache.parquet.column.statistics.Statistics<?> statsC1P2 = createStatistics("b", "d", C1); + + w.startColumn(C1, 6, CODEC); + long c1Starts = w.getPos(); + w.writeDataPageV2(4, 1, 3, repLevels, defLevels, PLAIN, data, 4, statsC1P1); + w.writeDataPageV2(3, 0, 3, repLevels, defLevels, PLAIN, data, 4, statsC1P2); + w.endColumn(); + long c1Ends = w.getPos(); + + w.startColumn(C2, 5, CODEC); + long c2Starts = w.getPos(); + w.writeDataPageV2(5, 2, 3, repLevels, defLevels, PLAIN, data2, 4, EMPTY_STATS); + w.writeDataPageV2(2, 0, 2, repLevels, defLevels, PLAIN, data2, 4, EMPTY_STATS); + w.endColumn(); + long c2Ends = w.getPos(); + + w.endBlock(); + w.end(new HashMap<>()); + + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader reader = new FileReader(file, options, cometOptions)) { + ParquetMetadata footer = reader.getFooter(); + assertEquals("footer: " + footer, 1, footer.getBlocks().size()); + assertEquals(c1Ends - c1Starts, footer.getBlocks().get(0).getColumns().get(0).getTotalSize()); + assertEquals(c2Ends - c2Starts, footer.getBlocks().get(0).getColumns().get(1).getTotalSize()); + assertEquals(c2Ends - c1Starts, footer.getBlocks().get(0).getTotalByteSize()); + + // check for stats + org.apache.parquet.column.statistics.Statistics<?> expectedStats = + createStatistics("b", "z", C1); + assertStatsValuesEqual( + expectedStats, footer.getBlocks().get(0).getColumns().get(0).getStatistics()); + + HashSet<Encoding> expectedEncoding = new HashSet<>(); + expectedEncoding.add(PLAIN); + assertEquals(expectedEncoding, footer.getBlocks().get(0).getColumns().get(0).getEncodings()); + } + + try (FileReader r = new FileReader(file, options, cometOptions)) { + r.setRequestedSchema( + Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); + PageReadStore pages = r.readNextRowGroup(); + assertEquals(14, pages.getRowCount()); + validateV2Page( + pages, + PATH1, + 3, + 4, + 1, + repLevels.toByteArray(), + defLevels.toByteArray(), + data.toByteArray(), + 12); + validateV2Page( + pages, + PATH1, + 3, + 3, + 0, + repLevels.toByteArray(), + defLevels.toByteArray(), + data.toByteArray(), + 12); + validateV2Page( + pages, + PATH2, + 3, + 5, + 2, + repLevels.toByteArray(), + defLevels.toByteArray(), + data2.toByteArray(), + 12); + validateV2Page( + pages, + PATH2, + 2, + 2, + 0, + repLevels.toByteArray(), + defLevels.toByteArray(), + data2.toByteArray(), + 12); + assertNull(r.readNextRowGroup()); + } + } + + @Test + public void testColumnIndexFilter() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + + w.start(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + // the first page contains one matching record + w.writeDataPage(1, 4, BytesInput.from(BYTES3), statsC2(2L), 1, RLE, RLE, PLAIN); + // all the records of the second page are larger than 2, so should be filtered out + w.writeDataPage(3, 4, BytesInput.from(BYTES4), statsC2(3L, 4L, 5L), 3, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + // the first page should be filtered out + w.writeDataPage(1, 4, BytesInput.from(BYTES3), statsC2(4L), 1, RLE, RLE, PLAIN); + // the second page will be read since it contains matching record + w.writeDataPage(3, 4, BytesInput.from(BYTES4), statsC2(0L, 1L, 3L), 3, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + + w.end(new HashMap<>()); + + // set a simple equality filter in the ParquetInputFormat + Operators.LongColumn c2 = FilterApi.longColumn("c.d"); + FilterPredicate p = FilterApi.eq(c2, 2L); + ParquetInputFormat.setFilterPredicate(configuration, p); + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = HadoopReadOptions.builder(configuration).build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader r = new FileReader(file, options, cometOptions)) { + assertEquals(4, r.getFilteredRecordCount()); + PageReadStore readStore = r.readNextFilteredRowGroup(); + + PageReader c1Reader = readStore.getPageReader(C1); + List<DataPage> c1Pages = new ArrayList<>(); + DataPage page; + while ((page = c1Reader.readPage()) != null) { + c1Pages.add(page); + } + // second page of c1 should be filtered out + assertEquals(1, c1Pages.size()); + validatePage(c1Pages.get(0), 2, BytesInput.from(BYTES1)); + + PageReader c2Reader = readStore.getPageReader(C2); + List<DataPage> c2Pages = new ArrayList<>(); + while ((page = c2Reader.readPage()) != null) { + c2Pages.add(page); + } + assertEquals(1, c2Pages.size()); + validatePage(c2Pages.get(0), 1, BytesInput.from(BYTES3)); + + // test the second row group + readStore = r.readNextFilteredRowGroup(); + assertNotNull(readStore); + + c1Reader = readStore.getPageReader(C1); + c1Pages.clear(); + while ((page = c1Reader.readPage()) != null) { + c1Pages.add(page); + } + // all pages of c1 should be retained + assertEquals(2, c1Pages.size()); + validatePage(c1Pages.get(0), 2, BytesInput.from(BYTES1)); + validatePage(c1Pages.get(1), 2, BytesInput.from(BYTES2)); + + c2Reader = readStore.getPageReader(C2); + c2Pages.clear(); + while ((page = c2Reader.readPage()) != null) { + c2Pages.add(page); + } + assertEquals(1, c2Pages.size()); + validatePage(c2Pages.get(0), 3, BytesInput.from(BYTES4)); + } + } + + @Test + public void testColumnIndexReadWrite() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + w.start(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, RLE, RLE, PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + w.startBlock(4); + w.startColumn(C1, 5, CODEC); + long c1p1Starts = w.getPos(); + w.writeDataPage( + 2, 4, BytesInput.from(BYTES1), statsC1(null, Binary.fromString("aaa")), 1, RLE, RLE, PLAIN); + long c1p2Starts = w.getPos(); + w.writeDataPage( + 3, + 4, + BytesInput.from(BYTES1), + statsC1(Binary.fromString("bbb"), Binary.fromString("ccc")), + 3, + RLE, + RLE, + PLAIN); + w.endColumn(); + long c1Ends = w.getPos(); + w.startColumn(C2, 6, CODEC); + long c2p1Starts = w.getPos(); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), statsC2(117L, 100L), 1, RLE, RLE, PLAIN); + long c2p2Starts = w.getPos(); + w.writeDataPage(3, 4, BytesInput.from(BYTES2), statsC2(null, null, null), 2, RLE, RLE, PLAIN); + long c2p3Starts = w.getPos(); + w.writeDataPage(1, 4, BytesInput.from(BYTES2), statsC2(0L), 1, RLE, RLE, PLAIN); + w.endColumn(); + long c2Ends = w.getPos(); + w.endBlock(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + w.writeDataPage( + 7, + 4, + BytesInput.from(BYTES3), + // Creating huge stats so the column index will reach the limit and won't be written + statsC1( + Binary.fromConstantByteArray(new byte[(int) MAX_STATS_SIZE]), + Binary.fromConstantByteArray(new byte[1])), + 4, + RLE, + RLE, + PLAIN); + w.endColumn(); + w.startColumn(C2, 8, CODEC); + w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, RLE, RLE, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<>()); + + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + try (FileReader reader = new FileReader(file, options, cometOptions)) { + ParquetMetadata footer = reader.getFooter(); + assertEquals(3, footer.getBlocks().size()); + BlockMetaData blockMeta = footer.getBlocks().get(1); + assertEquals(2, blockMeta.getColumns().size()); + + ColumnIndexReader indexReader = reader.getColumnIndexReader(1); + ColumnIndex columnIndex = indexReader.readColumnIndex(blockMeta.getColumns().get(0)); + assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder()); + assertEquals(Arrays.asList(1L, 0L), columnIndex.getNullCounts()); + assertEquals(Arrays.asList(false, false), columnIndex.getNullPages()); + List<ByteBuffer> minValues = columnIndex.getMinValues(); + assertEquals(2, minValues.size()); + List<ByteBuffer> maxValues = columnIndex.getMaxValues(); + assertEquals(2, maxValues.size()); + assertEquals("aaa", new String(minValues.get(0).array(), StandardCharsets.UTF_8)); + assertEquals("aaa", new String(maxValues.get(0).array(), StandardCharsets.UTF_8)); + assertEquals("bbb", new String(minValues.get(1).array(), StandardCharsets.UTF_8)); + assertEquals("ccc", new String(maxValues.get(1).array(), StandardCharsets.UTF_8)); + + columnIndex = indexReader.readColumnIndex(blockMeta.getColumns().get(1)); + assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder()); + assertEquals(Arrays.asList(0L, 3L, 0L), columnIndex.getNullCounts()); + assertEquals(Arrays.asList(false, true, false), columnIndex.getNullPages()); + minValues = columnIndex.getMinValues(); + assertEquals(3, minValues.size()); + maxValues = columnIndex.getMaxValues(); + assertEquals(3, maxValues.size()); + assertEquals(100, BytesUtils.bytesToLong(minValues.get(0).array())); + assertEquals(117, BytesUtils.bytesToLong(maxValues.get(0).array())); + assertEquals(0, minValues.get(1).array().length); + assertEquals(0, maxValues.get(1).array().length); + assertEquals(0, BytesUtils.bytesToLong(minValues.get(2).array())); + assertEquals(0, BytesUtils.bytesToLong(maxValues.get(2).array())); + + OffsetIndex offsetIndex = indexReader.readOffsetIndex(blockMeta.getColumns().get(0)); + assertEquals(2, offsetIndex.getPageCount()); + assertEquals(c1p1Starts, offsetIndex.getOffset(0)); + assertEquals(c1p2Starts, offsetIndex.getOffset(1)); + assertEquals(c1p2Starts - c1p1Starts, offsetIndex.getCompressedPageSize(0)); + assertEquals(c1Ends - c1p2Starts, offsetIndex.getCompressedPageSize(1)); + assertEquals(0, offsetIndex.getFirstRowIndex(0)); + assertEquals(1, offsetIndex.getFirstRowIndex(1)); + + offsetIndex = indexReader.readOffsetIndex(blockMeta.getColumns().get(1)); + assertEquals(3, offsetIndex.getPageCount()); + assertEquals(c2p1Starts, offsetIndex.getOffset(0)); + assertEquals(c2p2Starts, offsetIndex.getOffset(1)); + assertEquals(c2p3Starts, offsetIndex.getOffset(2)); + assertEquals(c2p2Starts - c2p1Starts, offsetIndex.getCompressedPageSize(0)); + assertEquals(c2p3Starts - c2p2Starts, offsetIndex.getCompressedPageSize(1)); + assertEquals(c2Ends - c2p3Starts, offsetIndex.getCompressedPageSize(2)); + assertEquals(0, offsetIndex.getFirstRowIndex(0)); + assertEquals(1, offsetIndex.getFirstRowIndex(1)); + assertEquals(3, offsetIndex.getFirstRowIndex(2)); + + assertNull(indexReader.readColumnIndex(footer.getBlocks().get(2).getColumns().get(0))); + } + } + + // Test reader with merging of scan ranges enabled + @Test + public void testWriteReadMergeScanRange() throws Throwable { + Configuration conf = new Configuration(); + conf.set(ReadOptions.COMET_IO_MERGE_RANGES, Boolean.toString(true)); + // Set the merge range delta so small that ranges do not get merged + conf.set(ReadOptions.COMET_IO_MERGE_RANGES_DELTA, Integer.toString(1024)); + testReadWrite(conf, 2, 1024); + // Set the merge range delta so large that all ranges get merged + conf.set(ReadOptions.COMET_IO_MERGE_RANGES_DELTA, Integer.toString(1024 * 1024)); + testReadWrite(conf, 2, 1024); + } + + // `addBloomFilter` is package-private in Parquet, so this uses reflection to access it + private void addBloomFilter(ParquetFileWriter w, String s, BloomFilter filter) throws Exception { + Method method = + ParquetFileWriter.class.getDeclaredMethod( + "addBloomFilter", String.class, BloomFilter.class); + method.setAccessible(true); + method.invoke(w, s, filter); + } + + private void validateContains(PageReadStore pages, String[] path, int values, BytesInput bytes) + throws IOException { + PageReader pageReader = pages.getPageReader(SCHEMA.getColumnDescription(path)); + DataPage page = pageReader.readPage(); + validatePage(page, values, bytes); + } + + private void validatePage(DataPage page, int values, BytesInput bytes) throws IOException { + assertEquals(values, page.getValueCount()); + assertArrayEquals(bytes.toByteArray(), ((DataPageV1) page).getBytes().toByteArray()); + } + + private void validateV2Page( + PageReadStore pages, + String[] path, + int values, + int rows, + int nullCount, + byte[] repetition, + byte[] definition, + byte[] data, + int uncompressedSize) + throws IOException { + PageReader pageReader = pages.getPageReader(SCHEMA.getColumnDescription(path)); + DataPageV2 page = (DataPageV2) pageReader.readPage(); + assertEquals(values, page.getValueCount()); + assertEquals(rows, page.getRowCount()); + assertEquals(nullCount, page.getNullCount()); + assertEquals(uncompressedSize, page.getUncompressedSize()); + assertArrayEquals(repetition, page.getRepetitionLevels().toByteArray()); + assertArrayEquals(definition, page.getDefinitionLevels().toByteArray()); + assertArrayEquals(data, page.getData().toByteArray()); + } + + private Statistics<?> createStatistics(String min, String max, ColumnDescriptor col) { + return Statistics.getBuilderForReading(col.getPrimitiveType()) + .withMin(Binary.fromString(min).getBytes()) + .withMax(Binary.fromString(max).getBytes()) + .withNumNulls(0) + .build(); + } + + public static void assertStatsValuesEqual(Statistics<?> expected, Statistics<?> actual) { + if (expected == actual) { + return; + } + if (expected == null || actual == null) { + assertEquals(expected, actual); + } + Assert.assertArrayEquals(expected.getMaxBytes(), actual.getMaxBytes()); + Assert.assertArrayEquals(expected.getMinBytes(), actual.getMinBytes()); + Assert.assertEquals(expected.getNumNulls(), actual.getNumNulls()); + } + + private Statistics<?> statsC1(Binary... values) { + Statistics<?> stats = Statistics.createStats(C1.getPrimitiveType()); + for (Binary value : values) { + if (value == null) { + stats.incrementNumNulls(); + } else { + stats.updateStats(value); + } + } + return stats; + } + + /** + * Generates arbitrary data for simple schemas, writes the data to a file and also returns the + * data. + * + * @return array of data pages for each column + */ + private HashMap<String, byte[][]> generateAndWriteData( + Configuration configuration, + Path path, + MessageType schema, + int numPages, + int numRecordsPerPage) + throws IOException { + + HashMap<String, byte[][]> dataPages = new HashMap<>(); + + Generator generator = new Generator(); + ParquetFileWriter writer = new ParquetFileWriter(configuration, schema, path); + writer.start(); + writer.startBlock((long) numPages * numRecordsPerPage); + for (ColumnDescriptor colDesc : schema.getColumns()) { + writer.startColumn(colDesc, (long) numPages * numRecordsPerPage, CODEC); + String type = colDesc.getPrimitiveType().getName(); + byte[][] allPages = new byte[numPages][]; + byte[] data; + for (int i = 0; i < numPages; i++) { + data = generator.generateValues(numRecordsPerPage, type); + writer.writeDataPage( + numRecordsPerPage, + data.length, + BytesInput.from(data), + EMPTY_STATS, + numRecordsPerPage, + RLE, + RLE, + PLAIN); + allPages[i] = data; + } + dataPages.put(String.join(".", colDesc.getPath()), allPages); + writer.endColumn(); + } + writer.endBlock(); + writer.end(new HashMap<>()); + return dataPages; + } + + private void readAndValidatePageData( + InputFile inputFile, + ParquetReadOptions options, + ReadOptions cometOptions, + MessageType schema, + HashMap<String, byte[][]> expected, + int expectedValuesPerPage) + throws IOException { + try (FileReader fileReader = new FileReader(inputFile, options, cometOptions)) { + fileReader.setRequestedSchema(schema.getColumns()); + PageReadStore pages = fileReader.readNextRowGroup(); + for (ColumnDescriptor colDesc : schema.getColumns()) { + byte[][] allExpectedPages = expected.get(String.join(".", colDesc.getPath())); + PageReader pageReader = pages.getPageReader(colDesc); + for (byte[] expectedPage : allExpectedPages) { + DataPage page = pageReader.readPage(); + validatePage(page, expectedValuesPerPage, BytesInput.from(expectedPage)); + } + } + } + } + + public void testReadWrite(Configuration configuration, int numPages, int numRecordsPerPage) + throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + HashMap<String, byte[][]> dataPages = + generateAndWriteData(configuration, path, SCHEMA2, numPages, numRecordsPerPage); + InputFile file = HadoopInputFile.fromPath(path, configuration); + ParquetReadOptions options = ParquetReadOptions.builder().build(); + ReadOptions cometOptions = ReadOptions.builder(configuration).build(); + + readAndValidatePageData( + file, options, cometOptions, PROJECTED_SCHEMA2, dataPages, numRecordsPerPage); + } + + static class Generator { + + static Random random = new Random(1729); + private static final String ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz -"; + private static final int STR_MIN_SIZE = 5; + private static final int STR_MAX_SIZE = 30; + + private byte[] getString(int minSize, int maxSize) { + int size = random.nextInt(maxSize - minSize) + minSize; + byte[] str = new byte[size]; + for (int i = 0; i < size; ++i) { + str[i] = (byte) ALPHABET.charAt(random.nextInt(ALPHABET.length())); + } + return str; + } + + private byte[] generateValues(int numValues, String type) throws IOException { + + if (type.equals("int32")) { + byte[] data = new byte[4 * numValues]; + random.nextBytes(data); + return data; + } else { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + for (int i = 0; i < numValues; i++) { + outputStream.write(getString(STR_MIN_SIZE, STR_MAX_SIZE)); + } + return outputStream.toByteArray(); + } + } + } + + private Statistics<?> statsC2(Long... values) { + Statistics<?> stats = Statistics.createStats(C2.getPrimitiveType()); + for (Long value : values) { + if (value == null) { + stats.incrementNumNulls(); + } else { + stats.updateStats(value); + } + } + return stats; + } +} diff --git a/common/src/test/resources/log4j.properties b/common/src/test/resources/log4j.properties new file mode 100644 index 0000000000..2f46ce1553 --- /dev/null +++ b/common/src/test/resources/log4j.properties @@ -0,0 +1,36 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You 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. +# + +# Set everything to be logged to the file target/unit-tests.log +test.appender=file +log4j.rootCategory=INFO, ${test.appender} +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.sparkproject.jetty=WARN diff --git a/common/src/test/resources/log4j2.properties b/common/src/test/resources/log4j2.properties new file mode 100644 index 0000000000..04cdf85330 --- /dev/null +++ b/common/src/test/resources/log4j2.properties @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn + diff --git a/conf/log4rs.yaml b/conf/log4rs.yaml new file mode 100644 index 0000000000..43277918f1 --- /dev/null +++ b/conf/log4rs.yaml @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +appenders: + unittest: + kind: file + path: "target/unit-tests.log" + +root: + level: info + appenders: + - unittest diff --git a/core/.lldbinit b/core/.lldbinit new file mode 100644 index 0000000000..e7242277d4 --- /dev/null +++ b/core/.lldbinit @@ -0,0 +1,37 @@ +## +## From https://github.com/bsmt/lldbinit/blob/master/lldbinit +## + +# wish lldb supported colors :/ +settings set prompt [lldb-rust-comet]$ + +# breakpoint shortcuts +# break on function/method/selector: b -n name +# break on C/C++ method: b -M method +# break on selector: b -S selector:here: +# break on address: b -a 0xfeedface +command alias b breakpoint set +command alias bd breakpoint disable +command alias be breakpoint enable +command alias bdel breakpoint delete +command alias bcommand breakpoint command add +command alias commands breakpoint command list + +# jump aliases +# jump 0xfeedface +command alias jump register write pc +command alias jmp register write pc +command alias j register write pc + +# fix p/s +# p/s rsi +command alias p/s register read + +# fscript (cbf to fix fscript anywhere) +command alias f_init p (char)[[NSBundle bundleWithPath:@"/Library/Frameworks/FScript.framework"] load] +command alias f_start p (void)[FScriptMenuItem insertInMainMenu] + +command alias return thread return + +# For Comet Debugging +# settings set platform.plugin.darwin.ignored-exceptions EXC_BAD_INSTRUCTION|EXC_BAD_ACCESS diff --git a/core/Cargo.lock b/core/Cargo.lock new file mode 100644 index 0000000000..0585d7ec7d --- /dev/null +++ b/core/Cargo.lock @@ -0,0 +1,3191 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "addr2line" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.8.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77c3a9648d43b9cd48db467b3f87fdd6e146bcc88ab0180006cef2179fe11d01" +dependencies = [ + "cfg-if", + "const-random", + "getrandom", + "once_cell", + "version_check", + "zerocopy", +] + +[[package]] +name = "aho-corasick" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2969dcb958b36655471fc61f7e416fa76033bdd4bfed0678d8fee1e2d07a1f0" +dependencies = [ + "memchr", +] + +[[package]] +name = "alloc-no-stdlib" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" + +[[package]] +name = "alloc-stdlib" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" +dependencies = [ + "alloc-no-stdlib", +] + +[[package]] +name = "allocator-api2" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" + +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "anes" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" + +[[package]] +name = "anstyle" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" + +[[package]] +name = "anyhow" +version = "1.0.79" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "080e9890a082662b09c1ad45f567faeeb47f22b5fb23895fbe1e651e718e25ca" + +[[package]] +name = "arc-swap" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bddcadddf5e9015d310179a59bb28c4d4b9920ad0f11e8e14dbadf654890c9a6" + +[[package]] +name = "arrayref" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" + +[[package]] +name = "arrayvec" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" + +[[package]] +name = "arrow" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5bc25126d18a012146a888a0298f2c22e1150327bd2765fc76d710a556b2d614" +dependencies = [ + "ahash", + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-csv", + "arrow-data", + "arrow-ipc", + "arrow-json", + "arrow-ord", + "arrow-row", + "arrow-schema", + "arrow-select", + "arrow-string", +] + +[[package]] +name = "arrow-arith" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34ccd45e217ffa6e53bbb0080990e77113bdd4e91ddb84e97b77649810bcf1a7" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-array" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bda9acea48b25123c08340f3a8ac361aa0f74469bb36f5ee9acf923fce23e9d" +dependencies = [ + "ahash", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "chrono-tz", + "half 2.1.0", + "hashbrown 0.14.3", + "num", +] + +[[package]] +name = "arrow-buffer" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01a0fc21915b00fc6c2667b069c1b64bdd920982f426079bc4a7cab86822886c" +dependencies = [ + "bytes", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-cast" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dc0368ed618d509636c1e3cc20db1281148190a78f43519487b2daf07b63b4a" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "base64", + "chrono", + "comfy-table", + "half 2.1.0", + "lexical-core", + "num", +] + +[[package]] +name = "arrow-csv" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e09aa6246a1d6459b3f14baeaa49606cfdbca34435c46320e14054d244987ca" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "csv", + "csv-core", + "lazy_static", + "lexical-core", + "regex", +] + +[[package]] +name = "arrow-data" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "907fafe280a3874474678c1858b9ca4cb7fd83fb8034ff5b6d6376205a08c634" +dependencies = [ + "arrow-buffer", + "arrow-schema", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-ipc" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "79a43d6808411886b8c7d4f6f7dd477029c1e77ffffffb7923555cc6579639cd" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "flatbuffers", +] + +[[package]] +name = "arrow-json" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d82565c91fd627922ebfe2810ee4e8346841b6f9361b87505a9acea38b614fee" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "half 2.1.0", + "indexmap 2.1.0", + "lexical-core", + "num", + "serde", + "serde_json", +] + +[[package]] +name = "arrow-ord" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b23b0e53c0db57c6749997fd343d4c0354c994be7eca67152dd2bdb9a3e1bb4" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "half 2.1.0", + "num", +] + +[[package]] +name = "arrow-row" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "361249898d2d6d4a6eeb7484be6ac74977e48da12a4dd81a708d620cc558117a" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "half 2.1.0", + "hashbrown 0.14.3", +] + +[[package]] +name = "arrow-schema" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09e28a5e781bf1b0f981333684ad13f5901f4cd2f20589eab7cf1797da8fc167" +dependencies = [ + "bitflags 2.4.1", +] + +[[package]] +name = "arrow-select" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f6208466590960efc1d2a7172bc4ff18a67d6e25c529381d7f96ddaf0dc4036" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "num", +] + +[[package]] +name = "arrow-string" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4a48149c63c11c9ff571e50ab8f017d2a7cb71037a882b42f6354ed2da9acc7" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "num", + "regex", + "regex-syntax", +] + +[[package]] +name = "assertables" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" + +[[package]] +name = "async-trait" +version = "0.1.77" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c980ee35e870bd1a4d2c8294d4c04d0499e67bca1e4b5cefcc693c2fa00caea9" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "autocfg" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" + +[[package]] +name = "backtrace" +version = "0.3.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2089b7e3f35b9dd2d0ed921ead4f6d318c27680d4a5bd167b3ee120edb105837" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "base64" +version = "0.21.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35636a1494ede3b646cc98f74f8e62c773a38a659ebc777a2cf26b9b74171df9" + +[[package]] +name = "bitflags" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + +[[package]] +name = "bitflags" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" + +[[package]] +name = "blake2" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" +dependencies = [ + "digest", +] + +[[package]] +name = "blake3" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0231f06152bf547e9c2b5194f247cd97aacf6dcd8b15d8e5ec0663f64580da87" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + +[[package]] +name = "brotli" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "516074a47ef4bce09577a3b379392300159ce5b1ba2e501ff1c819950066100f" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor", +] + +[[package]] +name = "brotli-decompressor" +version = "2.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "bumpalo" +version = "3.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" + +[[package]] +name = "bytemuck" +version = "1.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "374d28ec25809ee0e23827c2ab573d729e293f281dfe393500e7ad618baa61c6" + +[[package]] +name = "byteorder" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" + +[[package]] +name = "bytes" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" + +[[package]] +name = "cast" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" + +[[package]] +name = "cc" +version = "1.0.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +dependencies = [ + "jobserver", + "libc", +] + +[[package]] +name = "cesu8" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chrono" +version = "0.4.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f2c685bad3eb3d45a01354cedb7d5faa66194d1d58ba6e267a8de788f79db38" +dependencies = [ + "android-tzdata", + "iana-time-zone", + "js-sys", + "num-traits", + "wasm-bindgen", + "windows-targets 0.48.5", +] + +[[package]] +name = "chrono-tz" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91d7b79e99bfaa0d47da0687c43aa3b7381938a62ad3a6498599039321f660b7" +dependencies = [ + "chrono", + "chrono-tz-build", + "phf", +] + +[[package]] +name = "chrono-tz-build" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "433e39f13c9a060046954e0592a8d0a4bcb1040125cbf91cb8ee58964cfb350f" +dependencies = [ + "parse-zoneinfo", + "phf", + "phf_codegen", +] + +[[package]] +name = "ciborium" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "effd91f6c78e5a4ace8a5d3c0b6bfaec9e2baaef55f3efc00e45fb2e477ee926" +dependencies = [ + "ciborium-io", + "ciborium-ll", + "serde", +] + +[[package]] +name = "ciborium-io" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cdf919175532b369853f5d5e20b26b43112613fd6fe7aee757e35f7a44642656" + +[[package]] +name = "ciborium-ll" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "defaa24ecc093c77630e6c15e17c51f5e187bf35ee514f4e2d67baaa96dae22b" +dependencies = [ + "ciborium-io", + "half 1.8.2", +] + +[[package]] +name = "clap" +version = "4.4.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52bdc885e4cacc7f7c9eedc1ef6da641603180c783c41a15c264944deeaab642" +dependencies = [ + "clap_builder", +] + +[[package]] +name = "clap_builder" +version = "4.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb7fb5e4e979aec3be7791562fcba452f94ad85e954da024396433e0e25a79e9" +dependencies = [ + "anstyle", + "clap_lex", +] + +[[package]] +name = "clap_lex" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "702fc72eb24e5a1e48ce58027a675bc24edd52096d5397d4aea7c6dd9eca0bd1" + +[[package]] +name = "combine" +version = "4.6.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35ed6e9d84f0b51a7f52daf1c7d71dd136fd7a3f41a8462b8cdb8c78d920fad4" +dependencies = [ + "bytes", + "memchr", +] + +[[package]] +name = "comet" +version = "0.1.0" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-data", + "arrow-schema", + "arrow-string", + "assertables", + "async-trait", + "brotli", + "byteorder", + "bytes", + "chrono", + "chrono-tz", + "crc32fast", + "criterion", + "datafusion", + "datafusion-common", + "datafusion-physical-expr", + "flate2", + "futures", + "half 2.1.0", + "hashbrown 0.14.3", + "itertools 0.11.0", + "jni", + "lazy_static", + "log", + "log4rs", + "lz4", + "mimalloc", + "num", + "once_cell", + "parking_lot", + "parquet", + "parquet-format", + "paste", + "pprof", + "prost 0.12.3", + "prost-build", + "rand", + "regex", + "serde", + "simd-adler32", + "snap", + "tempfile", + "thiserror", + "thrift 0.17.0", + "tokio", + "tokio-stream", + "unicode-segmentation", + "zstd", +] + +[[package]] +name = "comfy-table" +version = "7.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" +dependencies = [ + "strum", + "strum_macros", + "unicode-width", +] + +[[package]] +name = "const-random" +version = "0.1.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5aaf16c9c2c612020bcfd042e170f6e32de9b9d75adb5277cdbbd2e2c8c8299a" +dependencies = [ + "const-random-macro", +] + +[[package]] +name = "const-random-macro" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" +dependencies = [ + "getrandom", + "once_cell", + "tiny-keccak", +] + +[[package]] +name = "constant_time_eq" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" + +[[package]] +name = "core-foundation-sys" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" + +[[package]] +name = "cpp_demangle" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e8227005286ec39567949b33df9896bcadfa6051bccca2488129f108ca23119" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "cpufeatures" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53fe5e26ff1b7aef8bca9c6080520cfb8d9333c7568e1829cef191a9723e5504" +dependencies = [ + "libc", +] + +[[package]] +name = "crc32fast" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b540bd8bc810d3885c6ea91e2018302f68baba2129ab3e88f32389ee9370880d" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "criterion" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" +dependencies = [ + "anes", + "cast", + "ciborium", + "clap", + "criterion-plot", + "is-terminal", + "itertools 0.10.5", + "num-traits", + "once_cell", + "oorandom", + "plotters", + "rayon", + "regex", + "serde", + "serde_derive", + "serde_json", + "tinytemplate", + "walkdir", +] + +[[package]] +name = "criterion-plot" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +dependencies = [ + "cast", + "itertools 0.10.5", +] + +[[package]] +name = "crossbeam-deque" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fca89a0e215bab21874660c67903c5f143333cab1da83d041c7ded6053774751" +dependencies = [ + "cfg-if", + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e3681d554572a651dda4186cd47240627c3d0114d45a95f6ad27f2f22e7548d" +dependencies = [ + "autocfg", + "cfg-if", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3a430a770ebd84726f584a90ee7f020d28db52c6d02138900f22341f866d39c" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + +[[package]] +name = "crypto-common" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" +dependencies = [ + "memchr", +] + +[[package]] +name = "dashmap" +version = "5.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" +dependencies = [ + "cfg-if", + "hashbrown 0.14.3", + "lock_api", + "once_cell", + "parking_lot_core", +] + +[[package]] +name = "datafusion" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "193fd1e7628278d0641c5122860f9a7fd6a1d77d055838d12f55d15bbe28d4d0" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-schema", + "async-trait", + "bytes", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-optimizer", + "datafusion-physical-expr", + "datafusion-physical-plan", + "datafusion-sql", + "futures", + "glob", + "half 2.1.0", + "hashbrown 0.14.3", + "indexmap 2.1.0", + "itertools 0.12.0", + "log", + "num_cpus", + "object_store", + "parking_lot", + "pin-project-lite", + "rand", + "sqlparser", + "tempfile", + "tokio", + "tokio-util", + "url", + "uuid", +] + +[[package]] +name = "datafusion-common" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "548bc49c4a489e3de474813831ea556dc9d368f9ed8d867b1493da42e8e9f613" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "chrono", + "half 2.1.0", + "libc", + "num_cpus", + "object_store", + "sqlparser", +] + +[[package]] +name = "datafusion-execution" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ecc865657ffcf4da5ff08bdc6436a9a833bc0aa96c3254c8d18ab8a0ad4e437d" +dependencies = [ + "arrow", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-expr", + "futures", + "hashbrown 0.14.3", + "log", + "object_store", + "parking_lot", + "rand", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-expr" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33c473f72d8d81a532e63f6e562ed66dd9209dfd8e433d9712abd42444ee161e" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "datafusion-common", + "paste", + "sqlparser", + "strum", + "strum_macros", +] + +[[package]] +name = "datafusion-optimizer" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb6218318001d2f6783b7fffa17592318f65f26609d7aab605a3dd0c7c2e2618" +dependencies = [ + "arrow", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr", + "hashbrown 0.14.3", + "itertools 0.12.0", + "log", + "regex-syntax", +] + +[[package]] +name = "datafusion-physical-expr" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e1ca7e35ca22f9dc506c2375b92054b03ccf91afe25c0a90b395a1473a09735" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-ord", + "arrow-schema", + "base64", + "blake2", + "blake3", + "chrono", + "datafusion-common", + "datafusion-expr", + "half 2.1.0", + "hashbrown 0.14.3", + "hex", + "indexmap 2.1.0", + "itertools 0.12.0", + "log", + "md-5", + "paste", + "petgraph", + "rand", + "regex", + "sha2", + "unicode-segmentation", + "uuid", +] + +[[package]] +name = "datafusion-physical-plan" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ddde97adefcca3a55257c646ffee2a95b6cac66f74d1146a6e3a6dbb37830631" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "futures", + "half 2.1.0", + "hashbrown 0.14.3", + "indexmap 2.1.0", + "itertools 0.12.0", + "log", + "once_cell", + "parking_lot", + "pin-project-lite", + "rand", + "tokio", + "uuid", +] + +[[package]] +name = "datafusion-sql" +version = "34.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a60d9d6460a64fddb8663db41da97e6b8b0bf79da42f997ebe81722731eaf0e5" +dependencies = [ + "arrow", + "arrow-schema", + "datafusion-common", + "datafusion-expr", + "log", + "sqlparser", +] + +[[package]] +name = "debugid" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" +dependencies = [ + "uuid", +] + +[[package]] +name = "derivative" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "destructure_traitobject" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c877555693c14d2f84191cfd3ad8582790fc52b5e2274b40b59cf5f5cea25c7" + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + +[[package]] +name = "doc-comment" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" + +[[package]] +name = "either" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" + +[[package]] +name = "errno" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a258e46cdc063eb8519c00b9fc845fc47bcfca4130e2f08e88665ceda8474245" +dependencies = [ + "libc", + "windows-sys", +] + +[[package]] +name = "fastrand" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" + +[[package]] +name = "findshlibs" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40b9e59cd0f7e0806cca4be089683ecb6434e602038df21fe6bf6711b2f07f64" +dependencies = [ + "cc", + "lazy_static", + "libc", + "winapi", +] + +[[package]] +name = "fixedbitset" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" + +[[package]] +name = "flatbuffers" +version = "23.5.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4dac53e22462d78c16d64a1cd22371b54cc3fe94aa15e7886a2fa6e5d1ab8640" +dependencies = [ + "bitflags 1.3.2", + "rustc_version", +] + +[[package]] +name = "flate2" +version = "1.0.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" +dependencies = [ + "crc32fast", + "miniz_oxide", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "form_urlencoded" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" +dependencies = [ + "percent-encoding", +] + +[[package]] +name = "futures" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" + +[[package]] +name = "futures-executor" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" + +[[package]] +name = "futures-macro" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "futures-sink" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" + +[[package]] +name = "futures-task" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" + +[[package]] +name = "futures-util" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "pin-utils", + "slab", +] + +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + +[[package]] +name = "getrandom" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe9006bed769170c11f845cf00c7c1e9092aeb3f268e007c3e760ac68008070f" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.28.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4271d37baee1b8c7e4b708028c57d816cf9d2434acb33a549475f78c181f6253" + +[[package]] +name = "glob" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" + +[[package]] +name = "half" +version = "1.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eabb4a44450da02c90444cf74558da904edde8fb4e9035a9a6a4e15445af0bd7" + +[[package]] +name = "half" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad6a9459c9c30b177b925162351f97e7d967c7ea8bab3b8352805327daf45554" +dependencies = [ + "crunchy", + "num-traits", +] + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" + +[[package]] +name = "hashbrown" +version = "0.14.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290f1a1d9242c78d09ce40a5e87e7554ee637af1351968159f4952f028f75604" +dependencies = [ + "ahash", + "allocator-api2", +] + +[[package]] +name = "heck" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" +dependencies = [ + "unicode-segmentation", +] + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "hermit-abi" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d77f7ec81a6d05a3abb01ab6eb7590f6083d08449fe5a1c8b1e620283546ccb7" + +[[package]] +name = "hex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" + +[[package]] +name = "home" +version = "0.5.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" +dependencies = [ + "windows-sys", +] + +[[package]] +name = "humantime" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" + +[[package]] +name = "iana-time-zone" +version = "0.1.59" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6a67363e2aa4443928ce15e57ebae94fd8949958fd1223c4cfc0cd473ad7539" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "wasm-bindgen", + "windows-core", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +dependencies = [ + "cc", +] + +[[package]] +name = "idna" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" +dependencies = [ + "unicode-bidi", + "unicode-normalization", +] + +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", +] + +[[package]] +name = "indexmap" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d530e1a18b1cb4c484e6e34556a0d948706958449fca0cab753d649f2bce3d1f" +dependencies = [ + "equivalent", + "hashbrown 0.14.3", +] + +[[package]] +name = "inferno" +version = "0.11.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "321f0f839cd44a4686e9504b0a62b4d69a50b62072144c71c68f5873c167b8d9" +dependencies = [ + "ahash", + "indexmap 2.1.0", + "is-terminal", + "itoa", + "log", + "num-format", + "once_cell", + "quick-xml", + "rgb", + "str_stack", +] + +[[package]] +name = "integer-encoding" +version = "1.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48dc51180a9b377fd75814d0cc02199c20f8e99433d6762f650d39cdbbd3b56f" + +[[package]] +name = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + +[[package]] +name = "is-terminal" +version = "0.4.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bad00257d07be169d870ab665980b06cdb366d792ad690bf2e76876dc503455" +dependencies = [ + "hermit-abi", + "rustix", + "windows-sys", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25db6b064527c5d482d0423354fcd07a89a2dfe07b67892e62411946db7f07b0" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1a46d1a171d865aa5f83f92695765caa047a9b4cbae2cbf37dbd613a793fd4c" + +[[package]] +name = "jni" +version = "0.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c6df18c2e3db7e453d3c6ac5b3e9d5182664d28788126d39b91f2d1e22b017ec" +dependencies = [ + "cesu8", + "combine", + "jni-sys", + "log", + "thiserror", + "walkdir", +] + +[[package]] +name = "jni-sys" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" + +[[package]] +name = "jobserver" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" +dependencies = [ + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.66" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cee9c64da59eae3b50095c18d3e74f8b73c0b86d2792824ff01bbce68ba229ca" +dependencies = [ + "wasm-bindgen", +] + +[[package]] +name = "lazy_static" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" + +[[package]] +name = "lexical-core" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2cde5de06e8d4c2faabc400238f9ae1c74d5412d03a7bd067645ccbc47070e46" +dependencies = [ + "lexical-parse-float", + "lexical-parse-integer", + "lexical-util", + "lexical-write-float", + "lexical-write-integer", +] + +[[package]] +name = "lexical-parse-float" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "683b3a5ebd0130b8fb52ba0bdc718cc56815b6a097e28ae5a6997d0ad17dc05f" +dependencies = [ + "lexical-parse-integer", + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-parse-integer" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d0994485ed0c312f6d965766754ea177d07f9c00c9b82a5ee62ed5b47945ee9" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-util" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5255b9ff16ff898710eb9eb63cb39248ea8a5bb036bea8085b1a767ff6c4e3fc" +dependencies = [ + "static_assertions", +] + +[[package]] +name = "lexical-write-float" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "accabaa1c4581f05a3923d1b4cfd124c329352288b7b9da09e766b0668116862" +dependencies = [ + "lexical-util", + "lexical-write-integer", + "static_assertions", +] + +[[package]] +name = "lexical-write-integer" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1b6f3d1f4422866b68192d62f77bc5c700bee84f3069f2469d7bc8c77852446" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "libc" +version = "0.2.151" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "302d7ab3130588088d277783b1e2d2e10c9e9e4a16dd9050e6ec93fb3e7048f4" + +[[package]] +name = "libm" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" + +[[package]] +name = "libmimalloc-sys" +version = "0.1.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3979b5c37ece694f1f5e51e7ecc871fdb0f517ed04ee45f88d15d6d553cb9664" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "linked-hash-map" +version = "0.5.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" + +[[package]] +name = "linux-raw-sys" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4cd1a83af159aa67994778be9070f0ae1bd732942279cabb14f86f986a21456" + +[[package]] +name = "lock_api" +version = "0.4.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" +dependencies = [ + "serde", +] + +[[package]] +name = "log-mdc" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a94d21414c1f4a51209ad204c1776a3d0765002c76c6abcb602a6f09f1e881c7" + +[[package]] +name = "log4rs" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d36ca1786d9e79b8193a68d480a0907b612f109537115c6ff655a3a1967533fd" +dependencies = [ + "anyhow", + "arc-swap", + "chrono", + "derivative", + "fnv", + "humantime", + "libc", + "log", + "log-mdc", + "parking_lot", + "serde", + "serde-value", + "serde_json", + "serde_yaml", + "thiserror", + "thread-id", + "typemap-ors", + "winapi", +] + +[[package]] +name = "lz4" +version = "1.24.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" +dependencies = [ + "libc", + "lz4-sys", +] + +[[package]] +name = "lz4-sys" +version = "1.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "md-5" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" +dependencies = [ + "cfg-if", + "digest", +] + +[[package]] +name = "memchr" +version = "2.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "523dc4f511e55ab87b694dc30d0f820d60906ef06413f93d4d7a1385599cc149" + +[[package]] +name = "memmap2" +version = "0.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "45fd3a57831bf88bc63f8cebc0cf956116276e97fef3966103e96416209f7c92" +dependencies = [ + "libc", +] + +[[package]] +name = "mimalloc" +version = "0.1.39" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa01922b5ea280a911e323e4d2fd24b7fe5cc4042e0d2cda3c40775cdc4bdc9c" +dependencies = [ + "libmimalloc-sys", +] + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + +[[package]] +name = "multimap" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" + +[[package]] +name = "nix" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "598beaf3cc6fdd9a5dfb1630c2800c7acd31df7aaf0f565796fba2b53ca1af1b" +dependencies = [ + "bitflags 1.3.2", + "cfg-if", + "libc", +] + +[[package]] +name = "num" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b05180d69e3da0e530ba2a1dae5110317e49e3b7f3d41be227dc5f92e49ee7af" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + +[[package]] +name = "num-bigint" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "608e7659b5c3d7cba262d894801b9ec9d00de989e8a82bd4bef91d08da45cdc0" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-complex" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ba157ca0885411de85d6ca030ba7e2a83a28636056c7c699b07c8b6f7383214" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-format" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a652d9771a63711fd3c3deb670acfbe5c30a4072e664d7a3bf5a9e1056ac72c3" +dependencies = [ + "arrayvec", + "itoa", +] + +[[package]] +name = "num-integer" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" +dependencies = [ + "autocfg", + "num-traits", +] + +[[package]] +name = "num-iter" +version = "0.1.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d03e6c028c5dc5cac6e2dec0efda81fc887605bb3d884578bb6d6bf7514e252" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0638a1c9d0a3c0914158145bc76cff373a75a627e6ecbfb71cbe6f453a5a19b0" +dependencies = [ + "autocfg", + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-traits" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39e3200413f237f41ab11ad6d161bc7239c84dcb631773ccd7de3dfe4b5c267c" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "num_cpus" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" +dependencies = [ + "hermit-abi", + "libc", +] + +[[package]] +name = "object" +version = "0.32.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +dependencies = [ + "memchr", +] + +[[package]] +name = "object_store" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2524735495ea1268be33d200e1ee97455096a0846295a21548cd2f3541de7050" +dependencies = [ + "async-trait", + "bytes", + "chrono", + "futures", + "humantime", + "itertools 0.11.0", + "parking_lot", + "percent-encoding", + "snafu", + "tokio", + "tracing", + "url", + "walkdir", +] + +[[package]] +name = "once_cell" +version = "1.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" + +[[package]] +name = "oorandom" +version = "11.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" + +[[package]] +name = "ordered-float" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3305af35278dd29f46fcdd139e0b1fbfae2153f0e5928b39b035542dd31e37b7" +dependencies = [ + "num-traits", +] + +[[package]] +name = "ordered-float" +version = "2.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" +dependencies = [ + "num-traits", +] + +[[package]] +name = "parking_lot" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall", + "smallvec", + "windows-targets 0.48.5", +] + +[[package]] +name = "parquet" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af88740a842787da39b3d69ce5fbf6fce97d20211d3b299fee0a0da6430c74d4" +dependencies = [ + "ahash", + "bytes", + "chrono", + "hashbrown 0.14.3", + "num", + "num-bigint", + "paste", + "seq-macro", + "thrift 0.17.0", + "twox-hash", +] + +[[package]] +name = "parquet-format" +version = "4.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f0c06cdcd5460967c485f9c40a821746f5955ad81990533c7fae95dbd9bc0b5" +dependencies = [ + "thrift 0.13.0", +] + +[[package]] +name = "parse-zoneinfo" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c705f256449c60da65e11ff6626e0c16a0a0b96aaa348de61376b249bc340f41" +dependencies = [ + "regex", +] + +[[package]] +name = "paste" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de3145af08024dea9fa9914f381a17b8fc6034dfb00f3a84013f7ff43f29ed4c" + +[[package]] +name = "percent-encoding" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" + +[[package]] +name = "petgraph" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" +dependencies = [ + "fixedbitset", + "indexmap 2.1.0", +] + +[[package]] +name = "phf" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" +dependencies = [ + "phf_shared", +] + +[[package]] +name = "phf_codegen" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8d39688d359e6b34654d328e262234662d16cc0f60ec8dcbe5e718709342a5a" +dependencies = [ + "phf_generator", + "phf_shared", +] + +[[package]] +name = "phf_generator" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48e4cc64c2ad9ebe670cb8fd69dd50ae301650392e81c05f9bfcb2d5bdbc24b0" +dependencies = [ + "phf_shared", + "rand", +] + +[[package]] +name = "phf_shared" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b" +dependencies = [ + "siphasher", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8afb450f006bf6385ca15ef45d71d2288452bc3683ce2e2cacc0d18e4be60b58" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "pkg-config" +version = "0.3.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69d3587f8a9e599cc7ec2c00e331f71c4e69a5f9a4b8a6efd5b07466b9736f9a" + +[[package]] +name = "plotters" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2c224ba00d7cadd4d5c660deaf2098e5e80e07846537c51f9cfa4be50c1fd45" +dependencies = [ + "num-traits", + "plotters-backend", + "plotters-svg", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "plotters-backend" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e76628b4d3a7581389a35d5b6e2139607ad7c75b17aed325f210aa91f4a9609" + +[[package]] +name = "plotters-svg" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38f6d39893cca0701371e3c27294f09797214b86f1fb951b89ade8ec04e2abab" +dependencies = [ + "plotters-backend", +] + +[[package]] +name = "pprof" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef5c97c51bd34c7e742402e216abdeb44d415fbe6ae41d56b114723e953711cb" +dependencies = [ + "backtrace", + "cfg-if", + "findshlibs", + "inferno", + "libc", + "log", + "nix", + "once_cell", + "parking_lot", + "smallvec", + "symbolic-demangle", + "tempfile", + "thiserror", +] + +[[package]] +name = "ppv-lite86" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" + +[[package]] +name = "proc-macro2" +version = "1.0.75" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "907a61bd0f64c2f29cd1cf1dc34d05176426a3f504a78010f08416ddb7b13708" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prost" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" +dependencies = [ + "bytes", + "prost-derive 0.9.0", +] + +[[package]] +name = "prost" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "146c289cda302b98a28d40c8b3b90498d6e526dd24ac2ecea73e4e491685b94a" +dependencies = [ + "bytes", + "prost-derive 0.12.3", +] + +[[package]] +name = "prost-build" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" +dependencies = [ + "bytes", + "heck 0.3.3", + "itertools 0.10.5", + "lazy_static", + "log", + "multimap", + "petgraph", + "prost 0.9.0", + "prost-types", + "regex", + "tempfile", + "which", +] + +[[package]] +name = "prost-derive" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" +dependencies = [ + "anyhow", + "itertools 0.10.5", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "prost-derive" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "efb6c9a1dd1def8e2124d17e83a20af56f1570d6c2d2bd9e266ccb768df3840e" +dependencies = [ + "anyhow", + "itertools 0.11.0", + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "prost-types" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" +dependencies = [ + "bytes", + "prost 0.9.0", +] + +[[package]] +name = "quick-xml" +version = "0.26.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f50b1c63b38611e7d4d7f68b82d3ad0cc71a2ad2e7f61fc10f1328d917c93cd" +dependencies = [ + "memchr", +] + +[[package]] +name = "quote" +version = "1.0.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "291ec9ab5efd934aaf503a6466c5d5251535d108ee747472c3977cc5acc868ef" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha", + "rand_core", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom", +] + +[[package]] +name = "rayon" +version = "1.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c27db03db7734835b3f53954b534c91069375ce6ccaa2e065441e07d9b6cdb1" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ce3fb6ad83f861aac485e76e1985cd109d9a3713802152be56c3b1f0e0658ed" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + +[[package]] +name = "redox_syscall" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" +dependencies = [ + "bitflags 1.3.2", +] + +[[package]] +name = "regex" +version = "1.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" + +[[package]] +name = "rgb" +version = "0.8.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05aaa8004b64fd573fc9d002f4e632d51ad4f026c2b5ba95fcb6c2f32c2c47d8" +dependencies = [ + "bytemuck", +] + +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + +[[package]] +name = "rustc_version" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +dependencies = [ + "semver", +] + +[[package]] +name = "rustix" +version = "0.38.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72e572a5e8ca657d7366229cdde4bd14c4eb5499a9573d4d366fe1b599daa316" +dependencies = [ + "bitflags 2.4.1", + "errno", + "libc", + "linux-raw-sys", + "windows-sys", +] + +[[package]] +name = "rustversion" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" + +[[package]] +name = "ryu" +version = "1.0.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c" + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "semver" +version = "1.0.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b97ed7a9823b74f99c7742f5336af7be5ecd3eeafcb1507d1fa93347b1d589b0" + +[[package]] +name = "seq-macro" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" + +[[package]] +name = "serde" +version = "1.0.194" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b114498256798c94a0689e1a15fec6005dee8ac1f41de56404b67afc2a4b773" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde-value" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +dependencies = [ + "ordered-float 2.10.1", + "serde", +] + +[[package]] +name = "serde_derive" +version = "1.0.194" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a3385e45322e8f9931410f01b3031ec534c3947d0e94c18049af4d9f9907d4e0" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "serde_json" +version = "1.0.111" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "176e46fa42316f18edd598015a5166857fc835ec732f5215eac6b7bdbf0a84f4" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.8.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "578a7433b776b56a35785ed5ce9a7e777ac0598aac5a6dd1b4b18a307c7fc71b" +dependencies = [ + "indexmap 1.9.3", + "ryu", + "serde", + "yaml-rust", +] + +[[package]] +name = "sha2" +version = "0.10.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest", +] + +[[package]] +name = "simd-adler32" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" + +[[package]] +name = "siphasher" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" + +[[package]] +name = "slab" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" +dependencies = [ + "autocfg", +] + +[[package]] +name = "smallvec" +version = "1.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970" + +[[package]] +name = "snafu" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" +dependencies = [ + "doc-comment", + "snafu-derive", +] + +[[package]] +name = "snafu-derive" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "snap" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" + +[[package]] +name = "sqlparser" +version = "0.40.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c80afe31cdb649e56c0d9bb5503be9166600d68a852c38dd445636d126858e5" +dependencies = [ + "log", + "sqlparser_derive", +] + +[[package]] +name = "sqlparser_derive" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "stable_deref_trait" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" + +[[package]] +name = "static_assertions" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + +[[package]] +name = "str_stack" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" + +[[package]] +name = "strum" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.25.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.48", +] + +[[package]] +name = "subtle" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" + +[[package]] +name = "symbolic-common" +version = "12.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1cccfffbc6bb3bb2d3a26cd2077f4d055f6808d266f9d4d158797a4c60510dfe" +dependencies = [ + "debugid", + "memmap2", + "stable_deref_trait", + "uuid", +] + +[[package]] +name = "symbolic-demangle" +version = "12.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76a99812da4020a67e76c4eb41f08c87364c14170495ff780f30dd519c221a68" +dependencies = [ + "cpp_demangle", + "rustc-demangle", + "symbolic-common", +] + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.48" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f3531638e407dfc0814761abb7c00a5b54992b849452a0646b7f65c9f770f3f" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "tempfile" +version = "3.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01ce4141aa927a6d1bd34a041795abd0db1cccba5d5f24b009f694bdf3a1f3fa" +dependencies = [ + "cfg-if", + "fastrand", + "redox_syscall", + "rustix", + "windows-sys", +] + +[[package]] +name = "thiserror" +version = "1.0.56" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d54378c645627613241d077a3a79db965db602882668f9136ac42af9ecb730ad" +dependencies = [ + "thiserror-impl", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.56" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa0faa943b50f3db30a20aa7e265dbc66076993efed8463e8de414e5d06d3471" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "thread-id" +version = "4.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0ec81c46e9eb50deaa257be2f148adf052d1fb7701cfd55ccfab2525280b70b" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "threadpool" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d050e60b33d41c19108b32cea32164033a9013fe3b46cbd4457559bfbf77afaa" +dependencies = [ + "num_cpus", +] + +[[package]] +name = "thrift" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c6d965454947cc7266d22716ebfd07b18d84ebaf35eec558586bbb2a8cb6b5b" +dependencies = [ + "byteorder", + "integer-encoding 1.1.7", + "log", + "ordered-float 1.1.1", + "threadpool", +] + +[[package]] +name = "thrift" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" +dependencies = [ + "byteorder", + "integer-encoding 3.0.4", + "log", + "ordered-float 2.10.1", + "threadpool", +] + +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + +[[package]] +name = "tinytemplate" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" +dependencies = [ + "serde", + "serde_json", +] + +[[package]] +name = "tinyvec" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" +dependencies = [ + "tinyvec_macros", +] + +[[package]] +name = "tinyvec_macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" + +[[package]] +name = "tokio" +version = "1.35.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c89b4efa943be685f629b149f53829423f8f5531ea21249408e8e2f8671ec104" +dependencies = [ + "backtrace", + "bytes", + "num_cpus", + "parking_lot", + "pin-project-lite", + "tokio-macros", +] + +[[package]] +name = "tokio-macros" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "tokio-stream" +version = "0.1.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "397c988d37662c7dda6d2208364a706264bf3d6138b11d436cbac0ad38832842" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tokio-util" +version = "0.7.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" +dependencies = [ + "bytes", + "futures-core", + "futures-sink", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tracing" +version = "0.1.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" +dependencies = [ + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "tracing-core" +version = "0.1.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" +dependencies = [ + "once_cell", +] + +[[package]] +name = "twox-hash" +version = "1.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +dependencies = [ + "cfg-if", + "static_assertions", +] + +[[package]] +name = "typemap-ors" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a68c24b707f02dd18f1e4ccceb9d49f2058c2fb86384ef9972592904d7a28867" +dependencies = [ + "unsafe-any-ors", +] + +[[package]] +name = "typenum" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" + +[[package]] +name = "unicode-bidi" +version = "0.3.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f2528f27a9eb2b21e69c95319b30bd0efd85d09c379741b0f78ea1d86be2416" + +[[package]] +name = "unicode-ident" +version = "1.0.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" + +[[package]] +name = "unicode-normalization" +version = "0.1.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" +dependencies = [ + "tinyvec", +] + +[[package]] +name = "unicode-segmentation" +version = "1.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1dd624098567895118886609431a7c3b8f516e41d30e0643f03d94592a147e36" + +[[package]] +name = "unicode-width" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e51733f11c9c4f72aa0c160008246859e340b00807569a0da0e7a1079b27ba85" + +[[package]] +name = "unsafe-any-ors" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0a303d30665362d9680d7d91d78b23f5f899504d4f08b3c4cf08d055d87c0ad" +dependencies = [ + "destructure_traitobject", +] + +[[package]] +name = "url" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" +dependencies = [ + "form_urlencoded", + "idna", + "percent-encoding", +] + +[[package]] +name = "uuid" +version = "1.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e395fcf16a7a3d8127ec99782007af141946b4795001f876d54fb0d55978560" +dependencies = [ + "getrandom", +] + +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + +[[package]] +name = "walkdir" +version = "2.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d71d857dc86794ca4c280d616f7da00d2dbfd8cd788846559a6813e6aa4b54ee" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "wasm-bindgen" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ed0d4f68a3015cc185aff4db9506a015f4b96f95303897bfa23f846db54064e" +dependencies = [ + "cfg-if", + "wasm-bindgen-macro", +] + +[[package]] +name = "wasm-bindgen-backend" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b56f625e64f3a1084ded111c4d5f477df9f8c92df113852fa5a374dbda78826" +dependencies = [ + "bumpalo", + "log", + "once_cell", + "proc-macro2", + "quote", + "syn 2.0.48", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0162dbf37223cd2afce98f3d0785506dcb8d266223983e4b5b525859e6e182b2" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", + "wasm-bindgen-backend", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ab9b36309365056cd639da3134bf87fa8f3d86008abf99e612384a6eecd459f" + +[[package]] +name = "web-sys" +version = "0.3.66" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50c24a44ec86bb68fbecd1b3efed7e85ea5621b39b35ef2766b66cd984f8010f" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "which" +version = "4.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" +dependencies = [ + "either", + "home", + "once_cell", + "rustix", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-util" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f29e6f9198ba0d26b4c9f07dbe6f9ed633e1f3d5b8b414090084349e46a52596" +dependencies = [ + "winapi", +] + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows-core" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" +dependencies = [ + "windows-targets 0.52.0", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.0", +] + +[[package]] +name = "windows-targets" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" +dependencies = [ + "windows_aarch64_gnullvm 0.48.5", + "windows_aarch64_msvc 0.48.5", + "windows_i686_gnu 0.48.5", + "windows_i686_msvc 0.48.5", + "windows_x86_64_gnu 0.48.5", + "windows_x86_64_gnullvm 0.48.5", + "windows_x86_64_msvc 0.48.5", +] + +[[package]] +name = "windows-targets" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a18201040b24831fbb9e4eb208f8892e1f50a37feb53cc7ff887feb8f50e7cd" +dependencies = [ + "windows_aarch64_gnullvm 0.52.0", + "windows_aarch64_msvc 0.52.0", + "windows_i686_gnu 0.52.0", + "windows_i686_msvc 0.52.0", + "windows_x86_64_gnu 0.52.0", + "windows_x86_64_gnullvm 0.52.0", + "windows_x86_64_msvc 0.52.0", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7764e35d4db8a7921e09562a0304bf2f93e0a51bfccee0bd0bb0b666b015ea" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbaa0368d4f1d2aaefc55b6fcfee13f41544ddf36801e793edbbfd7d7df075ef" + +[[package]] +name = "windows_i686_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" + +[[package]] +name = "windows_i686_gnu" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a28637cb1fa3560a16915793afb20081aba2c92ee8af57b4d5f28e4b3e7df313" + +[[package]] +name = "windows_i686_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" + +[[package]] +name = "windows_i686_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffe5e8e31046ce6230cc7215707b816e339ff4d4d67c65dffa206fd0f7aa7b9a" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d6fa32db2bc4a2f5abeacf2b69f7992cd09dca97498da74a151a3132c26befd" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a657e1e9d3f514745a572a6846d3c7aa7dbe1658c056ed9c3344c4109a6949e" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dff9641d1cd4be8d1a070daf9e3773c5f67e78b4d9d42263020c057706765c04" + +[[package]] +name = "yaml-rust" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56c1936c4cc7a1c9ab21a1ebb602eb942ba868cbd44a99cb7cdc5892335e1c85" +dependencies = [ + "linked-hash-map", +] + +[[package]] +name = "zerocopy" +version = "0.7.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + +[[package]] +name = "zstd" +version = "0.11.2+zstd.1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" +dependencies = [ + "zstd-safe", +] + +[[package]] +name = "zstd-safe" +version = "5.0.2+zstd.1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d2a5585e04f9eea4b2a3d1eca508c4dee9592a89ef6f450c11719da0726f4db" +dependencies = [ + "libc", + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.9+zstd.1.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e16efa8a874a0481a574084d34cc26fdb3b99627480f785888deb6386506656" +dependencies = [ + "cc", + "pkg-config", +] diff --git a/core/Cargo.toml b/core/Cargo.toml new file mode 100644 index 0000000000..adc3732e33 --- /dev/null +++ b/core/Cargo.toml @@ -0,0 +1,115 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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] +name = "comet" +version = "0.1.0" +edition = "2021" +include = [ + "benches/*.rs", + "src/**/*.rs", + "Cargo.toml", +] + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +parquet-format = "4.0.0" # This must be kept in sync with that from parquet crate +arrow = { version = "~49.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "~49.0.0" } +arrow-data = { version = "~49.0.0" } +arrow-schema = { version = "~49.0.0" } +arrow-string = { version = "~49.0.0" } +parquet = { version = "~49.0.0", default-features = false, features = ["experimental"] } +half = { version = "~2.1", default-features = false } +futures = "0.3.28" +mimalloc = { version = "*", default-features = false, optional = true } +tokio = { version = "1", features = ["rt-multi-thread"] } +tokio-stream = { version = "0.1.8" } +async-trait = "0.1" +log = "0.4" +log4rs = "1.2.0" +thiserror = "1" +serde = { version = "1", features = ["derive"] } +lazy_static = "1.4.0" +prost = "0.12.1" +thrift = "0.17" +jni = "0.19" +byteorder = "1.4.3" +snap = "1.1" +brotli = "3.3" +flate2 = "1.0" +lz4 = "1.24" +zstd = "0.11" +rand = "0.8" +num = "0.4" +bytes = "1.5.0" +hashbrown = { version = "0.14", features = ["raw"] } +parking_lot = "0.12" +tempfile = "3.8.0" +ahash = { version = "0.8", default-features = false } +itertools = "0.11.0" +chrono = { version = "0.4", default-features = false, features = ["clock"] } +chrono-tz = { version = "0.8" } +paste = "1.0.14" +datafusion-common = { version = "34.0.0" } +datafusion = { default-features = false, version = "34.0.0", features = ["unicode_expressions"] } +datafusion-physical-expr = { version = "34.0.0", default-features = false , features = ["unicode_expressions"] } +unicode-segmentation = "^1.10.1" +once_cell = "1.18.0" +regex = "1.9.6" +crc32fast = "1.3.2" +simd-adler32 = "0.3.7" + +[build-dependencies] +prost-build = "0.9.0" + +[dev-dependencies] +pprof = { version = "0.13.0", features = ["flamegraph"] } +criterion = "0.5.1" +jni = { version = "0.19", features = ["invocation"] } +lazy_static = "1.4" +assertables = "7" + +[features] +default = [] +nightly = [] + +[profile.release] +debug = true +overflow-checks = false +lto = "thin" +codegen-units = 1 +strip = "debuginfo" + +[lib] +name = "comet" +# "rlib" is for benchmarking with criterion. +crate_type = ["cdylib", "rlib"] + +[[bench]] +name = "parquet_read" +harness = false + +[[bench]] +name = "bit_util" +harness = false + +[[bench]] +name = "hash" +harness = false + diff --git a/core/benches/bit_util.rs b/core/benches/bit_util.rs new file mode 100644 index 0000000000..e92dd6375c --- /dev/null +++ b/core/benches/bit_util.rs @@ -0,0 +1,182 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{mem::size_of, time::Duration}; + +use rand::{thread_rng, Rng}; + +use arrow::buffer::Buffer; +use comet::common::bit::{ + log2, read_num_bytes_u32, read_num_bytes_u64, set_bits, BitReader, BitWriter, +}; +use criterion::{black_box, criterion_group, criterion_main, BenchmarkId, Criterion}; + +/// Benchmark to measure bit_util performance. +/// To run this benchmark: +/// `cd core && cargo bench --bench bit_util` +/// Results will be written to "core/target/criterion/bit_util/" +fn criterion_benchmark(c: &mut Criterion) { + let mut group = c.benchmark_group("bit_util"); + + const N: usize = 1024 * 1024; + let mut writer: BitWriter = BitWriter::new(N * 10); + for _ in 0..N { + if !writer.put_vlq_int(thread_rng().gen::<u64>()) { + break; + } + } + let buffer = writer.consume(); + let buffer = Buffer::from(&buffer); + + // log2 + for bits in (0..64).step_by(3) { + let x = 1u64 << bits; + group.bench_with_input(BenchmarkId::new("log2", bits), &x, |b, &x| { + b.iter(|| log2(black_box(x))); + }); + } + + // set_bits + for offset in (0..16).step_by(3) { + for length in (0..16).step_by(3) { + let x = (offset, length); + group.bench_with_input( + BenchmarkId::new("set_bits", format!("offset_{}_length_{}", x.0, x.1)), + &x, + |b, &x| { + b.iter(|| set_bits(&mut [0u8; 4], black_box(x.0), black_box(x.1))); + }, + ); + } + } + + // get_vlq_int + group.bench_function("get_vlq_int", |b| { + b.iter(|| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + bench_get_vlq_int(&mut reader) + }) + }); + + // get_bits + for offset in (0..32).step_by(17) { + for num_bits in (1..5).step_by(1) { + let x = (offset, num_bits); + group.bench_with_input( + BenchmarkId::new("get_bits", format!("offset_{}_num_bits_{}", x.0, x.1)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_bits(&mut [0u8; 4], black_box(x.0), black_box(x.1))); + }, + ); + } + } + + // get_aligned + for num_bytes in (1..=size_of::<u8>()).step_by(3) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("get_aligned", format!("u8_num_bytes_{}", x)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_aligned::<u8>(black_box(x))); + }, + ); + } + for num_bytes in (1..=size_of::<u32>()).step_by(3) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("get_aligned", format!("u32_num_bytes_{}", x)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_aligned::<u32>(black_box(x))); + }, + ); + } + for num_bytes in (1..=size_of::<i32>()).step_by(3) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("get_aligned", format!("i32_num_bytes_{}", x)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_aligned::<i32>(black_box(x))); + }, + ); + } + + // get_value + for num_bytes in (1..=size_of::<i32>()).step_by(3) { + let x = num_bytes * 8; + group.bench_with_input( + BenchmarkId::new("get_value", format!("i32_num_bits_{}", x)), + &x, + |b, &x| { + let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); + b.iter(|| reader.get_value::<i32>(black_box(x))); + }, + ); + } + + // read_num_bytes_u64 + for num_bytes in (1..=8).step_by(7) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("read_num_bytes_u64", format!("num_bytes_{}", x)), + &x, + |b, &x| { + b.iter(|| read_num_bytes_u64(black_box(x), black_box(buffer.as_slice()))); + }, + ); + } + + // read_num_bytes_u32 + for num_bytes in (1..=4).step_by(3) { + let x = num_bytes; + group.bench_with_input( + BenchmarkId::new("read_num_bytes_u32", format!("num_bytes_{}", x)), + &x, + |b, &x| { + b.iter(|| read_num_bytes_u32(black_box(x), black_box(buffer.as_slice()))); + }, + ); + } + + group.finish(); +} + +fn bench_get_vlq_int(reader: &mut BitReader) { + while let Some(v) = reader.get_vlq_int() { + black_box(v); + } +} + +fn config() -> Criterion { + Criterion::default() + .measurement_time(Duration::from_millis(500)) + .warm_up_time(Duration::from_millis(500)) +} + +criterion_group! { + name = benches; + config = config(); + targets = criterion_benchmark +} +criterion_main!(benches); diff --git a/core/benches/common.rs b/core/benches/common.rs new file mode 100644 index 0000000000..059721698a --- /dev/null +++ b/core/benches/common.rs @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{DictionaryArray, Int64Array, PrimitiveArray}, + datatypes::{ArrowPrimitiveType, Int32Type}, +}; +use arrow_schema::ArrowError; +use rand::{ + distributions::{Distribution, Standard}, + rngs::StdRng, + Rng, SeedableRng, +}; +use std::sync::Arc; + +/// Returns fixed seedable RNG +pub fn seedable_rng() -> StdRng { + StdRng::seed_from_u64(42) +} + +pub fn create_int64_array(size: usize, null_density: f32, min: i64, max: i64) -> Int64Array { + let mut rng = seedable_rng(); + (0..size) + .map(|_| { + if rng.gen::<f32>() < null_density { + None + } else { + Some(rng.gen_range(min..max)) + } + }) + .collect() +} + +pub fn create_primitive_array<T>(size: usize, null_density: f32) -> PrimitiveArray<T> +where + T: ArrowPrimitiveType, + Standard: Distribution<T::Native>, +{ + let mut rng = seedable_rng(); + (0..size) + .map(|_| { + if rng.gen::<f32>() < null_density { + None + } else { + Some(rng.gen()) + } + }) + .collect() +} + +/// Creates a dictionary with random keys and values, with value type `T`. +/// Note here the keys are the dictionary indices. +pub fn create_dictionary_array<T>( + size: usize, + value_size: usize, + null_density: f32, +) -> Result<DictionaryArray<Int32Type>, ArrowError> +where + T: ArrowPrimitiveType, + Standard: Distribution<T::Native>, +{ + // values are not null + let values = create_primitive_array::<T>(value_size, 0.0); + let keys = create_primitive_array::<Int32Type>(size, null_density) + .iter() + .map(|v| v.map(|w| w.abs() % (value_size as i32))) + .collect(); + DictionaryArray::try_new(keys, Arc::new(values)) +} diff --git a/core/benches/hash.rs b/core/benches/hash.rs new file mode 100644 index 0000000000..dafad79dd3 --- /dev/null +++ b/core/benches/hash.rs @@ -0,0 +1,109 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#[path = "common.rs"] +mod common; + +use arrow_array::ArrayRef; +use comet::execution::kernels::hash; +use common::*; +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +use std::sync::Arc; + +const BATCH_SIZE: usize = 1024 * 8; +const NUM_ITER: usize = 10; +const NULL_FRACTION: f32 = 0.1; + +fn criterion_benchmark(c: &mut Criterion) { + let mut group = c.benchmark_group("hash"); + + let a1: ArrayRef = Arc::new(create_int64_array(BATCH_SIZE, 0.0, 0, BATCH_SIZE as i64)); + let a2: ArrayRef = Arc::new(create_int64_array(BATCH_SIZE, 0.0, 0, BATCH_SIZE as i64)); + let a3: ArrayRef = Arc::new(create_int64_array( + BATCH_SIZE, + NULL_FRACTION, + 0, + BATCH_SIZE as i64, + )); + let a4: ArrayRef = Arc::new(create_int64_array( + BATCH_SIZE, + NULL_FRACTION, + 0, + BATCH_SIZE as i64, + )); + + group.bench_function( + BenchmarkId::new("hash_i64_single_nonnull", BATCH_SIZE), + |b| { + let input = vec![a1.clone()]; + let mut dst = vec![0; BATCH_SIZE]; + + b.iter(|| { + for _ in 0..NUM_ITER { + hash(&input, &mut dst); + } + }); + }, + ); + group.bench_function(BenchmarkId::new("hash_i64_single_null", BATCH_SIZE), |b| { + let input = vec![a3.clone()]; + let mut dst = vec![0; BATCH_SIZE]; + + b.iter(|| { + for _ in 0..NUM_ITER { + hash(&input, &mut dst); + } + }); + }); + group.bench_function( + BenchmarkId::new("hash_i64_multiple_nonnull", BATCH_SIZE), + |b| { + let input = vec![a1.clone(), a2.clone()]; + let mut dst = vec![0; BATCH_SIZE]; + + b.iter(|| { + for _ in 0..NUM_ITER { + hash(&input, &mut dst); + } + }); + }, + ); + group.bench_function( + BenchmarkId::new("hash_i64_multiple_null", BATCH_SIZE), + |b| { + let input = vec![a3.clone(), a4.clone()]; + let mut dst = vec![0; BATCH_SIZE]; + + b.iter(|| { + for _ in 0..NUM_ITER { + hash(&input, &mut dst); + } + }); + }, + ); +} + +fn config() -> Criterion { + Criterion::default() +} + +criterion_group! { + name = benches; + config = config(); + targets = criterion_benchmark +} +criterion_main!(benches); diff --git a/core/benches/parquet_read.rs b/core/benches/parquet_read.rs new file mode 100644 index 0000000000..7dcfab7a4d --- /dev/null +++ b/core/benches/parquet_read.rs @@ -0,0 +1,217 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +mod perf; + +use std::sync::Arc; + +use arrow::{array::ArrayData, buffer::Buffer}; +use comet::parquet::{read::ColumnReader, util::jni::TypePromotionInfo}; +use criterion::{criterion_group, criterion_main, Criterion}; +use parquet::{ + basic::{Encoding, Type as PhysicalType}, + column::page::{PageIterator, PageReader}, + data_type::Int32Type, + schema::types::{ + ColumnDescPtr, ColumnDescriptor, ColumnPath, PrimitiveTypeBuilder, SchemaDescPtr, TypePtr, + }, +}; + +use comet::parquet::util::test_common::page_util::{ + DataPageBuilder, DataPageBuilderImpl, InMemoryPageIterator, +}; + +use perf::FlamegraphProfiler; +use rand::{prelude::StdRng, Rng, SeedableRng}; + +fn bench(c: &mut Criterion) { + let expected_num_values: usize = NUM_PAGES * VALUES_PER_PAGE; + let mut group = c.benchmark_group("comet_parquet_read"); + let schema = build_test_schema(); + + let pages = build_plain_int32_pages(schema.clone(), schema.column(0), 0.0); + group.bench_function("INT/PLAIN/NOT_NULL", |b| { + let t = TypePtr::new( + PrimitiveTypeBuilder::new("f", PhysicalType::INT32) + .with_length(4) + .build() + .unwrap(), + ); + b.iter(|| { + let cd = ColumnDescriptor::new(t.clone(), 0, 0, ColumnPath::from(Vec::new())); + let promition_info = TypePromotionInfo::new(PhysicalType::INT32, -1); + let mut column_reader = TestColumnReader::new( + cd, + promition_info, + BATCH_SIZE, + pages.clone(), + expected_num_values, + ); + + let mut total = 0; + for batch in column_reader.by_ref() { + total += batch.len(); + ::std::mem::forget(batch); + } + assert_eq!(total, expected_num_values); + }); + }); +} + +fn profiled() -> Criterion { + Criterion::default().with_profiler(FlamegraphProfiler::new(100)) +} + +criterion_group! { + name = benches; + config = profiled(); + targets = bench +} +criterion_main!(benches); + +fn build_test_schema() -> SchemaDescPtr { + use parquet::schema::{parser::parse_message_type, types::SchemaDescriptor}; + let message_type = " + message test_schema { + REQUIRED INT32 c1; + OPTIONAL INT32 c2; + } + "; + parse_message_type(message_type) + .map(|t| Arc::new(SchemaDescriptor::new(Arc::new(t)))) + .unwrap() +} + +fn seedable_rng() -> StdRng { + StdRng::seed_from_u64(42) +} + +// test data params +const NUM_PAGES: usize = 1000; +const VALUES_PER_PAGE: usize = 10_000; +const BATCH_SIZE: usize = 4096; + +fn build_plain_int32_pages( + schema: SchemaDescPtr, + column_desc: ColumnDescPtr, + null_density: f32, +) -> impl PageIterator + Clone { + let max_def_level = column_desc.max_def_level(); + let max_rep_level = column_desc.max_rep_level(); + let rep_levels = vec![0; VALUES_PER_PAGE]; + let mut rng = seedable_rng(); + let mut pages: Vec<parquet::column::page::Page> = Vec::new(); + let mut int32_value = 0; + for _ in 0..NUM_PAGES { + // generate page + let mut values = Vec::with_capacity(VALUES_PER_PAGE); + let mut def_levels = Vec::with_capacity(VALUES_PER_PAGE); + for _ in 0..VALUES_PER_PAGE { + let def_level = if rng.gen::<f32>() < null_density { + max_def_level - 1 + } else { + max_def_level + }; + if def_level == max_def_level { + int32_value += 1; + values.push(int32_value); + } + def_levels.push(def_level); + } + let mut page_builder = + DataPageBuilderImpl::new(column_desc.clone(), values.len() as u32, true); + page_builder.add_rep_levels(max_rep_level, &rep_levels); + page_builder.add_def_levels(max_def_level, &def_levels); + page_builder.add_values::<Int32Type>(Encoding::PLAIN, &values); + pages.push(page_builder.consume()); + } + + // Since `InMemoryPageReader` is not exposed from parquet crate, here we use + // `InMemoryPageIterator` instead which is a Iter<Iter<Page>>. + InMemoryPageIterator::new(schema, column_desc, vec![pages]) +} + +struct TestColumnReader { + inner: ColumnReader, + pages: Box<dyn PageReader>, + batch_size: usize, + total_num_values: usize, + total_num_values_read: usize, + first_page_loaded: bool, +} + +impl TestColumnReader { + pub fn new( + cd: ColumnDescriptor, + promotion_info: TypePromotionInfo, + batch_size: usize, + mut page_iter: impl PageIterator + 'static, + total_num_values: usize, + ) -> Self { + let reader = ColumnReader::get(cd, promotion_info, batch_size, false, false); + let first = page_iter.next().unwrap().unwrap(); + Self { + inner: reader, + pages: first, + batch_size, + total_num_values, + total_num_values_read: 0, + first_page_loaded: false, + } + } + + fn load_page(&mut self) { + if let Some(page) = self.pages.get_next_page().unwrap() { + let num_values = page.num_values() as usize; + let buffer = Buffer::from_slice_ref(page.buffer().data()); + self.inner.set_page_v1(num_values, buffer, page.encoding()); + } + } +} + +impl Iterator for TestColumnReader { + type Item = ArrayData; + + fn next(&mut self) -> Option<Self::Item> { + if self.total_num_values_read >= self.total_num_values { + return None; + } + + if !self.first_page_loaded { + self.load_page(); + self.first_page_loaded = true; + } + + self.inner.reset_batch(); + let total = ::std::cmp::min( + self.batch_size, + self.total_num_values - self.total_num_values_read, + ); + + let mut left = total; + while left > 0 { + let (num_read, _) = self.inner.read_batch(left, 0); + if num_read < left { + self.load_page(); + } + left -= num_read; + } + self.total_num_values_read += total; + + Some(self.inner.current_batch()) + } +} diff --git a/core/benches/perf.rs b/core/benches/perf.rs new file mode 100644 index 0000000000..f92ec02508 --- /dev/null +++ b/core/benches/perf.rs @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{fs::File, os::raw::c_int, path::Path}; + +use criterion::profiler::Profiler; +use pprof::ProfilerGuard; + +/// A custom profiler for criterion which generates flamegraph. +/// +/// Mostly followed this blog post: https://www.jibbow.com/posts/criterion-flamegraphs/ +/// After `cargo bench --bench <bench-name> -- --profile-time=<time>` +/// You can find flamegraph.svg under `target/criterion/<bench-name>/<bench-method-name>/profile` +pub struct FlamegraphProfiler<'a> { + frequency: c_int, + active_profiler: Option<ProfilerGuard<'a>>, +} + +impl<'a> FlamegraphProfiler<'a> { + pub fn new(frequency: c_int) -> Self { + FlamegraphProfiler { + frequency, + active_profiler: None, + } + } +} + +impl<'a> Profiler for FlamegraphProfiler<'a> { + fn start_profiling(&mut self, _benchmark_id: &str, _benchmark_dir: &Path) { + self.active_profiler = Some(ProfilerGuard::new(self.frequency).unwrap()); + } + + fn stop_profiling(&mut self, _benchmark_id: &str, benchmark_dir: &Path) { + std::fs::create_dir_all(benchmark_dir).unwrap(); + let flamegraph_path = benchmark_dir.join("flamegraph.svg"); + let flamegraph_file = + File::create(flamegraph_path).expect("File system error while creating flamegraph.svg"); + if let Some(profiler) = self.active_profiler.take() { + profiler + .report() + .build() + .unwrap() + .flamegraph(flamegraph_file) + .expect("Error writing flamegraph"); + } + } +} diff --git a/core/build.rs b/core/build.rs new file mode 100644 index 0000000000..4322bbbc95 --- /dev/null +++ b/core/build.rs @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Build script for generating codes from .proto files. + +use std::{fs, io::Result, path::Path}; + +fn main() -> Result<()> { + println!("cargo:rerun-if-changed=src/execution/proto/*.proto"); + + let out_dir = "src/execution/generated"; + if !Path::new(out_dir).is_dir() { + fs::create_dir(out_dir)?; + } + + prost_build::Config::new().out_dir(out_dir).compile_protos( + &[ + "src/execution/proto/expr.proto", + "src/execution/proto/partitioning.proto", + "src/execution/proto/operator.proto", + ], + &["src/execution/proto"], + )?; + Ok(()) +} diff --git a/core/rustfmt.toml b/core/rustfmt.toml new file mode 100644 index 0000000000..39a3fe635a --- /dev/null +++ b/core/rustfmt.toml @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +edition = "2021" +max_width = 100 + +# The following requires nightly feature: +# rustup install nightly +# rustup component add rustfmt --toolchain nightly +# cargo +nightly fmt +wrap_comments = true +comment_width = 100 +imports_granularity = "Crate" # group imports by crate diff --git a/core/src/common/bit.rs b/core/src/common/bit.rs new file mode 100644 index 0000000000..4af560ffc8 --- /dev/null +++ b/core/src/common/bit.rs @@ -0,0 +1,1648 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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, cmp::min, mem::size_of}; + +use arrow::buffer::Buffer; + +use crate::{ + errors::CometResult as Result, + likely, + parquet::{data_type::AsBytes, util::bit_packing::unpack32}, + unlikely, +}; + +#[inline] +pub fn from_ne_slice<T: FromBytes>(bs: &[u8]) -> T { + let mut b = T::Buffer::default(); + { + let b = b.as_mut(); + let bs = &bs[..b.len()]; + b.copy_from_slice(bs); + } + T::from_ne_bytes(b) +} + +pub trait FromBytes: Sized { + type Buffer: AsMut<[u8]> + Default; + fn from_le_bytes(bs: Self::Buffer) -> Self; + fn from_be_bytes(bs: Self::Buffer) -> Self; + fn from_ne_bytes(bs: Self::Buffer) -> Self; + fn from(v: u64) -> Self; +} + +macro_rules! from_le_bytes { + ($($ty: ty),*) => { + $( + impl FromBytes for $ty { + type Buffer = [u8; size_of::<Self>()]; + fn from_le_bytes(bs: Self::Buffer) -> Self { + <$ty>::from_le_bytes(bs) + } + fn from_be_bytes(bs: Self::Buffer) -> Self { + <$ty>::from_be_bytes(bs) + } + fn from_ne_bytes(bs: Self::Buffer) -> Self { + <$ty>::from_ne_bytes(bs) + } + fn from(v: u64) -> Self { + v as $ty + } + } + )* + }; +} + +impl FromBytes for bool { + type Buffer = [u8; 1]; + fn from_le_bytes(bs: Self::Buffer) -> Self { + Self::from_ne_bytes(bs) + } + fn from_be_bytes(bs: Self::Buffer) -> Self { + Self::from_ne_bytes(bs) + } + fn from_ne_bytes(bs: Self::Buffer) -> Self { + match bs[0] { + 0 => false, + 1 => true, + _ => panic!("Invalid byte when reading bool"), + } + } + fn from(v: u64) -> Self { + (v & 1) == 1 + } +} + +// TODO: support f32 and f64 in the future, but there is no use case right now +// f32/f64::from(v: u64) will be like `from_ne_slice(v.as_bytes()))` and that is +// expensive as it involves copying buffers +from_le_bytes! { u8, u16, u32, u64, i8, i16, i32, i64 } + +/// Reads `$size` of bytes from `$src`, and reinterprets them as type `$ty`, in +/// little-endian order. `$ty` must implement the `Default` trait. Otherwise this won't +/// compile. +/// This is copied and modified from byteorder crate. +macro_rules! read_num_bytes { + ($ty:ty, $size:expr, $src:expr) => {{ + debug_assert!($size <= $src.len()); + let mut buffer = <$ty as $crate::common::bit::FromBytes>::Buffer::default(); + buffer.as_mut()[..$size].copy_from_slice(&$src[..$size]); + <$ty>::from_ne_bytes(buffer) + }}; +} + +/// u64 specific version of read_num_bytes! +/// This is faster than read_num_bytes! because this method avoids buffer copies. +#[inline] +pub fn read_num_bytes_u64(size: usize, src: &[u8]) -> u64 { + debug_assert!(size <= src.len()); + if unlikely(src.len() < 8) { + return read_num_bytes!(u64, size, src); + } + let in_ptr = src as *const [u8] as *const u8 as *const u64; + let v = unsafe { in_ptr.read_unaligned() }; + trailing_bits(v, size * 8) +} + +/// u32 specific version of read_num_bytes! +/// This is faster than read_num_bytes! because this method avoids buffer copies. +#[inline] +pub fn read_num_bytes_u32(size: usize, src: &[u8]) -> u32 { + debug_assert!(size <= src.len()); + if unlikely(src.len() < 4) { + return read_num_bytes!(u32, size, src); + } + let in_ptr = src as *const [u8] as *const u8 as *const u32; + let v = unsafe { in_ptr.read_unaligned() }; + trailing_bits(v as u64, size * 8) as u32 +} + +/// Converts value `val` of type `T` to a byte vector, by reading `num_bytes` from `val`. +/// NOTE: if `val` is less than the size of `T` then it can be truncated. +#[inline] +pub fn convert_to_bytes<T>(val: &T, num_bytes: usize) -> Vec<u8> +where + T: ?Sized + AsBytes, +{ + let mut bytes: Vec<u8> = vec![0; num_bytes]; + memcpy_value(val.as_bytes(), num_bytes, &mut bytes); + bytes +} + +#[inline] +pub fn memcpy(source: &[u8], target: &mut [u8]) { + debug_assert!(target.len() >= source.len(), "Copying from source to target is not possible. Source has {} bytes but target has {} bytes", source.len(), target.len()); + target[..source.len()].copy_from_slice(source) +} + +#[inline] +pub fn memcpy_value<T>(source: &T, num_bytes: usize, target: &mut [u8]) +where + T: ?Sized + AsBytes, +{ + debug_assert!( + target.len() >= num_bytes, + "Not enough space. Only had {} bytes but need to put {} bytes", + target.len(), + num_bytes + ); + memcpy(&source.as_bytes()[..num_bytes], target) +} + +/// Returns the ceil of value/divisor +#[inline] +pub fn ceil(value: usize, divisor: usize) -> usize { + value / divisor + ((value % divisor != 0) as usize) +} + +/// Returns ceil(log2(x)) +#[inline] +pub fn log2(mut x: u64) -> u32 { + if x == 1 { + return 0; + } + x -= 1; + 64u32 - x.leading_zeros() +} + +/// Returns the `num_bits` least-significant bits of `v` +#[inline] +pub fn trailing_bits(v: u64, num_bits: usize) -> u64 { + if unlikely(num_bits == 0) { + return 0; + } + if unlikely(num_bits >= 64) { + return v; + } + let n = 64 - num_bits; + (v << n) >> n +} + +pub fn set_bit_value(bits: &mut [u8], i: usize, val: bool) { + bits[i / 8] |= (val as u8) << (i % 8); +} + +#[inline] +pub fn set_bit(bits: &mut [u8], i: usize) { + bits[i / 8] |= 1 << (i % 8); +} + +/// Set the bit value at index `i`, for buffer `bits`. +/// +/// # Safety +/// This doesn't check bounds, the caller must ensure that `i` is in (0, bits.len() * 8) +#[inline] +pub unsafe fn set_bit_raw(bits: *mut u8, i: usize) { + *bits.add(i / 8) |= 1 << (i % 8); +} + +#[inline] +pub fn unset_bit(bits: &mut [u8], i: usize) { + bits[i / 8] &= !(1 << (i % 8)); +} + +#[inline] +pub fn set_bits(bits: &mut [u8], offset: usize, length: usize) { + let mut byte_i = offset / 8; + let offset_r = offset % 8; + let end = offset + length; + let end_byte_i = end / 8; + let end_r = end % 8; + + // if the offset starts in the middle of a byte, update the byte first + if offset_r != 0 { + let num_bits = min(length, 7); + bits[byte_i] |= ((1u8 << num_bits) - 1) << offset_r; + byte_i += 1; + } + + // See if there is an opportunity to do a bulk byte write + if byte_i < end_byte_i { + unsafe { + bits.as_mut_ptr() + .add(byte_i) + .write_bytes(255, end_byte_i - byte_i); + } + byte_i = end_byte_i; + } + + // take care of the last byte + if end_r > 0 && (byte_i == end_byte_i) { + bits[byte_i] |= (1u8 << end_r) - 1; + } +} + +/// Returns the minimum number of bits needed to represent the value 'x' +#[inline] +pub fn num_required_bits(x: u64) -> usize { + for i in (0..64).rev() { + if x & (1u64 << i) != 0 { + return i + 1; + } + } + 0 +} + +#[inline(always)] +pub fn mix_hash(lower: u64, upper: u64) -> u64 { + let hash = (17 * 37u64).wrapping_add(lower); + hash.wrapping_mul(37).wrapping_add(upper) +} + +static BIT_MASK: [u8; 8] = [1, 2, 4, 8, 16, 32, 64, 128]; + +/// Returns whether bit at position `i` in `data` is set or not +#[inline] +pub fn get_bit(data: &[u8], i: usize) -> bool { + (data[i >> 3] & BIT_MASK[i & 7]) != 0 +} + +/// Returns the boolean value at index `i`. +/// +/// # Safety +/// This doesn't check bounds, the caller must ensure that index < self.len() +#[inline] +pub unsafe fn get_bit_raw(ptr: *const u8, i: usize) -> bool { + (*ptr.add(i >> 3) & BIT_MASK[i & 7]) != 0 +} + +/// Utility class for writing bit/byte streams. This class can write data in either +/// bit packed or byte aligned fashion. +pub struct BitWriter { + buffer: Vec<u8>, + max_bytes: usize, + buffered_values: u64, + byte_offset: usize, + bit_offset: usize, + start: usize, +} + +impl BitWriter { + pub fn new(max_bytes: usize) -> Self { + Self { + buffer: vec![0; max_bytes], + max_bytes, + buffered_values: 0, + byte_offset: 0, + bit_offset: 0, + start: 0, + } + } + + /// Initializes the writer from the existing buffer `buffer` and starting + /// offset `start`. + pub fn new_from_buf(buffer: Vec<u8>, start: usize) -> Self { + debug_assert!(start < buffer.len()); + let len = buffer.len(); + Self { + buffer, + max_bytes: len, + buffered_values: 0, + byte_offset: start, + bit_offset: 0, + start, + } + } + + /// Extend buffer size by `increment` bytes + #[inline] + pub fn extend(&mut self, increment: usize) { + self.max_bytes += increment; + let extra = vec![0; increment]; + self.buffer.extend(extra); + } + + /// Report buffer size, in bytes + #[inline] + pub fn capacity(&mut self) -> usize { + self.max_bytes + } + + /// Consumes and returns the current buffer. + #[inline] + pub fn consume(mut self) -> Vec<u8> { + self.flush(); + self.buffer.truncate(self.byte_offset); + self.buffer + } + + /// Flushes the internal buffered bits and returns the buffer's content. + /// This is a borrow equivalent of `consume` method. + #[inline] + pub fn flush_buffer(&mut self) -> &[u8] { + self.flush(); + &self.buffer()[0..self.byte_offset] + } + + /// Clears the internal state so the buffer can be reused. + #[inline] + pub fn clear(&mut self) { + self.buffered_values = 0; + self.byte_offset = self.start; + self.bit_offset = 0; + } + + /// Flushes the internal buffered bits and the align the buffer to the next byte. + #[inline] + pub fn flush(&mut self) { + let num_bytes = ceil(self.bit_offset, 8); + debug_assert!(self.byte_offset + num_bytes <= self.max_bytes); + memcpy_value( + &self.buffered_values, + num_bytes, + &mut self.buffer[self.byte_offset..], + ); + self.buffered_values = 0; + self.bit_offset = 0; + self.byte_offset += num_bytes; + } + + /// Advances the current offset by skipping `num_bytes`, flushing the internal bit + /// buffer first. + /// This is useful when you want to jump over `num_bytes` bytes and come back later + /// to fill these bytes. + /// + /// Returns error if `num_bytes` is beyond the boundary of the internal buffer. + /// Otherwise, returns the old offset. + #[inline] + pub fn skip(&mut self, num_bytes: usize) -> Result<usize> { + self.flush(); + debug_assert!(self.byte_offset <= self.max_bytes); + if unlikely(self.byte_offset + num_bytes > self.max_bytes) { + return Err(general_err!( + "Not enough bytes left in BitWriter. Need {} but only have {}", + self.byte_offset + num_bytes, + self.max_bytes + )); + } + let result = self.byte_offset; + self.byte_offset += num_bytes; + Ok(result) + } + + /// Returns a slice containing the next `num_bytes` bytes starting from the current + /// offset, and advances the underlying buffer by `num_bytes`. + /// This is useful when you want to jump over `num_bytes` bytes and come back later + /// to fill these bytes. + #[inline] + pub fn get_next_byte_ptr(&mut self, num_bytes: usize) -> Result<&mut [u8]> { + let offset = self.skip(num_bytes)?; + Ok(&mut self.buffer[offset..offset + num_bytes]) + } + + #[inline] + pub fn bytes_written(&self) -> usize { + self.byte_offset - self.start + ceil(self.bit_offset, 8) + } + + #[inline] + pub fn buffer(&self) -> &[u8] { + &self.buffer[self.start..] + } + + #[inline] + pub fn byte_offset(&self) -> usize { + self.byte_offset + } + + /// Returns the internal buffer length. This is the maximum number of bytes that this + /// writer can write. User needs to call `consume` to consume the current buffer + /// before more data can be written. + #[inline] + pub fn buffer_len(&self) -> usize { + self.max_bytes + } + + /// Writes the entire byte `value` at the byte `offset` + pub fn write_at(&mut self, offset: usize, value: u8) { + self.buffer[offset] = value; + } + + /// Writes the `num_bits` LSB of value `v` to the internal buffer of this writer. + /// The `num_bits` must not be greater than 64. This is bit packed. + /// + /// Returns false if there's not enough room left. True otherwise. + #[inline] + #[allow(clippy::unnecessary_cast)] + pub fn put_value(&mut self, v: u64, num_bits: usize) -> bool { + debug_assert!(num_bits <= 64); + debug_assert_eq!(v.checked_shr(num_bits as u32).unwrap_or(0), 0); // covers case v >> 64 + + let num_bytes = self.byte_offset * 8 + self.bit_offset + num_bits; + if unlikely(num_bytes > self.max_bytes as usize * 8) { + return false; + } + + self.buffered_values |= v << self.bit_offset; + self.bit_offset += num_bits; + if self.bit_offset >= 64 { + memcpy_value( + &self.buffered_values, + 8, + &mut self.buffer[self.byte_offset..], + ); + self.byte_offset += 8; + self.bit_offset -= 64; + self.buffered_values = 0; + // Perform checked right shift: v >> offset, where offset < 64, otherwise we + // shift all bits + self.buffered_values = v + .checked_shr((num_bits - self.bit_offset) as u32) + .unwrap_or(0); + } + debug_assert!(self.bit_offset < 64); + true + } + + /// Writes `val` of `num_bytes` bytes to the next aligned byte. If size of `T` is + /// larger than `num_bytes`, extra higher ordered bytes will be ignored. + /// + /// Returns false if there's not enough room left. True otherwise. + #[inline] + pub fn put_aligned<T: AsBytes>(&mut self, val: T, num_bytes: usize) -> bool { + let result = self.get_next_byte_ptr(num_bytes); + if unlikely(result.is_err()) { + // TODO: should we return `Result` for this func? + return false; + } + let ptr = result.unwrap(); + memcpy_value(&val, num_bytes, ptr); + true + } + + /// Writes `val` of `num_bytes` bytes at the designated `offset`. The `offset` is the + /// offset starting from the beginning of the internal buffer that this writer + /// maintains. Note that this will overwrite any existing data between `offset` and + /// `offset + num_bytes`. Also that if size of `T` is larger than `num_bytes`, extra + /// higher ordered bytes will be ignored. + /// + /// Returns false if there's not enough room left, or the `pos` is not valid. + /// True otherwise. + #[inline] + pub fn put_aligned_offset<T: AsBytes>( + &mut self, + val: T, + num_bytes: usize, + offset: usize, + ) -> bool { + if unlikely(num_bytes + offset > self.max_bytes) { + return false; + } + memcpy_value( + &val, + num_bytes, + &mut self.buffer[offset..offset + num_bytes], + ); + true + } + + /// Writes a VLQ encoded integer `v` to this buffer. The value is byte aligned. + /// + /// Returns false if there's not enough room left. True otherwise. + #[inline] + pub fn put_vlq_int(&mut self, mut v: u64) -> bool { + let mut result = true; + while v & 0xFFFFFFFFFFFFFF80 != 0 { + result &= self.put_aligned::<u8>(((v & 0x7F) | 0x80) as u8, 1); + v >>= 7; + } + result &= self.put_aligned::<u8>((v & 0x7F) as u8, 1); + result + } + + /// Writes a zigzag-VLQ encoded (in little endian order) int `v` to this buffer. + /// Zigzag-VLQ is a variant of VLQ encoding where negative and positive + /// numbers are encoded in a zigzag fashion. + /// See: https://developers.google.com/protocol-buffers/docs/encoding + /// + /// Returns false if there's not enough room left. True otherwise. + #[inline] + pub fn put_zigzag_vlq_int(&mut self, v: i64) -> bool { + let u: u64 = ((v << 1) ^ (v >> 63)) as u64; + self.put_vlq_int(u) + } +} + +/// Maximum byte length for a VLQ encoded integer +/// MAX_VLQ_BYTE_LEN = 5 for i32, and MAX_VLQ_BYTE_LEN = 10 for i64 +pub const MAX_VLQ_BYTE_LEN: usize = 10; + +pub struct BitReader { + /// The byte buffer to read from, passed in by client + buffer: Buffer, // TODO: generalize this + + /// Bytes are memcpy'd from `buffer` and values are read from this variable. + /// This is faster than reading values byte by byte directly from `buffer` + buffered_values: u64, + + /// + /// End Start + /// |............|B|B|B|B|B|B|B|B|..............| + /// ^ ^ + /// bit_offset byte_offset + /// + /// Current byte offset in `buffer` + byte_offset: usize, + + /// Current bit offset in `buffered_values` + bit_offset: usize, + + /// Total number of bytes in `buffer` + total_bytes: usize, +} + +/// Utility class to read bit/byte stream. This class can read bits or bytes that are +/// either byte aligned or not. +impl BitReader { + pub fn new(buf: Buffer, len: usize) -> Self { + let num_bytes = cmp::min(8, len); + let buffered_values = read_num_bytes_u64(num_bytes, buf.as_slice()); + BitReader { + buffer: buf, + buffered_values, + byte_offset: 0, + bit_offset: 0, + total_bytes: len, + } + } + + pub fn new_all(buf: Buffer) -> Self { + let len = buf.len(); + Self::new(buf, len) + } + + pub fn reset(&mut self, buf: Buffer) { + self.buffer = buf; + self.total_bytes = self.buffer.len(); + let num_bytes = cmp::min(8, self.total_bytes); + self.buffered_values = read_num_bytes_u64(num_bytes, self.buffer.as_slice()); + self.byte_offset = 0; + self.bit_offset = 0; + } + + /// Gets the current byte offset + #[inline] + pub fn get_byte_offset(&self) -> usize { + self.byte_offset + ceil(self.bit_offset, 8) + } + + /// Reads a value of type `T` and of size `num_bits`. + /// + /// Returns `None` if there's not enough data available. `Some` otherwise. + pub fn get_value<T: FromBytes>(&mut self, num_bits: usize) -> Option<T> { + debug_assert!(num_bits <= 64); + debug_assert!(num_bits <= size_of::<T>() * 8); + + if unlikely(self.byte_offset * 8 + self.bit_offset + num_bits > self.total_bytes * 8) { + return None; + } + + let mut v = + trailing_bits(self.buffered_values, self.bit_offset + num_bits) >> self.bit_offset; + self.bit_offset += num_bits; + + if self.bit_offset >= 64 { + self.byte_offset += 8; + self.bit_offset -= 64; + + self.reload_buffer_values(); + v |= trailing_bits(self.buffered_values, self.bit_offset) + .wrapping_shl((num_bits - self.bit_offset) as u32); + } + + Some(T::from(v)) + } + + /// Reads a `u32` value encoded using `num_bits` of bits. + /// + /// # Safety + /// + /// This method asusumes the following: + /// + /// - the `num_bits` is <= 64 + /// - the remaining number of bits to read in this reader is >= `num_bits`. + /// + /// Undefined behavior will happen if any of the above assumptions is violated. + #[inline] + pub fn get_u32_value(&mut self, num_bits: usize) -> u32 { + let mut v = + trailing_bits(self.buffered_values, self.bit_offset + num_bits) >> self.bit_offset; + self.bit_offset += num_bits; + + if self.bit_offset >= 64 { + self.byte_offset += 8; + self.bit_offset -= 64; + + self.reload_buffer_values(); + v |= trailing_bits(self.buffered_values, self.bit_offset) + .wrapping_shl((num_bits - self.bit_offset) as u32); + } + + v as u32 + } + + /// Gets at most `num` bits from this reader, and append them to the `dst` byte slice, starting + /// at bit offset `offset`. + /// + /// Returns the actual number of bits appended. In case either the `dst` slice doesn't have + /// enough space or the current reader doesn't have enough bits to consume, the returned value + /// will be less than the input `num_bits`. + /// + /// # Preconditions + /// * `offset` MUST < dst.len() * 8 + pub fn get_bits(&mut self, dst: &mut [u8], offset: usize, num_bits: usize) -> usize { + debug_assert!(offset < dst.len() * 8); + + let remaining_bits = (self.total_bytes - self.byte_offset) * 8 - self.bit_offset; + let num_bits_to_read = min(remaining_bits, min(num_bits, dst.len() * 8 - offset)); + let mut i = 0; + + // First consume all the remaining bits from the `buffered_values` if there're any. + if likely(self.bit_offset != 0) { + i += self.get_bits_buffered(dst, offset, num_bits_to_read); + } + + debug_assert!(self.bit_offset == 0 || i == num_bits_to_read); + + // Check if there's opportunity to directly copy bytes using `memcpy`. + if (offset + i) % 8 == 0 && i < num_bits_to_read { + let num_bytes = (num_bits_to_read - i) / 8; + let dst_byte_offset = (offset + i) / 8; + if num_bytes > 0 { + memcpy( + &self.buffer[self.byte_offset..self.byte_offset + num_bytes], + &mut dst[dst_byte_offset..], + ); + i += num_bytes * 8; + self.byte_offset += num_bytes; + self.reload_buffer_values(); + } + } + + debug_assert!((offset + i) % 8 != 0 || num_bits_to_read - i < 8); + + // Now copy the remaining bits if there's any. + while i < num_bits_to_read { + i += self.get_bits_buffered(dst, offset + i, num_bits_to_read - i); + } + + num_bits_to_read + } + + /// Consume at most `n` bits from `buffered_values`. Returns the actual number of bits consumed. + /// + /// # Postcondition + /// - either bits from `buffered_values` are completely drained (i.e., `bit_offset` == 0) + /// - OR the `num_bits` is < the number of remaining bits in `buffered_values` and thus the + /// returned value is < `num_bits`. + /// + /// Either way, the returned value is in range [0, 64]. + #[inline] + fn get_bits_buffered(&mut self, dst: &mut [u8], offset: usize, num_bits: usize) -> usize { + if unlikely(num_bits == 0) { + return 0; + } + + let n = min(num_bits, 64 - self.bit_offset); + let offset_i = offset / 8; + let offset_r = offset % 8; + + // Extract the value to read out of the buffer + let mut v = trailing_bits(self.buffered_values >> self.bit_offset, n); + + // Read the first byte always because n > 0 + dst[offset_i] |= (v << offset_r) as u8; + v >>= 8 - offset_r; + + // Read the rest of the bytes + ((offset_i + 1)..(offset_i + ceil(n + offset_r, 8))).for_each(|i| { + dst[i] |= v as u8; + v >>= 8; + }); + + self.bit_offset += n; + if self.bit_offset == 64 { + self.byte_offset += 8; + self.bit_offset -= 64; + self.reload_buffer_values(); + } + + n + } + + /// Skips at most `num` bits from this reader. + /// + /// Returns the actual number of bits skipped. + pub fn skip_bits(&mut self, num_bits: usize) -> usize { + let remaining_bits = (self.total_bytes - self.byte_offset) * 8 - self.bit_offset; + let num_bits_to_read = min(remaining_bits, num_bits); + let mut i = 0; + + // First skip all the remaining bits by updating the offsets of `buffered_values`. + if likely(self.bit_offset != 0) { + let n = 64 - self.bit_offset; + if num_bits_to_read < n { + self.bit_offset += num_bits_to_read; + i = num_bits_to_read; + } else { + self.byte_offset += 8; + self.bit_offset = 0; + i = n; + } + } + + // Check if there's opportunity to skip by byte + if i + 7 < num_bits_to_read { + let num_bytes = (num_bits_to_read - i) / 8; + i += num_bytes * 8; + self.byte_offset += num_bytes; + } + + if self.bit_offset == 0 { + self.reload_buffer_values(); + } + + // Now skip the remaining bits if there's any. + if i < num_bits_to_read { + self.bit_offset += num_bits_to_read - i; + } + + num_bits_to_read + } + + /// Reads a batch of `u32` values encoded using `num_bits` of bits, into `dst`. + /// + /// # Safety + /// + /// This method asusumes the following: + /// + /// - the `num_bits` is <= 64 + /// - the remaining number of bits to read in this reader is >= `total * num_bits`. + /// + /// Undefined behavior will happen if any of the above assumptions is violated. + /// + /// Unlike `[get_batch]`, this method removes a few checks such as checking the remaining number + /// of bits as well as checking the bit width for the element type in `dst`. Therefore, it is + /// more efficient. + pub unsafe fn get_u32_batch(&mut self, mut dst: *mut u32, total: usize, num_bits: usize) { + let mut i = 0; + + // First align bit offset to byte offset + if likely(self.bit_offset != 0) { + while i < total && self.bit_offset != 0 { + *dst = self.get_u32_value(num_bits); + dst = dst.offset(1); + i += 1; + } + } + + let in_buf = &self.buffer.as_slice()[self.byte_offset..]; + let mut in_ptr = in_buf as *const [u8] as *const u8 as *const u32; + while total - i >= 32 { + in_ptr = unpack32(in_ptr, dst, num_bits); + self.byte_offset += 4 * num_bits; + dst = dst.offset(32); + i += 32; + } + + self.reload_buffer_values(); + while i < total { + *dst = self.get_u32_value(num_bits); + dst = dst.offset(1); + i += 1; + } + } + + pub fn get_batch<T: FromBytes>(&mut self, batch: &mut [T], num_bits: usize) -> usize { + debug_assert!(num_bits <= 32); + debug_assert!(num_bits <= size_of::<T>() * 8); + + let mut values_to_read = batch.len(); + let needed_bits = num_bits * values_to_read; + let remaining_bits = (self.total_bytes - self.byte_offset) * 8 - self.bit_offset; + if remaining_bits < needed_bits { + values_to_read = remaining_bits / num_bits; + } + + let mut i = 0; + + // First align bit offset to byte offset + if likely(self.bit_offset != 0) { + while i < values_to_read && self.bit_offset != 0 { + batch[i] = self + .get_value(num_bits) + .expect("expected to have more data"); + i += 1; + } + } + + unsafe { + let in_buf = &self.buffer.as_slice()[self.byte_offset..]; + let mut in_ptr = in_buf as *const [u8] as *const u8 as *const u32; + // FIXME assert!(memory::is_ptr_aligned(in_ptr)); + if size_of::<T>() == 4 { + while values_to_read - i >= 32 { + let out_ptr = &mut batch[i..] as *mut [T] as *mut T as *mut u32; + in_ptr = unpack32(in_ptr, out_ptr, num_bits); + self.byte_offset += 4 * num_bits; + i += 32; + } + } else { + let mut out_buf = [0u32; 32]; + let out_ptr = &mut out_buf as &mut [u32] as *mut [u32] as *mut u32; + while values_to_read - i >= 32 { + in_ptr = unpack32(in_ptr, out_ptr, num_bits); + self.byte_offset += 4 * num_bits; + for n in 0..32 { + // We need to copy from smaller size to bigger size to avoid + // overwriting other memory regions. + if size_of::<T>() > size_of::<u32>() { + std::ptr::copy_nonoverlapping( + out_buf[n..].as_ptr(), + &mut batch[i] as *mut T as *mut u32, + 1, + ); + } else { + std::ptr::copy_nonoverlapping( + out_buf[n..].as_ptr() as *const T, + &mut batch[i] as *mut T, + 1, + ); + } + i += 1; + } + } + } + } + + debug_assert!(values_to_read - i < 32); + + self.reload_buffer_values(); + while i < values_to_read { + batch[i] = self + .get_value(num_bits) + .expect("expected to have more data"); + i += 1; + } + + values_to_read + } + + /// Reads a `num_bytes`-sized value from this buffer and return it. + /// `T` needs to be a little-endian native type. The value is assumed to be byte + /// aligned so the bit reader will be advanced to the start of the next byte before + /// reading the value. + + /// Returns `Some` if there's enough bytes left to form a value of `T`. + /// Otherwise `None`. + pub fn get_aligned<T: FromBytes>(&mut self, num_bytes: usize) -> Option<T> { + debug_assert!(8 >= size_of::<T>()); + debug_assert!(num_bytes <= size_of::<T>()); + + let bytes_read = ceil(self.bit_offset, 8); + if unlikely(self.byte_offset + bytes_read + num_bytes > self.total_bytes) { + return None; + } + + if bytes_read + num_bytes > 8 { + // There may be still unread bytes in buffered_values; however, just reloading seems to + // be faster than stitching the buffer with the next buffer based on micro benchmarks + // because reloading logic can be simpler + + // Advance byte_offset to next unread byte + self.byte_offset += bytes_read; + // Reset buffered_values + self.reload_buffer_values(); + self.bit_offset = 0 + } else { + // Advance bit_offset to next unread byte + self.bit_offset = bytes_read * 8; + } + + let v = T::from(trailing_bits( + self.buffered_values >> self.bit_offset, + num_bytes * 8, + )); + self.bit_offset += num_bytes * 8; + + if self.bit_offset == 64 { + self.byte_offset += 8; + self.bit_offset -= 64; + self.reload_buffer_values(); + } + + Some(v) + } + + /// Reads a VLQ encoded (in little endian order) int from the stream. + /// The encoded int must start at the beginning of a byte. + /// + /// Returns `None` if there's not enough bytes in the stream. `Some` otherwise. + pub fn get_vlq_int(&mut self) -> Option<i64> { + let mut shift = 0; + let mut v: i64 = 0; + while let Some(byte) = self.get_aligned::<u8>(1) { + v |= ((byte & 0x7F) as i64) << shift; + shift += 7; + debug_assert!( + shift <= MAX_VLQ_BYTE_LEN * 7, + "Num of bytes exceed MAX_VLQ_BYTE_LEN ({})", + MAX_VLQ_BYTE_LEN + ); + if likely(byte & 0x80 == 0) { + return Some(v); + } + } + None + } + + /// Reads a zigzag-VLQ encoded (in little endian order) int from the stream + /// Zigzag-VLQ is a variant of VLQ encoding where negative and positive numbers are + /// encoded in a zigzag fashion. + /// See: https://developers.google.com/protocol-buffers/docs/encoding + /// + /// Note: the encoded int must start at the beginning of a byte. + /// + /// Returns `None` if the number of bytes there's not enough bytes in the stream. + /// `Some` otherwise. + #[inline] + pub fn get_zigzag_vlq_int(&mut self) -> Option<i64> { + self.get_vlq_int().map(|v| { + let u = v as u64; + (u >> 1) as i64 ^ -((u & 1) as i64) + }) + } + + fn reload_buffer_values(&mut self) { + let bytes_to_read = cmp::min(self.total_bytes - self.byte_offset, 8); + self.buffered_values = + read_num_bytes_u64(bytes_to_read, &self.buffer.as_slice()[self.byte_offset..]); + } +} + +impl From<Vec<u8>> for BitReader { + #[inline] + fn from(vec: Vec<u8>) -> Self { + let len = vec.len(); + BitReader::new(Buffer::from(vec), len) + } +} + +/// Returns the nearest multiple of `factor` that is `>=` than `num`. Here `factor` must +/// be a power of 2. +/// +/// Copied from the arrow crate to make arrow optional +pub fn round_upto_power_of_2(num: usize, factor: usize) -> usize { + debug_assert!(factor > 0 && (factor & (factor - 1)) == 0); + (num + (factor - 1)) & !(factor - 1) +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::parquet::util::test_common::*; + + use rand::{ + distributions::{Distribution, Standard}, + Rng, + }; + use std::fmt::Debug; + + #[test] + fn test_read_num_bytes_u64() { + let buffer: Vec<u8> = vec![0, 1, 2, 3, 4, 5, 6, 7]; + for size in 0..buffer.len() { + assert_eq!( + read_num_bytes_u64(size, &buffer), + read_num_bytes!(u64, size, &buffer), + ); + } + } + + #[test] + fn test_read_num_bytes_u32() { + let buffer: Vec<u8> = vec![0, 1, 2, 3]; + for size in 0..buffer.len() { + assert_eq!( + read_num_bytes_u32(size, &buffer), + read_num_bytes!(u32, size, &buffer), + ); + } + } + + #[test] + fn test_ceil() { + assert_eq!(ceil(0, 1), 0); + assert_eq!(ceil(1, 1), 1); + assert_eq!(ceil(1, 2), 1); + assert_eq!(ceil(1, 8), 1); + assert_eq!(ceil(7, 8), 1); + assert_eq!(ceil(8, 8), 1); + assert_eq!(ceil(9, 8), 2); + assert_eq!(ceil(9, 9), 1); + assert_eq!(ceil(10000000000, 10), 1000000000); + assert_eq!(ceil(10, 10000000000), 1); + assert_eq!(ceil(10000000000, 1000000000), 10); + } + + #[test] + fn test_bit_reader_get_byte_offset() { + let buffer = vec![255; 10]; + let mut bit_reader = BitReader::from(buffer); + assert_eq!(bit_reader.get_byte_offset(), 0); // offset (0 bytes, 0 bits) + bit_reader.get_value::<i32>(6); + assert_eq!(bit_reader.get_byte_offset(), 1); // offset (0 bytes, 6 bits) + bit_reader.get_value::<i32>(10); + assert_eq!(bit_reader.get_byte_offset(), 2); // offset (0 bytes, 16 bits) + bit_reader.get_value::<i32>(20); + assert_eq!(bit_reader.get_byte_offset(), 5); // offset (0 bytes, 36 bits) + bit_reader.get_value::<i32>(30); + assert_eq!(bit_reader.get_byte_offset(), 9); // offset (8 bytes, 2 bits) + } + + #[test] + fn test_bit_reader_get_value() { + let buffer = vec![255, 0]; + let mut bit_reader = BitReader::from(buffer); + assert_eq!(bit_reader.get_value::<i32>(1), Some(1)); + assert_eq!(bit_reader.get_value::<i32>(2), Some(3)); + assert_eq!(bit_reader.get_value::<i32>(3), Some(7)); + assert_eq!(bit_reader.get_value::<i32>(4), Some(3)); + } + + #[test] + fn test_bit_reader_get_value_boundary() { + let buffer = vec![10, 0, 0, 0, 20, 0, 30, 0, 0, 0, 40, 0]; + let mut bit_reader = BitReader::from(buffer); + assert_eq!(bit_reader.get_value::<i64>(32), Some(10)); + assert_eq!(bit_reader.get_value::<i64>(16), Some(20)); + assert_eq!(bit_reader.get_value::<i64>(32), Some(30)); + assert_eq!(bit_reader.get_value::<i64>(16), Some(40)); + } + + #[test] + fn test_bit_reader_get_aligned() { + // 01110101 11001011 + let buffer = Buffer::from(vec![0x75, 0xCB]); + let mut bit_reader = BitReader::new_all(buffer.clone()); + assert_eq!(bit_reader.get_value::<i32>(3), Some(5)); + assert_eq!(bit_reader.get_aligned::<i32>(1), Some(203)); + assert_eq!(bit_reader.get_value::<i32>(1), None); + bit_reader.reset(buffer); + assert_eq!(bit_reader.get_aligned::<i32>(3), None); + } + + #[test] + fn test_bit_reader_get_vlq_int() { + // 10001001 00000001 11110010 10110101 00000110 + let buffer: Vec<u8> = vec![0x89, 0x01, 0xF2, 0xB5, 0x06]; + let mut bit_reader = BitReader::from(buffer); + assert_eq!(bit_reader.get_vlq_int(), Some(137)); + assert_eq!(bit_reader.get_vlq_int(), Some(105202)); + } + + #[test] + fn test_bit_reader_get_zigzag_vlq_int() { + let buffer: Vec<u8> = vec![0, 1, 2, 3]; + let mut bit_reader = BitReader::from(buffer); + assert_eq!(bit_reader.get_zigzag_vlq_int(), Some(0)); + assert_eq!(bit_reader.get_zigzag_vlq_int(), Some(-1)); + assert_eq!(bit_reader.get_zigzag_vlq_int(), Some(1)); + assert_eq!(bit_reader.get_zigzag_vlq_int(), Some(-2)); + } + + #[test] + fn test_set_bit() { + let mut buffer = vec![0, 0, 0]; + set_bit(&mut buffer[..], 1); + assert_eq!(buffer, vec![2, 0, 0]); + set_bit(&mut buffer[..], 4); + assert_eq!(buffer, vec![18, 0, 0]); + unset_bit(&mut buffer[..], 1); + assert_eq!(buffer, vec![16, 0, 0]); + set_bit(&mut buffer[..], 10); + assert_eq!(buffer, vec![16, 4, 0]); + set_bit(&mut buffer[..], 10); + assert_eq!(buffer, vec![16, 4, 0]); + set_bit(&mut buffer[..], 11); + assert_eq!(buffer, vec![16, 12, 0]); + unset_bit(&mut buffer[..], 10); + assert_eq!(buffer, vec![16, 8, 0]); + } + + #[test] + fn test_set_bits() { + for offset in 0..=16 { + for length in 0..=16 { + let mut actual = vec![0, 0, 0, 0]; + set_bits(&mut actual[..], offset, length); + let mut expected = vec![0, 0, 0, 0]; + for i in 0..length { + set_bit(&mut expected, offset + i); + } + assert_eq!(actual, expected); + } + } + } + + #[test] + fn test_num_required_bits() { + assert_eq!(num_required_bits(0), 0); + assert_eq!(num_required_bits(1), 1); + assert_eq!(num_required_bits(2), 2); + assert_eq!(num_required_bits(4), 3); + assert_eq!(num_required_bits(8), 4); + assert_eq!(num_required_bits(10), 4); + assert_eq!(num_required_bits(12), 4); + assert_eq!(num_required_bits(16), 5); + } + + #[test] + fn test_get_bit() { + // 00001101 + assert!(get_bit(&[0b00001101], 0)); + assert!(!get_bit(&[0b00001101], 1)); + assert!(get_bit(&[0b00001101], 2)); + assert!(get_bit(&[0b00001101], 3)); + + // 01001001 01010010 + assert!(get_bit(&[0b01001001, 0b01010010], 0)); + assert!(!get_bit(&[0b01001001, 0b01010010], 1)); + assert!(!get_bit(&[0b01001001, 0b01010010], 2)); + assert!(get_bit(&[0b01001001, 0b01010010], 3)); + assert!(!get_bit(&[0b01001001, 0b01010010], 4)); + assert!(!get_bit(&[0b01001001, 0b01010010], 5)); + assert!(get_bit(&[0b01001001, 0b01010010], 6)); + assert!(!get_bit(&[0b01001001, 0b01010010], 7)); + assert!(!get_bit(&[0b01001001, 0b01010010], 8)); + assert!(get_bit(&[0b01001001, 0b01010010], 9)); + assert!(!get_bit(&[0b01001001, 0b01010010], 10)); + assert!(!get_bit(&[0b01001001, 0b01010010], 11)); + assert!(get_bit(&[0b01001001, 0b01010010], 12)); + assert!(!get_bit(&[0b01001001, 0b01010010], 13)); + assert!(get_bit(&[0b01001001, 0b01010010], 14)); + assert!(!get_bit(&[0b01001001, 0b01010010], 15)); + } + + #[test] + fn test_log2() { + assert_eq!(log2(1), 0); + assert_eq!(log2(2), 1); + assert_eq!(log2(3), 2); + assert_eq!(log2(4), 2); + assert_eq!(log2(5), 3); + assert_eq!(log2(5), 3); + assert_eq!(log2(6), 3); + assert_eq!(log2(7), 3); + assert_eq!(log2(8), 3); + assert_eq!(log2(9), 4); + } + + #[test] + fn test_skip() { + let mut writer = BitWriter::new(5); + let old_offset = writer.skip(1).expect("skip() should return OK"); + writer.put_aligned(42, 4); + writer.put_aligned_offset(0x10, 1, old_offset); + let result = writer.consume(); + assert_eq!(result.as_ref(), [0x10, 42, 0, 0, 0]); + + writer = BitWriter::new(4); + let result = writer.skip(5); + assert!(result.is_err()); + } + + #[test] + fn test_get_next_byte_ptr() { + let mut writer = BitWriter::new(5); + { + let first_byte = writer + .get_next_byte_ptr(1) + .expect("get_next_byte_ptr() should return OK"); + first_byte[0] = 0x10; + } + writer.put_aligned(42, 4); + let result = writer.consume(); + assert_eq!(result.as_ref(), [0x10, 42, 0, 0, 0]); + } + + #[test] + fn test_consume_flush_buffer() { + let mut writer1 = BitWriter::new(3); + let mut writer2 = BitWriter::new(3); + for i in 1..10 { + writer1.put_value(i, 4); + writer2.put_value(i, 4); + } + let res1 = writer1.flush_buffer(); + let res2 = writer2.consume(); + assert_eq!(res1, &res2[..]); + } + + #[test] + fn test_put_get_bool() { + let len = 8; + let mut writer = BitWriter::new(len); + + for i in 0..8 { + let result = writer.put_value(i % 2, 1); + assert!(result); + } + + writer.flush(); + { + let buffer = writer.buffer(); + assert_eq!(buffer[0], 0b10101010); + } + + // Write 00110011 + for i in 0..8 { + let result = match i { + 0 | 1 | 4 | 5 => writer.put_value(false as u64, 1), + _ => writer.put_value(true as u64, 1), + }; + assert!(result); + } + writer.flush(); + { + let buffer = writer.buffer(); + assert_eq!(buffer[0], 0b10101010); + assert_eq!(buffer[1], 0b11001100); + } + + let mut reader = BitReader::from(writer.consume()); + + for i in 0..8 { + let val = reader + .get_value::<u8>(1) + .expect("get_value() should return OK"); + assert_eq!(val, i % 2); + } + + for i in 0..8 { + let val = reader + .get_value::<bool>(1) + .expect("get_value() should return OK"); + match i { + 0 | 1 | 4 | 5 => assert!(!val), + _ => assert!(val), + } + } + } + + #[test] + fn test_put_value_roundtrip() { + test_put_value_rand_numbers(32, 2); + test_put_value_rand_numbers(32, 3); + test_put_value_rand_numbers(32, 4); + test_put_value_rand_numbers(32, 5); + test_put_value_rand_numbers(32, 6); + test_put_value_rand_numbers(32, 7); + test_put_value_rand_numbers(32, 8); + test_put_value_rand_numbers(64, 16); + test_put_value_rand_numbers(64, 24); + test_put_value_rand_numbers(64, 32); + } + + fn test_put_value_rand_numbers(total: usize, num_bits: usize) { + assert!(num_bits < 64); + let num_bytes = ceil(num_bits, 8); + let mut writer = BitWriter::new(num_bytes * total); + let values: Vec<u64> = random_numbers::<u64>(total) + .iter() + .map(|v| v & ((1 << num_bits) - 1)) + .collect(); + (0..total).for_each(|i| { + assert!( + writer.put_value(values[i], num_bits), + "[{}]: put_value() failed", + i + ); + }); + + let mut reader = BitReader::from(writer.consume()); + (0..total).for_each(|i| { + let v = reader + .get_value::<u64>(num_bits) + .expect("get_value() should return OK"); + assert_eq!( + v, values[i], + "[{}]: expected {} but got {}", + i, values[i], v + ); + }); + } + + #[test] + fn test_get_bits() { + const NUM_BYTES: usize = 100; + + let mut vec = vec![0; NUM_BYTES]; + let total_num_bits = NUM_BYTES * 8; + let v = random_bools(total_num_bits); + (0..total_num_bits).for_each(|i| { + if v[i] { + set_bit(&mut vec, i); + } else { + unset_bit(&mut vec, i); + } + }); + + let expected = vec.clone(); + + // test reading the first time from a buffer + for &(offset, num_bits) in [(0, 10), (2, 10), (8, 16), (25, 40), (7, 64)].iter() { + let mut reader = BitReader::from(vec.clone()); + let mut buffer = vec![0; NUM_BYTES]; + + let actual_bits_read = reader.get_bits(&mut buffer, offset, num_bits); + let expected_bits_read = ::std::cmp::min(buffer.len() * 8 - offset, num_bits); + assert_eq!(expected_bits_read, actual_bits_read); + + for i in 0..actual_bits_read { + assert_eq!(get_bit(&expected, i), get_bit(&buffer, offset + i)); + } + } + + // test reading consecutively from a buffer + let mut reader = BitReader::from(vec); + let mut buffer = vec![0; NUM_BYTES]; + let mut rng = rand::thread_rng(); + let mut bits_read = 0; + + loop { + if bits_read >= total_num_bits { + break; + } + let n: usize = rng.gen(); + let num_bits = n % 20; + bits_read += reader.get_bits(&mut buffer, bits_read, num_bits); + } + + assert_eq!(total_num_bits, bits_read); + assert_eq!(&expected, &buffer); + } + + #[test] + fn test_skip_bits() { + const NUM_BYTES: usize = 100; + + let mut vec = vec![0; NUM_BYTES]; + let total_num_bits = NUM_BYTES * 8; + let v = random_bools(total_num_bits); + (0..total_num_bits).for_each(|i| { + if v[i] { + set_bit(&mut vec, i); + } else { + unset_bit(&mut vec, i); + } + }); + + let expected = vec.clone(); + + // test skipping and check the next value + let mut reader = BitReader::from(vec); + let mut bits_read = 0; + for &num_bits in [10, 60, 8].iter() { + let actual_bits_read = reader.skip_bits(num_bits); + assert_eq!(num_bits, actual_bits_read); + + bits_read += num_bits; + assert_eq!(Some(get_bit(&expected, bits_read)), reader.get_value(1)); + bits_read += 1; + } + + // test skipping consecutively + let mut rng = rand::thread_rng(); + loop { + if bits_read >= total_num_bits { + break; + } + let n: usize = rng.gen(); + let num_bits = n % 20; + bits_read += reader.skip_bits(num_bits); + } + + assert_eq!(total_num_bits, bits_read); + } + + #[test] + fn test_get_batch() { + const SIZE: &[usize] = &[1, 31, 32, 33, 128, 129]; + for s in SIZE { + for i in 0..33 { + match i { + 0..=8 => test_get_batch_helper::<u8>(*s, i), + 9..=16 => test_get_batch_helper::<u16>(*s, i), + _ => test_get_batch_helper::<u32>(*s, i), + } + } + } + } + + fn test_get_batch_helper<T>(total: usize, num_bits: usize) + where + T: FromBytes + Default + Clone + Debug + Eq, + { + assert!(num_bits <= 32); + let num_bytes = ceil(num_bits, 8); + let mut writer = BitWriter::new(num_bytes * total); + + let values: Vec<u32> = random_numbers::<u32>(total) + .iter() + .map(|v| v & ((1u64 << num_bits) - 1) as u32) + .collect(); + + // Generic values used to check against actual values read from `get_batch`. + let expected_values: Vec<T> = values.iter().map(|v| from_ne_slice(v.as_bytes())).collect(); + + (0..total).for_each(|i| { + assert!(writer.put_value(values[i] as u64, num_bits)); + }); + + let buf = writer.consume(); + let mut reader = BitReader::from(buf); + let mut batch = vec![T::default(); values.len()]; + let values_read = reader.get_batch::<T>(&mut batch, num_bits); + assert_eq!(values_read, values.len()); + for i in 0..batch.len() { + assert_eq!( + batch[i], expected_values[i], + "num_bits = {}, index = {}", + num_bits, i + ); + } + } + + #[test] + fn test_get_u32_batch() { + const SIZE: &[usize] = &[1, 31, 32, 33, 128, 129]; + for total in SIZE { + for num_bits in 1..33 { + let num_bytes = ceil(num_bits, 8); + let mut writer = BitWriter::new(num_bytes * total); + + let values: Vec<u32> = random_numbers::<u32>(*total) + .iter() + .map(|v| v & ((1u64 << num_bits) - 1) as u32) + .collect(); + + (0..*total).for_each(|i| { + assert!(writer.put_value(values[i] as u64, num_bits)); + }); + + let buf = writer.consume(); + let mut reader = BitReader::from(buf); + let mut batch = vec![0u32; values.len()]; + unsafe { + reader.get_u32_batch(batch.as_mut_ptr(), *total, num_bits); + } + for i in 0..batch.len() { + assert_eq!( + batch[i], values[i], + "num_bits = {}, index = {}", + num_bits, i + ); + } + } + } + } + + #[test] + fn test_put_aligned_roundtrip() { + test_put_aligned_rand_numbers::<u8>(4, 3); + test_put_aligned_rand_numbers::<u8>(16, 5); + test_put_aligned_rand_numbers::<i16>(32, 7); + test_put_aligned_rand_numbers::<i16>(32, 9); + test_put_aligned_rand_numbers::<i32>(32, 11); + test_put_aligned_rand_numbers::<i32>(32, 13); + test_put_aligned_rand_numbers::<i64>(32, 17); + test_put_aligned_rand_numbers::<i64>(32, 23); + } + + fn test_put_aligned_rand_numbers<T>(total: usize, num_bits: usize) + where + T: Copy + FromBytes + AsBytes + Debug + PartialEq, + Standard: Distribution<T>, + { + assert!(num_bits <= 32); + assert!(total % 2 == 0); + + let aligned_value_byte_width = std::mem::size_of::<T>(); + let value_byte_width = ceil(num_bits, 8); + let mut writer = + BitWriter::new((total / 2) * (aligned_value_byte_width + value_byte_width)); + let values: Vec<u32> = random_numbers::<u32>(total / 2) + .iter() + .map(|v| v & ((1 << num_bits) - 1)) + .collect(); + let aligned_values = random_numbers::<T>(total / 2); + + for i in 0..total { + let j = i / 2; + if i % 2 == 0 { + assert!( + writer.put_value(values[j] as u64, num_bits), + "[{}]: put_value() failed", + i + ); + } else { + assert!( + writer.put_aligned::<T>(aligned_values[j], aligned_value_byte_width), + "[{}]: put_aligned() failed", + i + ); + } + } + + let mut reader = BitReader::from(writer.consume()); + for i in 0..total { + let j = i / 2; + if i % 2 == 0 { + let v = reader + .get_value::<u64>(num_bits) + .expect("get_value() should return OK"); + assert_eq!( + v, values[j] as u64, + "[{}]: expected {} but got {}", + i, values[j], v + ); + } else { + let v = reader + .get_aligned::<T>(aligned_value_byte_width) + .expect("get_aligned() should return OK"); + assert_eq!( + v, aligned_values[j], + "[{}]: expected {:?} but got {:?}", + i, aligned_values[j], v + ); + } + } + } + + #[test] + fn test_put_vlq_int() { + let total = 64; + let mut writer = BitWriter::new(total * 32); + let values = random_numbers::<u32>(total); + (0..total).for_each(|i| { + assert!( + writer.put_vlq_int(values[i] as u64), + "[{}]; put_vlq_int() failed", + i + ); + }); + + let mut reader = BitReader::from(writer.consume()); + (0..total).for_each(|i| { + let v = reader + .get_vlq_int() + .expect("get_vlq_int() should return OK"); + assert_eq!( + v as u32, values[i], + "[{}]: expected {} but got {}", + i, values[i], v + ); + }); + } + + #[test] + fn test_put_zigzag_vlq_int() { + let total = 64; + let mut writer = BitWriter::new(total * 32); + let values = random_numbers::<i32>(total); + (0..total).for_each(|i| { + assert!( + writer.put_zigzag_vlq_int(values[i] as i64), + "[{}]; put_zigzag_vlq_int() failed", + i + ); + }); + + let mut reader = BitReader::from(writer.consume()); + (0..total).for_each(|i| { + let v = reader + .get_zigzag_vlq_int() + .expect("get_zigzag_vlq_int() should return OK"); + assert_eq!( + v as i32, values[i], + "[{}]: expected {} but got {}", + i, values[i], v + ); + }); + } +} diff --git a/core/src/common/buffer.rs b/core/src/common/buffer.rs new file mode 100644 index 0000000000..f24038a955 --- /dev/null +++ b/core/src/common/buffer.rs @@ -0,0 +1,342 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 crate::common::bit; +use arrow::buffer::Buffer as ArrowBuffer; +use std::{ + alloc::{handle_alloc_error, Layout}, + ptr::NonNull, + sync::Arc, +}; + +/// A buffer implementation. This is very similar to Arrow's [`MutableBuffer`] implementation, +/// except that there are two modes depending on whether `owned` is true or false. +/// +/// If `owned` is true, this behaves the same way as a Arrow [`MutableBuffer`], and the struct is +/// the unique owner for the memory it wraps. The holder of this buffer can read or write the +/// buffer, and the buffer itself will be released when it goes out of scope. +/// +/// Also note that, in `owned` mode, the buffer is always filled with 0s, and its length is always +/// equal to its capacity. It's up to the caller to decide which part of the buffer contains valid +/// data. +/// +/// If `owned` is false, this buffer is an alias to another buffer. The buffer itself becomes +/// immutable and can only be read. +#[derive(Debug)] +pub struct CometBuffer { + data: NonNull<u8>, + len: usize, + capacity: usize, + /// Whether this buffer owns the data it points to. + owned: bool, +} + +unsafe impl Sync for CometBuffer {} +unsafe impl Send for CometBuffer {} + +/// All buffers are aligned to 64 bytes. +const ALIGNMENT: usize = 64; + +impl CometBuffer { + /// Initializes a owned buffer filled with 0. + pub fn new(capacity: usize) -> Self { + let aligned_capacity = bit::round_upto_power_of_2(capacity, ALIGNMENT); + unsafe { + let layout = Layout::from_size_align_unchecked(aligned_capacity, ALIGNMENT); + let ptr = std::alloc::alloc_zeroed(layout); + Self { + data: NonNull::new(ptr).unwrap_or_else(|| handle_alloc_error(layout)), + len: aligned_capacity, + capacity: aligned_capacity, + owned: true, + } + } + } + + pub fn from_ptr(ptr: *const u8, len: usize, capacity: usize) -> Self { + assert_eq!( + capacity % ALIGNMENT, + 0, + "input buffer is not aligned to {} bytes", + ALIGNMENT + ); + Self { + data: NonNull::new(ptr as *mut u8).unwrap_or_else(|| { + panic!( + "cannot create CometBuffer from (ptr: {:?}, len: {}, capacity: {}", + ptr, len, capacity + ) + }), + len, + capacity, + owned: false, + } + } + + /// Returns the capacity of this buffer. + pub fn capacity(&self) -> usize { + self.capacity + } + + /// Returns the length (i.e., number of bytes) in this buffer. + pub fn len(&self) -> usize { + self.len + } + + /// Whether this buffer is empty. + pub fn is_empty(&self) -> bool { + self.len == 0 + } + + /// Returns the data stored in this buffer as a slice. + pub fn as_slice(&self) -> &[u8] { + self + } + + /// Returns the data stored in this buffer as a mutable slice. + pub fn as_slice_mut(&mut self) -> &mut [u8] { + debug_assert!(self.owned, "cannot modify un-owned buffer"); + self + } + + /// Extends this buffer (must be an owned buffer) by appending bytes from `src`, + /// starting from `offset`. + pub fn extend_from_slice(&mut self, offset: usize, src: &[u8]) { + debug_assert!(self.owned, "cannot modify un-owned buffer"); + debug_assert!( + offset + src.len() <= self.capacity(), + "buffer overflow, offset = {}, src.len = {}, capacity = {}", + offset, + src.len(), + self.capacity() + ); + + unsafe { + let dst = self.data.as_ptr().add(offset); + std::ptr::copy_nonoverlapping(src.as_ptr(), dst, src.len()) + } + } + + /// Returns a raw pointer to this buffer's internal memory + /// This pointer is guaranteed to be aligned along cache-lines. + #[inline] + pub const fn as_ptr(&self) -> *const u8 { + self.data.as_ptr() + } + + /// Returns a mutable raw pointer to this buffer's internal memory + /// This pointer is guaranteed to be aligned along cache-lines. + #[inline] + pub fn as_mut_ptr(&mut self) -> *mut u8 { + debug_assert!(self.owned, "cannot modify un-owned buffer"); + self.data.as_ptr() + } + + /// Returns an immutable Arrow buffer on the content of this buffer. + /// + /// # Safety + /// + /// This function is highly unsafe since it leaks the raw pointer to the memory region that the + /// originally this buffer is tracking. Because of this, the caller of this function is + /// expected to make sure the returned immutable [`ArrowBuffer`] will never live longer than the + /// this buffer. Otherwise it will result to dangling pointers. + /// + /// In the particular case of the columnar reader, we'll guarantee the above since the reader + /// itself is closed at the very end, after the Spark task is completed (either successfully or + /// unsuccessfully) through task completion listener. + /// + /// When re-using [`MutableVector`] in Comet native operators, across multiple input batches, + /// because of the iterator-style pattern, the content of the original mutable buffer will only + /// be updated once upstream operators fully consumed the previous output batch. For breaking + /// operators, they are responsible for copying content out of the buffers. + pub unsafe fn to_arrow(&self) -> ArrowBuffer { + let ptr = NonNull::new_unchecked(self.data.as_ptr()); + // Uses a dummy `Arc::new(0)` as `Allocation` to ensure the memory region pointed by + // `ptr` won't be freed when the returned `ArrowBuffer` goes out of scope. + ArrowBuffer::from_custom_allocation(ptr, self.len, Arc::new(0)) + } + + /// Resets this buffer by filling all bytes with zeros. + pub fn reset(&mut self) { + debug_assert!(self.owned, "cannot modify un-owned buffer"); + unsafe { + std::ptr::write_bytes(self.as_mut_ptr(), 0, self.len); + } + } + + /// Resize this buffer to the `new_capacity`. For additional bytes allocated, they are filled + /// with 0. if `new_capacity` is less than the current capacity of this buffer, this is a no-op. + #[inline(always)] + pub fn resize(&mut self, new_capacity: usize) { + debug_assert!(self.owned, "cannot modify un-owned buffer"); + if new_capacity > self.len { + let (ptr, new_capacity) = + unsafe { Self::reallocate(self.data, self.capacity, new_capacity) }; + let diff = new_capacity - self.len; + self.data = ptr; + self.capacity = new_capacity; + // write the value + unsafe { self.data.as_ptr().add(self.len).write_bytes(0, diff) }; + self.len = new_capacity; + } + } + + unsafe fn reallocate( + ptr: NonNull<u8>, + old_capacity: usize, + new_capacity: usize, + ) -> (NonNull<u8>, usize) { + let new_capacity = bit::round_upto_power_of_2(new_capacity, ALIGNMENT); + let new_capacity = std::cmp::max(new_capacity, old_capacity * 2); + let raw_ptr = std::alloc::realloc( + ptr.as_ptr(), + Layout::from_size_align_unchecked(old_capacity, ALIGNMENT), + new_capacity, + ); + let ptr = NonNull::new(raw_ptr).unwrap_or_else(|| { + handle_alloc_error(Layout::from_size_align_unchecked(new_capacity, ALIGNMENT)) + }); + (ptr, new_capacity) + } +} + +impl Drop for CometBuffer { + fn drop(&mut self) { + if self.owned { + unsafe { + std::alloc::dealloc( + self.data.as_ptr(), + Layout::from_size_align_unchecked(self.capacity, ALIGNMENT), + ) + } + } + } +} + +impl PartialEq for CometBuffer { + fn eq(&self, other: &CometBuffer) -> bool { + if self.data.as_ptr() == other.data.as_ptr() { + return true; + } + if self.len != other.len { + return false; + } + if self.capacity != other.capacity { + return false; + } + self.as_slice() == other.as_slice() + } +} + +impl From<&ArrowBuffer> for CometBuffer { + fn from(value: &ArrowBuffer) -> Self { + assert_eq!(value.len(), value.capacity()); + CometBuffer::from_ptr(value.as_ptr(), value.len(), value.capacity()) + } +} + +impl std::ops::Deref for CometBuffer { + type Target = [u8]; + + fn deref(&self) -> &[u8] { + unsafe { std::slice::from_raw_parts(self.as_ptr(), self.len) } + } +} + +impl std::ops::DerefMut for CometBuffer { + fn deref_mut(&mut self) -> &mut [u8] { + assert!(self.owned, "cannot modify un-owned buffer"); + unsafe { std::slice::from_raw_parts_mut(self.as_mut_ptr(), self.capacity) } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::buffer::Buffer as ArrowBuffer; + + #[test] + fn test_buffer_new() { + let buf = CometBuffer::new(63); + assert_eq!(64, buf.capacity()); + assert_eq!(64, buf.len()); + assert!(!buf.is_empty()); + } + + #[test] + fn test_resize() { + let mut buf = CometBuffer::new(1); + assert_eq!(64, buf.capacity()); + assert_eq!(64, buf.len()); + + buf.resize(100); + assert_eq!(128, buf.capacity()); + assert_eq!(128, buf.len()); + + // resize with less capacity is no-op + buf.resize(20); + assert_eq!(128, buf.capacity()); + assert_eq!(128, buf.len()); + } + + #[test] + fn test_extend_from_slice() { + let mut buf = CometBuffer::new(100); + buf.extend_from_slice(0, b"hello"); + assert_eq!(b"hello", &buf.as_slice()[0..5]); + + buf.extend_from_slice(5, b" world"); + assert_eq!(b"hello world", &buf.as_slice()[0..11]); + + buf.reset(); + buf.extend_from_slice(0, b"hello arrow"); + assert_eq!(b"hello arrow", &buf.as_slice()[0..11]); + } + + #[test] + fn test_to_arrow() { + let mut buf = CometBuffer::new(1); + + let str = b"aaaa bbbb cccc dddd"; + buf.extend_from_slice(0, str.as_slice()); + + assert_eq!(64, buf.len()); + assert_eq!(64, buf.capacity()); + assert_eq!(b"aaaa bbbb cccc dddd", &buf.as_slice()[0..str.len()]); + + unsafe { + let immutable_buf: ArrowBuffer = buf.to_arrow(); + assert_eq!(64, immutable_buf.len()); + assert_eq!(str, &immutable_buf.as_slice()[0..str.len()]); + } + } + + #[test] + fn test_unowned() { + let arrow_buf = ArrowBuffer::from(b"hello comet"); + let buf = CometBuffer::from_ptr(arrow_buf.as_ptr(), arrow_buf.len(), arrow_buf.capacity()); + + assert_eq!(11, buf.len()); + assert_eq!(64, buf.capacity()); + assert_eq!(b"hello comet", &buf.as_slice()[0..11]); + + unsafe { + let arrow_buf2 = buf.to_arrow(); + assert_eq!(arrow_buf, arrow_buf2); + } + } +} diff --git a/core/src/common/mod.rs b/core/src/common/mod.rs new file mode 100644 index 0000000000..8d5030c023 --- /dev/null +++ b/core/src/common/mod.rs @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#[macro_use] +pub mod bit; + +use crate::TypeTrait; + +/// Getter APIs for Comet vectors. +trait ValueGetter<T: TypeTrait> { + /// Gets the non-null value at `idx`. + /// + /// Note that null check needs to be done before the call, to ensure the value at `idx` is + /// not null. + fn value(&self, idx: usize) -> T::Native; +} + +/// Setter APIs for Comet mutable vectors. +trait ValueSetter<T: TypeTrait> { + /// Appends a non-null value `v` to the end of this vector. + fn append_value(&mut self, v: &T::Native); +} + +mod vector; + +mod buffer; +pub use buffer::*; + +mod mutable_vector; +pub use mutable_vector::*; diff --git a/core/src/common/mutable_vector.rs b/core/src/common/mutable_vector.rs new file mode 100644 index 0000000000..ba29fc01a5 --- /dev/null +++ b/core/src/common/mutable_vector.rs @@ -0,0 +1,409 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::ArrayData, buffer::Buffer as ArrowBuffer, datatypes::DataType as ArrowDataType, +}; + +use crate::{ + common::{bit, CometBuffer, ValueGetter, ValueSetter}, + BinaryType, StringType, StringView, TypeTrait, +}; + +const DEFAULT_ARRAY_LEN: usize = 4; + +/// A mutable vector that can be re-used across batches. +#[derive(Debug)] +pub struct MutableVector { + /// The Arrow type for the elements of this vector. + pub(crate) arrow_type: ArrowDataType, + + /// The number of total elements in this vector. + pub(crate) num_values: usize, + + /// The number of null elements in this vector, must <= `num_values`. + pub(crate) num_nulls: usize, + + /// The capacity of the vector + pub(crate) capacity: usize, + + /// How many bits are required to store a single value + pub(crate) bit_width: usize, + + /// The validity buffer of this Arrow vector. A bit set at position `i` indicates the `i`th + /// element is not null. Otherwise, an unset bit at position `i` indicates the `i`th element is + /// null. + pub(crate) validity_buffer: CometBuffer, + + /// The value buffer of this Arrow vector. This could store either offsets if the vector + /// is of list or struct type, or actual values themselves otherwise. + pub(crate) value_buffer: CometBuffer, + + /// Child vectors for non-primitive types (e.g., list, struct). + pub(crate) children: Vec<MutableVector>, + + /// Dictionary (i.e., values) associated with this vector. Only set if using dictionary + /// encoding. + pub(crate) dictionary: Option<Box<MutableVector>>, + + /// Whether all the values in the vector are not null. + /// + /// This is useful so we can skip setting non-null for each individual value, in the + /// `validity_buffer`. + all_non_null: bool, +} + +impl MutableVector { + pub fn new(capacity: usize, arrow_type: &ArrowDataType) -> Self { + let bit_width = Self::bit_width(arrow_type); + Self::new_with_bit_width(capacity, arrow_type.clone(), bit_width) + } + + pub fn new_with_bit_width( + capacity: usize, + arrow_type: ArrowDataType, + bit_width: usize, + ) -> Self { + let validity_len = bit::ceil(capacity, 8); + let validity_buffer = CometBuffer::new(validity_len); + + let mut value_capacity = capacity; + if Self::is_binary_type(&arrow_type) { + // Arrow offset array needs to have one extra slot + value_capacity += 1; + } + // Make sure the capacity is positive + let len = bit::ceil(value_capacity * bit_width, 8); + let mut value_buffer = CometBuffer::new(len); + + let mut children = Vec::new(); + + match arrow_type { + ArrowDataType::Binary | ArrowDataType::Utf8 => { + children.push(MutableVector::new_with_bit_width( + capacity, + ArrowDataType::Int8, + DEFAULT_ARRAY_LEN * 8, + )); + } + _ => {} + } + + if Self::is_binary_type(&arrow_type) { + // Setup the first offset which is always 0. + let zero: u32 = 0; + bit::memcpy_value(&zero, 4, &mut value_buffer); + } + + Self { + arrow_type, + num_values: 0, + num_nulls: 0, + capacity, + bit_width, + validity_buffer, + value_buffer, + children, + dictionary: None, + all_non_null: false, + } + } + + /// Appends a non-null value `v` to the end of this vector. + #[inline] + pub fn append_value<T: TypeTrait>(&mut self, v: &T::Native) { + <dyn ValueSetter<T>>::append_value(self, v); + } + + /// Gets the non-null value at `idx` of this vector. + #[inline] + pub fn value<T: TypeTrait>(&self, idx: usize) -> T::Native { + <dyn ValueGetter<T>>::value(self, idx) + } + + /// Whether the given value at `idx` of this vector is null. + #[inline] + pub fn is_null(&self, idx: usize) -> bool { + unsafe { !bit::get_bit_raw(self.validity_buffer.as_ptr(), idx) } + } + + /// Resets this vector to the initial state. + #[inline] + pub fn reset(&mut self) { + self.num_values = 0; + self.num_nulls = 0; + self.all_non_null = false; + self.validity_buffer.reset(); + if Self::is_binary_type(&self.arrow_type) { + // Reset the first offset to 0 + let zero: u32 = 0; + bit::memcpy_value(&zero, 4, &mut self.value_buffer); + // Also reset the child value vector + let child = &mut self.children[0]; + child.reset(); + } else if Self::should_reset_value_buffer(&self.arrow_type) { + self.value_buffer.reset(); + } + } + + /// Appends a new null value to the end of this vector. + #[inline] + pub fn put_null(&mut self) { + self.put_nulls(1) + } + + /// Appends `n` null values to the end of this vector. + #[inline] + pub fn put_nulls(&mut self, n: usize) { + // We need to update offset buffer for binary. + if Self::is_binary_type(&self.arrow_type) { + let mut offset = self.num_values * 4; + let prev_offset_value = bit::read_num_bytes_u32(4, &self.value_buffer[offset..]); + offset += 4; + (0..n).for_each(|_| { + bit::memcpy_value(&prev_offset_value, 4, &mut self.value_buffer[offset..]); + offset += 4; + }); + } + + self.num_nulls += n; + self.num_values += n; + } + + /// Returns the number of total values (including both null and non-null) of this vector. + #[inline] + pub fn num_values(&self) -> usize { + self.num_values + } + + /// Returns the number of null values of this vector. + #[inline] + pub fn num_nulls(&self) -> usize { + self.num_nulls + } + + #[inline] + pub fn set_not_null(&mut self, i: usize) { + unsafe { + bit::set_bit_raw(self.validity_buffer.as_mut_ptr(), i); + } + } + + /// Sets all values in this vector to be non-null. + #[inline] + pub fn set_all_non_null(&mut self) { + self.all_non_null = true; + } + + /// Sets the content of validity buffer to be `buffer`. + pub fn set_validity_buffer(&mut self, buffer: &ArrowBuffer) { + self.validity_buffer = buffer.into(); + } + + /// Sets the content of value buffer to be `buffer`. + pub fn set_value_buffer(&mut self, buffer: &ArrowBuffer) { + self.value_buffer = buffer.into(); + } + + /// Sets the dictionary of this to be `dict`. + pub fn set_dictionary(&mut self, dict: MutableVector) { + self.dictionary = Some(Box::new(dict)) + } + + /// Clones this into an Arrow [`ArrayData`](arrow::array::ArrayData). Note that the caller of + /// this method MUST make sure the returned `ArrayData` won't live longer than this vector + /// itself. Otherwise, dangling pointer may happen. + /// + /// # Safety + /// + /// This method is highly unsafe since it calls `to_immutable` which leaks raw pointer to the + /// memory region that are tracked by `ArrowMutableBuffer`. Please see comments on + /// `to_immutable` buffer to understand the motivation. + pub fn get_array_data(&mut self) -> ArrayData { + unsafe { + let data_type = if let Some(d) = &self.dictionary { + ArrowDataType::Dictionary( + Box::new(ArrowDataType::Int32), + Box::new(d.arrow_type.clone()), + ) + } else { + self.arrow_type.clone() + }; + let mut builder = ArrayData::builder(data_type) + .len(self.num_values) + .add_buffer(self.value_buffer.to_arrow()); + + builder = if self.all_non_null { + builder.null_count(0) + } else { + builder + .null_bit_buffer(Some(self.validity_buffer.to_arrow())) + .null_count(self.num_nulls) + }; + + if Self::is_binary_type(&self.arrow_type) && self.dictionary.is_none() { + let child = &mut self.children[0]; + builder = builder.add_buffer(child.value_buffer.to_arrow()); + } + + if let Some(d) = &mut self.dictionary { + builder = builder.add_child_data(d.get_array_data()); + } + + builder.build_unchecked() + } + } + + /// Returns the number of bits it takes to store one element of `arrow_type` in the value buffer + /// of this vector. + pub fn bit_width(arrow_type: &ArrowDataType) -> usize { + match arrow_type { + ArrowDataType::Boolean => 1, + ArrowDataType::Int8 => 8, + ArrowDataType::Int16 => 16, + ArrowDataType::Int32 | ArrowDataType::Float32 | ArrowDataType::Date32 => 32, + ArrowDataType::Int64 | ArrowDataType::Float64 | ArrowDataType::Timestamp(_, _) => 64, + ArrowDataType::FixedSizeBinary(type_length) => *type_length as usize * 8, + ArrowDataType::Decimal128(..) => 128, // Arrow stores decimal with 16 bytes + ArrowDataType::Binary | ArrowDataType::Utf8 => 32, // Only count offset size + dt => panic!("Unsupported Arrow data type: {:?}", dt), + } + } + + #[inline] + fn is_binary_type(dt: &ArrowDataType) -> bool { + matches!(dt, ArrowDataType::Binary | ArrowDataType::Utf8) + } + + #[inline] + fn should_reset_value_buffer(dt: &ArrowDataType) -> bool { + // - Boolean type expects have a zeroed value buffer + // - Decimal may pad buffer with 0xff so we need to clear them before a new batch + matches!(dt, ArrowDataType::Boolean | ArrowDataType::Decimal128(_, _)) + } + + /// Creates an immutable reference from a mutable Arrow buffer `buf`. + /// + /// # Safety + /// + /// This function is highly unsafe. Please see documentation of the [`to_arrow`] method for + /// details. + #[inline] + unsafe fn to_immutable(buf: &CometBuffer) -> ArrowBuffer { + buf.to_arrow() + } +} + +impl<T: TypeTrait> ValueGetter<T> for MutableVector { + default fn value(&self, idx: usize) -> T::Native { + unsafe { + let ptr = self.value_buffer.as_ptr() as *const T::Native; + *ptr.add(idx) + } + } +} + +impl ValueGetter<StringType> for MutableVector { + fn value(&self, _: usize) -> StringView { + unimplemented!("'value' on StringType is currently unsupported"); + } +} + +impl ValueGetter<BinaryType> for MutableVector { + fn value(&self, _: usize) -> StringView { + unimplemented!("'value' on BinaryType is currently unsupported"); + } +} + +impl<T: TypeTrait> ValueSetter<T> for MutableVector { + default fn append_value(&mut self, v: &T::Native) { + unsafe { + let ptr = self.value_buffer.as_mut_ptr() as *mut T::Native; + *ptr.add(self.num_values) = *v; + } + self.num_values += 1; + } +} + +impl ValueSetter<StringType> for MutableVector { + fn append_value(&mut self, _: &StringView) { + unimplemented!("'append_value' on StringType is currently unsupported"); + } +} + +impl ValueSetter<BinaryType> for MutableVector { + fn append_value(&mut self, _: &StringView) { + unimplemented!("'append_value' on BinaryType is currently unsupported"); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::IntegerType; + + #[test] + fn set_and_get() { + let mut mv = MutableVector::new(1024, &ArrowDataType::Int32); + + for n in 0..100 { + mv.append_value::<IntegerType>(&(n * n)); + } + assert_eq!(mv.num_values(), 100); + assert_eq!(mv.num_nulls(), 0); + + for n in 0..100 { + assert_eq!(mv.value::<IntegerType>(n) as usize, n * n); + } + + mv.reset(); + + for n in 0..200 { + if n % 2 == 0 { + mv.put_null(); + } else { + mv.append_value::<IntegerType>(&(n * 2)); + } + } + + assert_eq!(mv.num_values(), 200); + assert_eq!(mv.num_nulls(), 100); + + for n in 0..200 { + if n % 2 == 0 { + assert!(mv.is_null(n)); + } else { + assert_eq!(mv.value::<IntegerType>(n) as usize, n * 2); + } + } + } + + #[test] + #[should_panic] + fn set_string_unsupported() { + let mut mv = MutableVector::new(1024, &ArrowDataType::Utf8); + let sv = StringView::default(); + mv.append_value::<StringType>(&sv); + } + + #[test] + #[should_panic] + fn get_string_unsupported() { + let mv = MutableVector::new(1024, &ArrowDataType::Utf8); + mv.value::<StringType>(0); + } +} diff --git a/core/src/common/vector.rs b/core/src/common/vector.rs new file mode 100644 index 0000000000..1afb1e78fd --- /dev/null +++ b/core/src/common/vector.rs @@ -0,0 +1,523 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 crate::{ + common::{bit, ValueGetter}, + BoolType, DataType, TypeTrait, BITS_PER_BYTE, STRING_VIEW_LEN, STRING_VIEW_PREFIX_LEN, +}; +use arrow::{ + array::{Array, ArrayRef}, + buffer::{Buffer, MutableBuffer}, + datatypes::DataType as ArrowDataType, +}; +use arrow_data::ArrayData; + +/// A vector that holds elements of plain types (i.e., no nested type such as list, map, struct). +pub struct PlainVector { + /// The data type for elements in this vector + data_type: DataType, + /// Total number of values in this vector + num_values: usize, + /// Total number of nulls in this vector. Must <= `num_values`. + num_nulls: usize, + /// The value buffer + value_buffer: ValueBuffer, + /// Number of bytes for each element in the vector. For variable length types such as string + /// and binary, this will be the size of [`StringView`] which is always 16 bytes. + value_size: usize, + /// Offsets into buffers + offset: usize, + /// The validity buffer. If empty, all values in this vector are not null. + validity_buffer: Option<Buffer>, + /// Whether this vector is dictionary encoded + is_dictionary: bool, + /// Indices (or dictionary keys) when `is_dictionary` is true. Otherwise, this is always + /// an empty vector. + indices: IndexBuffer, +} + +impl<T: TypeTrait> ValueGetter<T> for PlainVector { + default fn value(&self, idx: usize) -> T::Native { + let offset = self.offset(idx); + unsafe { + let ptr = self.value_buffer.as_ptr() as *const T::Native; + *ptr.add(offset) + } + } +} + +impl ValueGetter<BoolType> for PlainVector { + fn value(&self, idx: usize) -> bool { + let offset = self.offset(idx); + unsafe { bit::get_bit_raw(self.value_buffer.as_ptr(), offset) } + } +} + +impl PlainVector { + /// Returns the data type of this vector. + pub fn data_type(&self) -> &DataType { + &self.data_type + } + + /// Returns the total number of elements in this vector. + pub fn num_values(&self) -> usize { + self.num_values + } + + /// Returns the total number of nulls in this vector. + pub fn num_nulls(&self) -> usize { + self.num_nulls + } + + /// Whether there is any null in this vector. + pub fn has_null(&self) -> bool { + self.num_nulls > 0 + } + + /// Whether the element at `idx` is null. + pub fn is_null(&self, idx: usize) -> bool { + if let Some(validity_buffer) = &self.validity_buffer { + unsafe { + return !bit::get_bit_raw(validity_buffer.as_ptr(), self.offset + idx); + } + } + + false + } + + #[inline(always)] + pub fn value<T: TypeTrait>(&self, idx: usize) -> T::Native { + <dyn ValueGetter<T>>::value(self, idx) + } + + #[inline(always)] + fn offset(&self, idx: usize) -> usize { + let idx = self.offset + idx; + if self.is_dictionary { + self.indices.get(idx) + } else { + idx + } + } +} + +impl From<ArrayData> for PlainVector { + fn from(data: ArrayData) -> Self { + assert!(!data.buffers().is_empty(), "expected at least one buffer"); + let arrow_dt = data.data_type(); + let dt: DataType = arrow_dt.into(); + let is_dictionary = matches!(arrow_dt, ArrowDataType::Dictionary(_, _)); + + let mut value_buffers = data.buffers(); + let mut indices = IndexBuffer::empty(); + let validity_buffer = data.nulls().map(|nb| nb.buffer().clone()); + + if is_dictionary { + // in case of dictionary data, the dictionary values are stored in child data, while + // dictionary keys are stored in `value_buffer`. + assert_eq!( + data.child_data().len(), + 1, + "child data should contain a single array" + ); + let child_data = &data.child_data()[0]; + indices = IndexBuffer::new(value_buffers[0].clone(), data.len() + data.offset()); + value_buffers = child_data.buffers(); + } + + let value_size = dt.kind().type_size() / BITS_PER_BYTE; + let value_buffer = ValueBuffer::new(&dt, value_buffers.to_vec(), data.len()); + + Self { + data_type: dt, + num_values: data.len(), + num_nulls: data.null_count(), + value_buffer, + value_size, + offset: data.offset(), + validity_buffer, + is_dictionary, + indices, + } + } +} + +impl From<ArrayRef> for PlainVector { + fn from(value: ArrayRef) -> Self { + Self::from(value.into_data()) + } +} + +struct ValueBuffer { + ptr: *const u8, + /// Keep the `ptr` alive + original_buffers: Vec<Buffer>, +} + +impl ValueBuffer { + pub fn new(dt: &DataType, buffers: Vec<Buffer>, len: usize) -> Self { + if matches!(dt, DataType::String | DataType::Binary) { + assert_eq!( + 2, + buffers.len(), + "expected two buffers (offset, value) for string/binary" + ); + + let mut string_view_buf = MutableBuffer::from_len_zeroed(len * 16); + let buf_mut = string_view_buf.as_mut_ptr(); + + let offsets = buffers[0].as_ptr() as *const i32; + let values = buffers[1].as_ptr(); + + let mut dst_offset = 0; + let mut start = 0; + unsafe { + for i in 0..len { + // StringView format: + // - length (4 bytes) + // - first 4 bytes of the string/binary (4 bytes) + // - pointer to the string/binary (8 bytes) + let end = *offsets.add(i + 1); + let len = end - start; + let value = values.add(start as usize); + *(buf_mut.add(dst_offset) as *mut i32) = len; + if len >= STRING_VIEW_PREFIX_LEN as i32 { + // only store prefix if the string has at least 4 bytes, otherwise, we'll + // zero pad the bytes. + std::ptr::copy_nonoverlapping( + value, + buf_mut.add(dst_offset + STRING_VIEW_PREFIX_LEN), + STRING_VIEW_PREFIX_LEN, + ); + } + *(buf_mut.add(dst_offset + STRING_VIEW_PREFIX_LEN + 4) as *mut usize) = + value as usize; + start = end; + dst_offset += STRING_VIEW_LEN; + } + } + + let string_buffer: Buffer = string_view_buf.into(); + let ptr = string_buffer.as_ptr(); + + Self { + ptr, + original_buffers: vec![string_buffer, buffers[1].clone()], + } + } else { + let ptr = buffers[0].as_ptr(); + Self { + ptr, + original_buffers: buffers, + } + } + } + + /// Returns the raw pointer for the data in this value buffer. + /// NOTE: caller of this should NOT store the raw pointer to avoid dangling pointers. + pub fn as_ptr(&self) -> *const u8 { + self.ptr + } +} + +struct IndexBuffer { + ptr: *const u8, + /// Keep the `ptr` alive. + buf: Option<Buffer>, + /// Total number of elements in the index buffer + len: usize, +} + +impl IndexBuffer { + pub fn new(buf: Buffer, len: usize) -> Self { + let ptr = buf.as_ptr(); + Self { + buf: Some(buf), + ptr, + len, + } + } + + pub fn empty() -> Self { + Self { + buf: None, + ptr: std::ptr::null(), + len: 0, + } + } + + #[inline] + pub fn get(&self, i: usize) -> usize { + debug_assert!(i < self.len); + unsafe { + let ptr = self.ptr as *const i32; + *ptr.add(i) as usize + } + } +} + +#[cfg(test)] +mod tests { + use crate::{ + BoolType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, + NativeEqual, ShortType, StringType, TimestampType, TypeTrait, STRING_VIEW_PREFIX_LEN, + }; + + use crate::common::vector::PlainVector; + use arrow::{ + array::{ + Array, BooleanArray, Date32Array, Decimal128Array, Float32Array, Float64Array, + Int16Array, Int32Array, Int8Array, StringArray, + }, + buffer::Buffer, + datatypes::{DataType as ArrowDataType, ToByteSlice}, + }; + use arrow_array::TimestampMicrosecondArray; + use arrow_data::ArrayData; + + #[test] + fn primitive_no_null() { + let arr = Int32Array::from(vec![0, 1, 2, 3, 4]); + let vector = PlainVector::from(arr.into_data()); + + assert_eq!(5, vector.num_values()); + assert_eq!(0, vector.num_nulls()); + assert_eq!(4, vector.value_size); + assert!(vector.validity_buffer.is_none()); + + for i in 0..5 { + assert!(!vector.is_null(i)); + assert_eq!(i as i32, vector.value::<IntegerType>(i)) + } + } + + fn check_answer<T: TypeTrait>(expected: &[Option<T::Native>], actual: &PlainVector) { + assert_eq!(expected.len(), actual.num_values()); + let nulls = expected + .iter() + .filter(|v| v.is_none()) + .collect::<Vec<&Option<T::Native>>>(); + assert_eq!(nulls.len(), actual.num_nulls()); + + for i in 0..expected.len() { + if let Some(v) = expected[i] { + assert!(!actual.is_null(i)); + assert!(v.is_equal(&actual.value::<T>(i))); + } else { + assert!(actual.is_null(i)); + } + } + } + + #[test] + fn primitive_with_nulls() { + let data = vec![Some(0), None, Some(2), None, Some(4)]; + let arr = TimestampMicrosecondArray::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<TimestampType>(&data, &vector); + } + + #[test] + fn primitive_with_offsets_nulls() { + let arr = Int32Array::from(vec![Some(0), None, Some(2), None, Some(4), None, Some(7)]); + let data = arr.into_data(); + let vector = PlainVector::from(data.slice(2, 3)); + + assert_eq!(3, vector.num_values()); + assert_eq!(1, vector.num_nulls()); + + for i in 0..2 { + if i % 2 == 0 { + assert!(!vector.is_null(i)); + assert_eq!((i + 2) as i32, vector.value::<IntegerType>(i)); + } else { + assert!(vector.is_null(i)); + } + } + } + + #[test] + fn primitive_dictionary() { + let value_data = ArrayData::builder(ArrowDataType::Int8) + .len(8) + .add_buffer(Buffer::from( + &[10_i8, 11, 12, 13, 14, 15, 16, 17].to_byte_slice(), + )) + .build() + .unwrap(); + + // Construct a buffer for value offsets, for the nested array: + let keys = Buffer::from(&[2_i32, 3, 4].to_byte_slice()); + + // Construct a dictionary array from the above two + let key_type = ArrowDataType::Int32; + let value_type = ArrowDataType::Int8; + let dict_data_type = ArrowDataType::Dictionary(Box::new(key_type), Box::new(value_type)); + let dict_data = ArrayData::builder(dict_data_type) + .len(3) + .add_buffer(keys) + .add_child_data(value_data) + .build() + .unwrap(); + + let vector = PlainVector::from(dict_data); + + assert_eq!(DataType::Byte, *vector.data_type()); + assert_eq!(3, vector.num_values()); + assert_eq!(0, vector.num_nulls()); + assert!(!vector.has_null()); + assert_eq!(12, vector.value::<ByteType>(0)); + assert_eq!(13, vector.value::<ByteType>(1)); + assert_eq!(14, vector.value::<ByteType>(2)); + } + + #[test] + fn bools() { + let data = vec![Some(true), None, Some(false), None, Some(true)]; + let arr = BooleanArray::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<BoolType>(&data, &vector); + } + + #[test] + fn bytes() { + let data = vec![Some(4_i8), None, None, Some(5_i8), Some(7_i8)]; + let arr = Int8Array::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<ByteType>(&data, &vector); + } + + #[test] + fn shorts() { + let data = vec![Some(4_i16), None, None, Some(-40_i16), Some(-3_i16)]; + let arr = Int16Array::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<ShortType>(&data, &vector); + } + + #[test] + fn floats() { + let data = vec![ + Some(4.0_f32), + Some(-0.0_f32), + Some(-3.0_f32), + Some(0.0_f32), + Some(std::f32::consts::PI), + ]; + let arr = Float32Array::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<FloatType>(&data, &vector); + } + + #[test] + fn doubles() { + let data = vec![ + None, + Some(std::f64::consts::PI), + Some(4.0_f64), + Some(f64::NAN), + ]; + let arr = Float64Array::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<DoubleType>(&data, &vector); + } + + #[test] + fn decimals() { + let data = vec![Some(1_i128), None, None, Some(i128::MAX)]; + let arr = Decimal128Array::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<DecimalType>(&data, &vector); + } + + #[test] + fn timestamps() { + // 1: 00:00:00.001 + // 37800005: 10:30:00.005 + // 86399210: 23:59:59.210 + let data = vec![Some(1), None, Some(37_800_005), Some(86_399_210)]; + let arr = TimestampMicrosecondArray::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<TimestampType>(&data, &vector); + } + + #[test] + fn dates() { + let data = vec![Some(100), None, Some(200), None]; + let arr = Date32Array::from(data.clone()); + let vector = PlainVector::from(arr.into_data()); + + check_answer::<DateType>(&data, &vector); + } + + #[test] + fn string_no_nulls() { + let values: Vec<&str> = vec!["hello", "", "comet"]; + let arr = StringArray::from(values.clone()); + + let vector = PlainVector::from(arr.into_data()); + assert_eq!(3, vector.num_values()); + assert_eq!(0, vector.num_nulls()); + + for i in 0..values.len() { + let expected = values[i]; + let actual = vector.value::<StringType>(i); + assert_eq!(expected.len(), actual.len as usize); + if expected.len() >= STRING_VIEW_PREFIX_LEN { + assert_eq!( + &expected[..STRING_VIEW_PREFIX_LEN], + String::from_utf8_lossy(&actual.prefix) + ); + } + assert_eq!(expected, actual.as_utf8_str()); + } + } + + #[test] + fn string_with_nulls() { + let data = [Some("hello"), None, Some("comet")]; + let arr = StringArray::from(data.to_vec().clone()); + + let vector = PlainVector::from(arr.into_data()); + assert_eq!(3, vector.num_values()); + assert_eq!(1, vector.num_nulls()); + + for i in 0..data.len() { + if data[i].is_none() { + assert!(vector.is_null(i)); + } else { + let expected = data[i].unwrap(); + let actual = vector.value::<StringType>(i); + if expected.len() >= STRING_VIEW_PREFIX_LEN { + assert_eq!( + &expected[..STRING_VIEW_PREFIX_LEN], + String::from_utf8_lossy(&actual.prefix) + ); + } + assert_eq!(expected, actual.as_utf8_str()); + } + } + } +} diff --git a/core/src/data_type.rs b/core/src/data_type.rs new file mode 100644 index 0000000000..b275de1c65 --- /dev/null +++ b/core/src/data_type.rs @@ -0,0 +1,241 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::datatypes::DataType as ArrowDataType; +use arrow_schema::TimeUnit; +use std::{cmp, fmt::Debug}; + +#[derive(Debug, PartialEq)] +pub enum DataType { + Boolean, + Byte, + Short, + Integer, + Long, + Float, + Double, + Decimal(u8, i8), + String, + Binary, + Timestamp, + Date, +} + +impl From<&ArrowDataType> for DataType { + fn from(dt: &ArrowDataType) -> Self { + match dt { + ArrowDataType::Boolean => DataType::Boolean, + ArrowDataType::Int8 => DataType::Byte, + ArrowDataType::Int16 => DataType::Short, + ArrowDataType::Int32 => DataType::Integer, + ArrowDataType::Int64 => DataType::Long, + ArrowDataType::Float32 => DataType::Float, + ArrowDataType::Float64 => DataType::Double, + ArrowDataType::Decimal128(precision, scale) => DataType::Decimal(*precision, *scale), + ArrowDataType::Utf8 => DataType::String, + ArrowDataType::Binary => DataType::Binary, + // Spark always store timestamp in micro seconds + ArrowDataType::Timestamp(TimeUnit::Microsecond, _) => DataType::Timestamp, + ArrowDataType::Date32 => DataType::Date, + ArrowDataType::Dictionary(key_dt, value_dt) if is_valid_key_type(key_dt) => { + Self::from(value_dt.as_ref()) + } + dt => panic!("unsupported Arrow data type: {:?}", dt), + } + } +} + +impl DataType { + pub fn kind(&self) -> TypeKind { + match self { + DataType::Boolean => TypeKind::Boolean, + DataType::Byte => TypeKind::Byte, + DataType::Short => TypeKind::Short, + DataType::Integer => TypeKind::Integer, + DataType::Long => TypeKind::Long, + DataType::Float => TypeKind::Float, + DataType::Double => TypeKind::Double, + DataType::Decimal(_, _) => TypeKind::Decimal, + DataType::String => TypeKind::String, + DataType::Binary => TypeKind::Binary, + DataType::Timestamp => TypeKind::Timestamp, + DataType::Date => TypeKind::Date, + } + } +} + +/// Comet only use i32 as dictionary key +fn is_valid_key_type(dt: &ArrowDataType) -> bool { + matches!(dt, ArrowDataType::Int32) +} + +/// Unlike [`DataType`], [`TypeKind`] doesn't carry extra information about the type itself, such as +/// decimal precision & scale. Instead, it is merely a token that is used to do runtime case +/// analysis depending on the actual type. It can be obtained from a `TypeTrait` generic parameter. +#[derive(Debug, PartialEq)] +pub enum TypeKind { + Boolean, + Byte, + Short, + Integer, + Long, + Float, + Double, + Decimal, + String, + Binary, + Timestamp, + Date, +} + +pub const BITS_PER_BYTE: usize = 8; + +impl TypeKind { + /// Returns the size of this type, in number of bits. + pub fn type_size(&self) -> usize { + match self { + TypeKind::Boolean => 1, + TypeKind::Byte => BITS_PER_BYTE, + TypeKind::Short => BITS_PER_BYTE * 2, + TypeKind::Integer | TypeKind::Float => BITS_PER_BYTE * 4, + TypeKind::Long | TypeKind::Double => BITS_PER_BYTE * 8, + TypeKind::Decimal => BITS_PER_BYTE * 16, + TypeKind::String | TypeKind::Binary => BITS_PER_BYTE * 16, + TypeKind::Timestamp => BITS_PER_BYTE * 8, + TypeKind::Date => BITS_PER_BYTE * 4, + } + } +} + +pub const STRING_VIEW_LEN: usize = 16; // StringView is stored using 16 bytes +pub const STRING_VIEW_PREFIX_LEN: usize = 4; // String prefix in StringView is stored using 4 bytes + +#[repr(C, align(16))] +#[derive(Clone, Copy, Debug)] +pub struct StringView { + pub len: u32, + pub prefix: [u8; STRING_VIEW_PREFIX_LEN], + pub ptr: usize, +} + +impl StringView { + pub fn as_utf8_str(&self) -> &str { + unsafe { + let slice = std::slice::from_raw_parts(self.ptr as *const u8, self.len as usize); + std::str::from_utf8_unchecked(slice) + } + } +} + +impl Default for StringView { + fn default() -> Self { + Self { + len: 0, + prefix: [0; STRING_VIEW_PREFIX_LEN], + ptr: 0, + } + } +} + +impl PartialEq for StringView { + fn eq(&self, other: &Self) -> bool { + if self.len != other.len { + return false; + } + if self.prefix != other.prefix { + return false; + } + self.as_utf8_str() == other.as_utf8_str() + } +} + +pub trait NativeEqual { + fn is_equal(&self, other: &Self) -> bool; +} + +macro_rules! make_native_equal { + ($native_ty:ty) => { + impl NativeEqual for $native_ty { + fn is_equal(&self, other: &Self) -> bool { + self == other + } + } + }; +} + +make_native_equal!(bool); +make_native_equal!(i8); +make_native_equal!(i16); +make_native_equal!(i32); +make_native_equal!(i64); +make_native_equal!(i128); +make_native_equal!(StringView); + +impl NativeEqual for f32 { + fn is_equal(&self, other: &Self) -> bool { + self.total_cmp(other) == cmp::Ordering::Equal + } +} + +impl NativeEqual for f64 { + fn is_equal(&self, other: &Self) -> bool { + self.total_cmp(other) == cmp::Ordering::Equal + } +} +pub trait NativeType: Debug + Default + Copy + NativeEqual {} + +impl NativeType for bool {} +impl NativeType for i8 {} +impl NativeType for i16 {} +impl NativeType for i32 {} +impl NativeType for i64 {} +impl NativeType for i128 {} +impl NativeType for f32 {} +impl NativeType for f64 {} +impl NativeType for StringView {} + +/// A trait for Comet data type. This should only be used as generic parameter during method +/// invocations. +pub trait TypeTrait: 'static { + type Native: NativeType; + fn type_kind() -> TypeKind; +} + +macro_rules! make_type_trait { + ($name:ident, $native_ty:ty, $kind:path) => { + pub struct $name {} + impl TypeTrait for $name { + type Native = $native_ty; + fn type_kind() -> TypeKind { + $kind + } + } + }; +} + +make_type_trait!(BoolType, bool, TypeKind::Boolean); +make_type_trait!(ByteType, i8, TypeKind::Byte); +make_type_trait!(ShortType, i16, TypeKind::Short); +make_type_trait!(IntegerType, i32, TypeKind::Integer); +make_type_trait!(LongType, i64, TypeKind::Long); +make_type_trait!(FloatType, f32, TypeKind::Float); +make_type_trait!(DoubleType, f64, TypeKind::Double); +make_type_trait!(DecimalType, i128, TypeKind::Decimal); +make_type_trait!(StringType, StringView, TypeKind::String); +make_type_trait!(BinaryType, StringView, TypeKind::Binary); +make_type_trait!(TimestampType, i64, TypeKind::Timestamp); +make_type_trait!(DateType, i32, TypeKind::Date); diff --git a/core/src/errors.rs b/core/src/errors.rs new file mode 100644 index 0000000000..5b53c654eb --- /dev/null +++ b/core/src/errors.rs @@ -0,0 +1,820 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Common Parquet errors and macros. + +use arrow::error::ArrowError; +use datafusion_common::DataFusionError; +use jni::errors::{Exception, ToException}; +use regex::Regex; +use std::{ + any::Any, + convert, + fmt::Write, + panic::{catch_unwind, UnwindSafe}, + result, str, + str::Utf8Error, + sync::{Arc, Mutex}, +}; + +// This is just a pointer. We'll be returning it from our function. We +// can't return one of the objects with lifetime information because the +// lifetime checker won't let us. +use jni::sys::{jboolean, jbyte, jchar, jdouble, jfloat, jint, jlong, jobject, jshort}; + +use crate::execution::operators::ExecutionError; +use jni::JNIEnv; +use lazy_static::lazy_static; +use parquet::errors::ParquetError; +use thiserror::Error; + +lazy_static! { + static ref PANIC_BACKTRACE: Arc<Mutex<Option<String>>> = Arc::new(Mutex::new(None)); +} + +#[derive(thiserror::Error, Debug)] +pub enum CometError { + #[error("Configuration Error: {0}")] + Config(String), + + #[error("{0}")] + NullPointer(String), + + #[error("Out of bounds{0}")] + IndexOutOfBounds(usize), + + #[error("Comet Internal Error: {0}")] + Internal(String), + + #[error(transparent)] + Arrow { + #[from] + source: ArrowError, + }, + + #[error(transparent)] + Parquet { + #[from] + source: ParquetError, + }, + + #[error(transparent)] + Expression { + #[from] + source: ExpressionError, + }, + + #[error(transparent)] + Execution { + #[from] + source: ExecutionError, + }, + + #[error(transparent)] + IO { + #[from] + source: std::io::Error, + }, + + #[error(transparent)] + NumberIntFormat { + #[from] + source: std::num::ParseIntError, + }, + + #[error(transparent)] + NumberFloatFormat { + #[from] + source: std::num::ParseFloatError, + }, + + #[error(transparent)] + Format { + #[from] + source: Utf8Error, + }, + + #[error(transparent)] + JNI { + #[from] + source: jni::errors::Error, + }, + + #[error("{msg}")] + Panic { msg: String }, + + #[error(transparent)] + DataFusion { + #[from] + source: DataFusionError, + }, +} + +pub fn init() { + std::panic::set_hook(Box::new(|_panic_info| { + // Capture the backtrace for a panic + *PANIC_BACKTRACE.lock().unwrap() = + Some(std::backtrace::Backtrace::force_capture().to_string()); + })); +} + +/// Converts the results from `panic::catch_unwind` (e.g. a panic) to a `CometError` +impl convert::From<Box<dyn Any + Send>> for CometError { + fn from(e: Box<dyn Any + Send>) -> Self { + CometError::Panic { + msg: match e.downcast_ref::<&str>() { + Some(s) => s.to_string(), + None => match e.downcast_ref::<String>() { + Some(msg) => msg.to_string(), + None => "unknown panic".to_string(), + }, + }, + } + } +} + +impl From<CometError> for DataFusionError { + fn from(value: CometError) -> Self { + match value { + CometError::DataFusion { source } => source, + _ => DataFusionError::Execution(value.to_string()), + } + } +} + +impl jni::errors::ToException for CometError { + fn to_exception(&self) -> Exception { + match self { + CometError::IndexOutOfBounds(..) => Exception { + class: "java/lang/IndexOutOfBoundsException".to_string(), + msg: self.to_string(), + }, + CometError::NullPointer(..) => Exception { + class: "java/lang/NullPointerException".to_string(), + msg: self.to_string(), + }, + CometError::NumberIntFormat { source: s } => Exception { + class: "java/lang/NumberFormatException".to_string(), + msg: s.to_string(), + }, + CometError::NumberFloatFormat { source: s } => Exception { + class: "java/lang/NumberFormatException".to_string(), + msg: s.to_string(), + }, + CometError::IO { .. } => Exception { + class: "java/io/IOException".to_string(), + msg: self.to_string(), + }, + CometError::Parquet { .. } => Exception { + class: "org/apache/comet/ParquetRuntimeException".to_string(), + msg: self.to_string(), + }, + _other => Exception { + class: "org/apache/comet/CometRuntimeException".to_string(), + msg: self.to_string(), + }, + } + } +} + +/// Error returned when there is an error during executing an expression. +#[derive(thiserror::Error, Debug)] +pub enum ExpressionError { + /// Simple error + #[error("General expression error with reason {0}.")] + General(String), + + /// Deserialization error + #[error("Fail to deserialize to native expression with reason {0}.")] + Deserialize(String), + + /// Evaluation error + #[error("Fail to evaluate native expression with reason {0}.")] + Evaluation(String), + + /// Error when processing Arrow array. + #[error("Fail to process Arrow array with reason {0}.")] + ArrowError(String), +} + +/// A specialized `Result` for Comet errors. +pub type CometResult<T> = result::Result<T, CometError>; + +// ---------------------------------------------------------------------- +// Convenient macros for different errors + +macro_rules! general_err { + ($fmt:expr, $($args:expr),*) => (crate::CometError::from(parquet::errors::ParquetError::General(format!($fmt, $($args),*)))); +} + +/// Returns the "default value" for a type. This is used for JNI code in order to facilitate +/// returning a value in cases where an exception is thrown. This value will never be used, as the +/// JVM will note the pending exception. +/// +/// Default values are often some kind of initial value, identity value, or anything else that +/// may make sense as a default. +/// +/// NOTE: We can't just use [Default] since both the trait and the object are defined in other +/// crates. +/// See [Rust Compiler Error Index - E0117](https://doc.rust-lang.org/error-index.html#E0117) +pub trait JNIDefault { + fn default() -> Self; +} + +impl JNIDefault for jboolean { + fn default() -> jboolean { + 0 + } +} + +impl JNIDefault for jbyte { + fn default() -> jbyte { + 0 + } +} + +impl JNIDefault for jchar { + fn default() -> jchar { + 0 + } +} + +impl JNIDefault for jdouble { + fn default() -> jdouble { + 0.0 + } +} + +impl JNIDefault for jfloat { + fn default() -> jfloat { + 0.0 + } +} + +impl JNIDefault for jint { + fn default() -> jint { + 0 + } +} + +impl JNIDefault for jlong { + fn default() -> jlong { + 0 + } +} + +/// The "default value" for all returned objects, such as [jstring], [jlongArray], etc. +impl JNIDefault for jobject { + fn default() -> jobject { + std::ptr::null_mut() + } +} + +impl JNIDefault for jshort { + fn default() -> jshort { + 0 + } +} + +impl JNIDefault for () { + fn default() {} +} + +// Unwrap the result returned from `panic::catch_unwind` when `Ok`, otherwise throw a +// `RuntimeException` back to the calling Java. Since a return result is required, use `JNIDefault` +// to create a reasonable result. This returned default value will be ignored due to the exception. +pub fn unwrap_or_throw_default<T: JNIDefault>( + env: &JNIEnv, + result: std::result::Result<T, CometError>, +) -> T { + match result { + Ok(value) => value, + Err(err) => { + let backtrace = match err { + CometError::Panic { msg: _ } => PANIC_BACKTRACE.lock().unwrap().take(), + _ => None, + }; + throw_exception(env, &err, backtrace); + T::default() + } + } +} + +fn throw_exception<E: ToException>(env: &JNIEnv, error: &E, backtrace: Option<String>) { + // If there isn't already an exception? + if env.exception_check().is_ok() { + // ... then throw new exception + let exception = error.to_exception(); + match backtrace { + Some(backtrace_string) => env.throw_new( + exception.class, + to_stacktrace_string(exception.msg, backtrace_string).unwrap(), + ), + _ => env.throw_new(exception.class, exception.msg), + } + .expect("Thrown exception") + } +} + +#[derive(Debug, Error)] +enum StacktraceError { + #[error("Unable to initialize message: {0}")] + Message(String), + #[error("Unable to initialize backtrace regex: {0}")] + Regex(#[from] regex::Error), + #[error("Required field missing: {0}")] + Required_Field(String), + #[error("Unable to format stacktrace element: {0}")] + Element(#[from] std::fmt::Error), +} + +fn to_stacktrace_string(msg: String, backtrace_string: String) -> Result<String, StacktraceError> { + let mut res = String::new(); + write!(&mut res, "{}", msg).map_err(|error| StacktraceError::Message(error.to_string()))?; + + // Use multi-line mode and named capture groups to identify the following stacktrace fields: + // - dc = declaredClass + // - mn = methodName + // - fn = fileName (optional) + // - line = file line number (optional) + // - col = file col number within the line (optional) + let re = Regex::new( + r"(?m)^\s*\d+: (?<dc>.*?)(?<mn>[^:]+)\n(\s*at\s+(?<fn>[^:]+):(?<line>\d+):(?<col>\d+)$)?", + )?; + for c in re.captures_iter(backtrace_string.as_str()) { + write!( + &mut res, + "\n at {}{}({}:{})", + c.name("dc") + .ok_or_else(|| StacktraceError::Required_Field("declared class".to_string()))? + .as_str(), + c.name("mn") + .ok_or_else(|| StacktraceError::Required_Field("method name".to_string()))? + .as_str(), + // There are internal calls within the backtrace that don't provide file information + c.name("fn").map(|m| m.as_str()).unwrap_or("__internal__"), + c.name("line") + .map(|m| m.as_str().parse().expect("numeric line number")) + .unwrap_or(0) + )?; + } + + Ok(res) +} + +fn flatten<T, E>(result: Result<Result<T, E>, E>) -> Result<T, E> { + result.and_then(convert::identity) +} + +// It is currently undefined behavior to unwind from Rust code into foreign code, so we can wrap +// our JNI functions and turn these panics into a `RuntimeException`. +pub fn try_or_throw<T, F>(env: JNIEnv, f: F) -> T +where + T: JNIDefault, + F: FnOnce() -> T + UnwindSafe, +{ + unwrap_or_throw_default(&env, catch_unwind(f).map_err(CometError::from)) +} + +// This is a duplicate of `try_unwrap_or_throw`, which is used to work around Arrow's lack of +// `UnwindSafe` handling. +pub fn try_assert_unwind_safe_or_throw<T, F>(env: JNIEnv, f: F) -> T +where + T: JNIDefault, + F: FnOnce() -> Result<T, CometError>, +{ + unwrap_or_throw_default( + &env, + flatten(catch_unwind(std::panic::AssertUnwindSafe(f)).map_err(CometError::from)), + ) +} + +// It is currently undefined behavior to unwind from Rust code into foreign code, so we can wrap +// our JNI functions and turn these panics into a `RuntimeException`. +pub fn try_unwrap_or_throw<T, F>(env: JNIEnv, f: F) -> T +where + T: JNIDefault, + F: FnOnce() -> Result<T, CometError> + UnwindSafe, +{ + unwrap_or_throw_default(&env, flatten(catch_unwind(f).map_err(CometError::from))) +} + +#[cfg(test)] +mod tests { + use super::*; + use std::{ + fs::File, + io, + io::Read, + path::PathBuf, + sync::{Arc, Once}, + }; + + use jni::{ + objects::{JClass, JObject, JString, JThrowable}, + sys::{jintArray, jstring}, + AttachGuard, InitArgsBuilder, JNIEnv, JNIVersion, JavaVM, + }; + + use assertables::{ + assert_contains, assert_contains_as_result, assert_starts_with, + assert_starts_with_as_result, + }; + + pub fn jvm() -> &'static Arc<JavaVM> { + static mut JVM: Option<Arc<JavaVM>> = None; + static INIT: Once = Once::new(); + + // Capture panic backtraces + init(); + + INIT.call_once(|| { + // Add common classes to the classpath in so that we can find CometException + let mut common_classes = PathBuf::from(env!("CARGO_MANIFEST_DIR")); + common_classes.push("../common/target/classes"); + let mut class_path = common_classes + .as_path() + .to_str() + .expect("common classes as an str") + .to_string(); + class_path.insert_str(0, "-Djava.class.path="); + + // Build the VM properties + let jvm_args = InitArgsBuilder::new() + // Pass the JNI API version (default is 8) + .version(JNIVersion::V8) + // You can additionally pass any JVM options (standard, like a system property, + // or VM-specific). + // Here we enable some extra JNI checks useful during development + .option("-Xcheck:jni") + .option(class_path.as_str()) + .build() + .unwrap_or_else(|e| panic!("{:#?}", e)); + + let jvm = JavaVM::new(jvm_args).unwrap_or_else(|e| panic!("{:#?}", e)); + + unsafe { + JVM = Some(Arc::new(jvm)); + } + }); + + unsafe { JVM.as_ref().unwrap() } + } + + fn attach_current_thread() -> AttachGuard<'static> { + jvm().attach_current_thread().expect("Unable to attach JVM") + } + + #[test] + pub fn error_from_panic() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + try_or_throw(env, || { + panic!("oops!"); + }); + + assert_pending_java_exception_detailed( + &env, + Some("java/lang/RuntimeException"), + Some("oops!"), + ); + } + + // Verify that functions that return an object are handled correctly. This is basically + // a test of the "happy path". + #[test] + pub fn object_result() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + let clazz = env.find_class("java/lang/Object").unwrap(); + let input = env.new_string("World".to_string()).unwrap(); + let actual = Java_Errors_hello(env, clazz, input); + + let actual_string = String::from(env.get_string(actual.into()).unwrap().to_str().unwrap()); + assert_eq!("Hello, World!", actual_string); + } + + // Verify that functions that return an object can handle throwing exceptions. The test + // causes an exception by passing a `null` where a string value is expected. + #[test] + pub fn object_panic_exception() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + // Class java.lang.object is just a stand-in + let class = env.find_class("java/lang/Object").unwrap(); + let input = JString::from(JObject::null()); + let _actual = Java_Errors_hello(env, class, input); + + assert!(env.exception_check().unwrap()); + let exception = env.exception_occurred().expect("Unable to get exception"); + env.exception_clear().unwrap(); + + assert_exception_message_with_stacktrace( + &env, + exception, + "Couldn't get java string!: NullPtr(\"get_string obj argument\")", + "at Java_Errors_hello(", + ); + } + + // Verify that functions that return an native time are handled correctly. This is basically + // a test of the "happy path". + #[test] + pub fn jlong_result() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + // Class java.lang.object is just a stand-in + let class = env.find_class("java/lang/Object").unwrap(); + let a: jlong = 6; + let b: jlong = 3; + let actual = Java_Errors_div(env, class, a, b); + + assert_eq!(2, actual); + } + + // Verify that functions that return an array can handle throwing exceptions. The test + // causes an exception by dividing by zero. + #[test] + pub fn jlong_panic_exception() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + // Class java.lang.object is just a stand-in + let class = env.find_class("java/lang/Object").unwrap(); + let a: jlong = 6; + let b: jlong = 0; + let _actual = Java_Errors_div(env, class, a, b); + + assert_pending_java_exception_detailed( + &env, + Some("java/lang/RuntimeException"), + Some("attempt to divide by zero"), + ); + } + + // Verify that functions that return an native time are handled correctly. This is basically + // a test of the "happy path". + #[test] + pub fn jlong_result_ok() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + // Class java.lang.object is just a stand-in + let class = env.find_class("java/lang/Object").unwrap(); + let a: JString = env.new_string("9".to_string()).unwrap(); + let b: JString = env.new_string("3".to_string()).unwrap(); + let actual = Java_Errors_div_with_parse(env, class, a, b); + + assert_eq!(3, actual); + } + + // Verify that functions that return an native time are handled correctly. This is basically + // a test of the "happy path". + #[test] + pub fn jlong_result_err() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + // Class java.lang.object is just a stand-in + let class = env.find_class("java/lang/Object").unwrap(); + let a: JString = env.new_string("NaN".to_string()).unwrap(); + let b: JString = env.new_string("3".to_string()).unwrap(); + let _actual = Java_Errors_div_with_parse(env, class, a, b); + + assert_pending_java_exception_detailed( + &env, + Some("java/lang/NumberFormatException"), + Some("invalid digit found in string"), + ); + } + + // Verify that functions that return an array are handled correctly. This is basically + // a test of the "happy path". + #[test] + pub fn jint_array_result() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + // Class java.lang.object is just a stand-in + let class = env.find_class("java/lang/Object").unwrap(); + let buf = [2, 4, 6]; + let input = env.new_int_array(3).unwrap(); + env.set_int_array_region(input, 0, &buf).unwrap(); + let actual = Java_Errors_array_div(env, class, input, 2); + + let mut buf: [i32; 3] = [0; 3]; + env.get_int_array_region(actual, 0, &mut buf).unwrap(); + assert_eq!([1, 2, 3], buf); + } + + // Verify that functions that return an array can handle throwing exceptions. The test + // causes an exception by dividing by zero. + #[test] + pub fn jint_array_panic_exception() { + let _guard = attach_current_thread(); + let env = jvm().get_env().unwrap(); + + // Class java.lang.object is just a stand-in + let class = env.find_class("java/lang/Object").unwrap(); + let buf = [2, 4, 6]; + let input = env.new_int_array(3).unwrap(); + env.set_int_array_region(input, 0, &buf).unwrap(); + let _actual = Java_Errors_array_div(env, class, input, 0); + + assert_pending_java_exception_detailed( + &env, + Some("java/lang/RuntimeException"), + Some("attempt to divide by zero"), + ); + } + + /// Test that conversion of a serialized backtrace to an equivalent stacktrace message. + /// + /// See [`object_panic_exception`] for a test which involves generating a panic and verifying + /// that the resulting stack trace includes the offending call. + #[test] + pub fn stacktrace_string() { + // Setup: Start with a backtrace that includes all of the expected scenarios, including + // cases where the file and location are not provided as part of the backtrace capture + let backtrace_string = read_resource("testdata/backtrace.txt").expect("backtrace content"); + + // Test: Reformat the serialized backtrace as a multi-line message which includes the + // backtrace formatted as a stacktrace + let stacktrace_string = + to_stacktrace_string("Some Error Message".to_string(), backtrace_string).unwrap(); + + // Verify: The message matches the expected output. Trim the expected string to remove + // the carriage return + let expected_string = read_resource("testdata/stacktrace.txt").expect("stacktrace content"); + assert_eq!(expected_string.trim(), stacktrace_string.as_str()); + } + + fn read_resource(path: &str) -> Result<String, io::Error> { + let mut path_buf = PathBuf::from(env!("CARGO_MANIFEST_DIR")); + path_buf.push(path); + + let mut f = File::open(path_buf.as_path())?; + let mut s = String::new(); + f.read_to_string(&mut s)?; + Ok(s) + } + + // Example of a simple JNI "Hello World" program. It can be used to demonstrate: + // * returning an object + // * throwing an exception from `.expect()` + #[no_mangle] + pub extern "system" fn Java_Errors_hello( + env: JNIEnv, + _class: JClass, + input: JString, + ) -> jstring { + try_or_throw(env, || { + let input: String = env + .get_string(input) + .expect("Couldn't get java string!") + .into(); + + let output = env + .new_string(format!("Hello, {}!", input)) + .expect("Couldn't create java string!"); + + output.into_inner() + }) + } + + // Example of a simple JNI function that divides. It can be used to demonstrate: + // * returning an native type + // * throwing an exception when dividing by zero + #[no_mangle] + pub extern "system" fn Java_Errors_div( + env: JNIEnv, + _class: JClass, + a: jlong, + b: jlong, + ) -> jlong { + try_or_throw(env, || a / b) + } + + #[no_mangle] + pub extern "system" fn Java_Errors_div_with_parse( + env: JNIEnv, + _class: JClass, + a: JString, + b: JString, + ) -> jlong { + try_unwrap_or_throw(env, || { + let a_value: i64 = env.get_string(a)?.to_str()?.parse()?; + let b_value: i64 = env.get_string(b)?.to_str()?.parse()?; + Ok(a_value / b_value) + }) + } + + // Example of a simple JNI function that divides. It can be used to demonstrate: + // * returning an array + // * throwing an exception when dividing by zero + #[no_mangle] + pub extern "system" fn Java_Errors_array_div( + env: JNIEnv, + _class: JClass, + input: jintArray, + divisor: jint, + ) -> jintArray { + try_or_throw(env, || { + let mut input_buf: [jint; 3] = [0; 3]; + env.get_int_array_region(input, 0, &mut input_buf).unwrap(); + + let buf = input_buf.map(|v| -> jint { v / divisor }); + + let result = env.new_int_array(3).unwrap(); + env.set_int_array_region(result, 0, &buf).unwrap(); + result + }) + } + + // Helper method that asserts there is a pending Java exception which is an `instance_of` + // `expected_type` with a message matching `expected_message` and clears it if any. + fn assert_pending_java_exception_detailed( + env: &JNIEnv, + expected_type: Option<&str>, + expected_message: Option<&str>, + ) { + assert!(env.exception_check().unwrap()); + let exception = env.exception_occurred().expect("Unable to get exception"); + env.exception_clear().unwrap(); + + if let Some(expected_type) = expected_type { + assert_exception_type(env, exception, expected_type); + } + + if let Some(expected_message) = expected_message { + assert_exception_message(env, exception, expected_message); + } + } + + // Asserts that exception is an `instance_of` `expected_type` type. + fn assert_exception_type(env: &JNIEnv, exception: JThrowable, expected_type: &str) { + if !env.is_instance_of(exception, expected_type).unwrap() { + let class: JClass = env.get_object_class(exception).unwrap(); + let name = env + .call_method(class, "getName", "()Ljava/lang/String;", &[]) + .unwrap() + .l() + .unwrap(); + let class_name: String = env.get_string(name.into()).unwrap().into(); + assert_eq!(class_name.replace('.', "/"), expected_type); + }; + } + + // Asserts that exception's message matches `expected_message`. + fn assert_exception_message(env: &JNIEnv, exception: JThrowable, expected_message: &str) { + let message = env + .call_method(exception, "getMessage", "()Ljava/lang/String;", &[]) + .unwrap() + .l() + .unwrap(); + let msg_rust: String = env.get_string(message.into()).unwrap().into(); + println!("{}", msg_rust); + // Since panics result in multi-line messages which include the backtrace, just use the + // first line. + assert_starts_with!(msg_rust, expected_message); + } + + // Asserts that exception's message matches `expected_message`. + fn assert_exception_message_with_stacktrace( + env: &JNIEnv, + exception: JThrowable, + expected_message: &str, + stacktrace_contains: &str, + ) { + let message = env + .call_method(exception, "getMessage", "()Ljava/lang/String;", &[]) + .unwrap() + .l() + .unwrap(); + let msg_rust: String = env.get_string(message.into()).unwrap().into(); + // Since panics result in multi-line messages which include the backtrace, just use the + // first line. + assert_starts_with!(msg_rust, expected_message); + + // Check that the stacktrace is included by checking for a specific element + assert_contains!(msg_rust, stacktrace_contains); + } +} diff --git a/core/src/execution/datafusion/expressions/avg.rs b/core/src/execution/datafusion/expressions/avg.rs new file mode 100644 index 0000000000..dc2b34747a --- /dev/null +++ b/core/src/execution/datafusion/expressions/avg.rs @@ -0,0 +1,340 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 crate::execution::datafusion::expressions::utils::down_cast_any_ref; +use arrow::compute::sum; +use arrow_array::{ + builder::PrimitiveBuilder, + cast::AsArray, + types::{Float64Type, Int64Type}, + Array, ArrayRef, ArrowNumericType, Int64Array, PrimitiveArray, +}; +use arrow_schema::{DataType, Field}; +use datafusion::logical_expr::{type_coercion::aggregates::avg_return_type, Accumulator}; +use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; +use datafusion_physical_expr::{ + expressions::format_state_name, AggregateExpr, EmitTo, GroupsAccumulator, PhysicalExpr, +}; +use std::{any::Any, sync::Arc}; + +use arrow_array::ArrowNativeTypeOp; + +use DataType::*; + +/// AVG aggregate expression +#[derive(Debug, Clone)] +pub struct Avg { + name: String, + expr: Arc<dyn PhysicalExpr>, + input_data_type: DataType, + result_data_type: DataType, +} + +impl Avg { + /// Create a new AVG aggregate function + pub fn new(expr: Arc<dyn PhysicalExpr>, name: impl Into<String>, data_type: DataType) -> Self { + let result_data_type = avg_return_type(&data_type).unwrap(); + + Self { + name: name.into(), + expr, + input_data_type: data_type, + result_data_type, + } + } +} + +impl AggregateExpr for Avg { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn field(&self) -> Result<Field> { + Ok(Field::new(&self.name, self.result_data_type.clone(), true)) + } + + fn create_accumulator(&self) -> Result<Box<dyn Accumulator>> { + // instantiate specialized accumulator based for the type + match (&self.input_data_type, &self.result_data_type) { + (Float64, Float64) => Ok(Box::<AvgAccumulator>::default()), + _ => not_impl_err!( + "AvgAccumulator for ({} --> {})", + self.input_data_type, + self.result_data_type + ), + } + } + + fn state_fields(&self) -> Result<Vec<Field>> { + Ok(vec![ + Field::new( + format_state_name(&self.name, "sum"), + self.input_data_type.clone(), + true, + ), + Field::new( + format_state_name(&self.name, "count"), + DataType::Int64, + true, + ), + ]) + } + + fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.expr.clone()] + } + + fn name(&self) -> &str { + &self.name + } + + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator(&self) -> Result<Box<dyn GroupsAccumulator>> { + // instantiate specialized accumulator based for the type + match (&self.input_data_type, &self.result_data_type) { + (Float64, Float64) => Ok(Box::new(AvgGroupsAccumulator::<Float64Type, _>::new( + &self.input_data_type, + |sum: f64, count: i64| Ok(sum / count as f64), + ))), + + _ => not_impl_err!( + "AvgGroupsAccumulator for ({} --> {})", + self.input_data_type, + self.result_data_type + ), + } + } +} + +impl PartialEq<dyn Any> for Avg { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| { + self.name == x.name + && self.input_data_type == x.input_data_type + && self.result_data_type == x.result_data_type + && self.expr.eq(&x.expr) + }) + .unwrap_or(false) + } +} + +/// An accumulator to compute the average +#[derive(Debug, Default)] +pub struct AvgAccumulator { + sum: Option<f64>, + count: i64, +} + +impl Accumulator for AvgAccumulator { + fn state(&self) -> Result<Vec<ScalarValue>> { + Ok(vec![ + ScalarValue::Float64(self.sum), + ScalarValue::from(self.count), + ]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = values[0].as_primitive::<Float64Type>(); + self.count += (values.len() - values.null_count()) as i64; + let v = self.sum.get_or_insert(0.); + if let Some(x) = sum(values) { + *v += x; + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + // counts are summed + self.count += sum(states[1].as_primitive::<Int64Type>()).unwrap_or_default(); + + // sums are summed + if let Some(x) = sum(states[0].as_primitive::<Float64Type>()) { + let v = self.sum.get_or_insert(0.); + *v += x; + } + Ok(()) + } + + fn evaluate(&self) -> Result<ScalarValue> { + Ok(ScalarValue::Float64( + self.sum.map(|f| f / self.count as f64), + )) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } +} + +/// An accumulator to compute the average of `[PrimitiveArray<T>]`. +/// Stores values as native types, and does overflow checking +/// +/// F: Function that calculates the average value from a sum of +/// T::Native and a total count +#[derive(Debug)] +struct AvgGroupsAccumulator<T, F> +where + T: ArrowNumericType + Send, + F: Fn(T::Native, i64) -> Result<T::Native> + Send, +{ + /// The type of the returned average + return_data_type: DataType, + + /// Count per group (use i64 to make Int64Array) + counts: Vec<i64>, + + /// Sums per group, stored as the native type + sums: Vec<T::Native>, + + /// Function that computes the final average (value / count) + avg_fn: F, +} + +impl<T, F> AvgGroupsAccumulator<T, F> +where + T: ArrowNumericType + Send, + F: Fn(T::Native, i64) -> Result<T::Native> + Send, +{ + pub fn new(return_data_type: &DataType, avg_fn: F) -> Self { + Self { + return_data_type: return_data_type.clone(), + counts: vec![], + sums: vec![], + avg_fn, + } + } +} + +impl<T, F> GroupsAccumulator for AvgGroupsAccumulator<T, F> +where + T: ArrowNumericType + Send, + F: Fn(T::Native, i64) -> Result<T::Native> + Send, +{ + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::<T>(); + let data = values.values(); + + // increment counts, update sums + self.counts.resize(total_num_groups, 0); + self.sums.resize(total_num_groups, T::default_value()); + + let iter = group_indices.iter().zip(data.iter()); + if values.null_count() == 0 { + for (&group_index, &value) in iter { + let sum = &mut self.sums[group_index]; + *sum = (*sum).add_wrapping(value); + self.counts[group_index] += 1; + } + } else { + for (idx, (&group_index, &value)) in iter.enumerate() { + if values.is_null(idx) { + continue; + } + let sum = &mut self.sums[group_index]; + *sum = (*sum).add_wrapping(value); + + self.counts[group_index] += 1; + } + } + + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to merge_batch"); + // first batch is partial sums, second is counts + let partial_sums = values[0].as_primitive::<T>(); + let partial_counts = values[1].as_primitive::<Int64Type>(); + // update counts with partial counts + self.counts.resize(total_num_groups, 0); + let iter1 = group_indices.iter().zip(partial_counts.values().iter()); + for (&group_index, &partial_count) in iter1 { + self.counts[group_index] += partial_count; + } + + // update sums + self.sums.resize(total_num_groups, T::default_value()); + let iter2 = group_indices.iter().zip(partial_sums.values().iter()); + for (&group_index, &new_value) in iter2 { + let sum = &mut self.sums[group_index]; + *sum = sum.add_wrapping(new_value); + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result<ArrayRef> { + let counts = emit_to.take_needed(&mut self.counts); + let sums = emit_to.take_needed(&mut self.sums); + let mut builder = PrimitiveBuilder::<T>::with_capacity(sums.len()); + let iter = sums.into_iter().zip(counts); + + for (sum, count) in iter { + if count != 0 { + builder.append_value((self.avg_fn)(sum, count)?) + } else { + builder.append_null(); + } + } + let array: PrimitiveArray<T> = builder.finish(); + + Ok(Arc::new(array)) + } + + // return arrays for sums and counts + fn state(&mut self, emit_to: EmitTo) -> Result<Vec<ArrayRef>> { + assert!( + matches!(emit_to, EmitTo::All), + "EmitTo::First is not supported" + ); + let counts = emit_to.take_needed(&mut self.counts); + let counts = Int64Array::new(counts.into(), None); + + let sums = emit_to.take_needed(&mut self.sums); + let sums = PrimitiveArray::<T>::new(sums.into(), None) + .with_data_type(self.return_data_type.clone()); + + Ok(vec![ + Arc::new(sums) as ArrayRef, + Arc::new(counts) as ArrayRef, + ]) + } + + fn size(&self) -> usize { + self.counts.capacity() * std::mem::size_of::<i64>() + + self.sums.capacity() * std::mem::size_of::<T>() + } +} diff --git a/core/src/execution/datafusion/expressions/avg_decimal.rs b/core/src/execution/datafusion/expressions/avg_decimal.rs new file mode 100644 index 0000000000..dc7bf1599b --- /dev/null +++ b/core/src/execution/datafusion/expressions/avg_decimal.rs @@ -0,0 +1,534 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 crate::execution::datafusion::expressions::utils::down_cast_any_ref; +use arrow::{array::BooleanBufferBuilder, buffer::NullBuffer, compute::sum}; +use arrow_array::{ + builder::PrimitiveBuilder, + cast::AsArray, + types::{Decimal128Type, Int64Type}, + Array, ArrayRef, Decimal128Array, Int64Array, PrimitiveArray, +}; +use arrow_schema::{DataType, Field}; +use datafusion::logical_expr::Accumulator; +use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; +use datafusion_physical_expr::{ + expressions::format_state_name, AggregateExpr, EmitTo, GroupsAccumulator, PhysicalExpr, +}; +use std::{any::Any, sync::Arc}; + +use arrow_array::ArrowNativeTypeOp; +use arrow_data::decimal::{ + validate_decimal_precision, MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION, +}; + +use num::Integer; +use DataType::*; + +/// AVG aggregate expression +#[derive(Debug, Clone)] +pub struct AvgDecimal { + name: String, + expr: Arc<dyn PhysicalExpr>, + sum_data_type: DataType, + result_data_type: DataType, +} + +impl AvgDecimal { + /// Create a new AVG aggregate function + pub fn new( + expr: Arc<dyn PhysicalExpr>, + name: impl Into<String>, + result_type: DataType, + sum_type: DataType, + ) -> Self { + Self { + name: name.into(), + expr, + result_data_type: result_type, + sum_data_type: sum_type, + } + } +} + +impl AggregateExpr for AvgDecimal { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn field(&self) -> Result<Field> { + Ok(Field::new(&self.name, self.result_data_type.clone(), true)) + } + + fn create_accumulator(&self) -> Result<Box<dyn Accumulator>> { + match (&self.sum_data_type, &self.result_data_type) { + (Decimal128(sum_precision, sum_scale), Decimal128(target_precision, target_scale)) => { + Ok(Box::new(AvgDecimalAccumulator::new( + *sum_scale, + *sum_precision, + *target_precision, + *target_scale, + ))) + } + _ => not_impl_err!( + "AvgDecimalAccumulator for ({} --> {})", + self.sum_data_type, + self.result_data_type + ), + } + } + + fn state_fields(&self) -> Result<Vec<Field>> { + Ok(vec![ + Field::new( + format_state_name(&self.name, "sum"), + self.sum_data_type.clone(), + true, + ), + Field::new( + format_state_name(&self.name, "count"), + DataType::Int64, + true, + ), + ]) + } + + fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.expr.clone()] + } + + fn name(&self) -> &str { + &self.name + } + + fn reverse_expr(&self) -> Option<Arc<dyn AggregateExpr>> { + None + } + + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator(&self) -> Result<Box<dyn GroupsAccumulator>> { + // instantiate specialized accumulator based for the type + match (&self.sum_data_type, &self.result_data_type) { + (Decimal128(sum_precision, sum_scale), Decimal128(target_precision, target_scale)) => { + Ok(Box::new(AvgDecimalGroupsAccumulator::new( + &self.result_data_type, + &self.sum_data_type, + *target_precision, + *target_scale, + *sum_precision, + *sum_scale, + ))) + } + _ => not_impl_err!( + "AvgDecimalGroupsAccumulator for ({} --> {})", + self.sum_data_type, + self.result_data_type + ), + } + } +} + +impl PartialEq<dyn Any> for AvgDecimal { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| { + self.name == x.name + && self.sum_data_type == x.sum_data_type + && self.result_data_type == x.result_data_type + && self.expr.eq(&x.expr) + }) + .unwrap_or(false) + } +} + +/// An accumulator to compute the average for decimals +#[derive(Debug)] +struct AvgDecimalAccumulator { + sum: Option<i128>, + count: i64, + is_empty: bool, + is_not_null: bool, + sum_scale: i8, + sum_precision: u8, + target_precision: u8, + target_scale: i8, +} + +impl AvgDecimalAccumulator { + pub fn new(sum_scale: i8, sum_precision: u8, target_precision: u8, target_scale: i8) -> Self { + Self { + sum: None, + count: 0, + is_empty: true, + is_not_null: true, + sum_scale, + sum_precision, + target_precision, + target_scale, + } + } + + fn update_single(&mut self, values: &Decimal128Array, idx: usize) { + let v = unsafe { values.value_unchecked(idx) }; + let (new_sum, is_overflow) = match self.sum { + Some(sum) => sum.overflowing_add(v), + None => (v, false), + }; + + if is_overflow || validate_decimal_precision(new_sum, self.sum_precision).is_err() { + // Overflow: set buffer accumulator to null + self.is_not_null = false; + return; + } + + self.sum = Some(new_sum); + + if let Some(new_count) = self.count.checked_add(1) { + self.count = new_count; + } else { + self.is_not_null = false; + return; + } + + self.is_not_null = true; + } +} + +impl Accumulator for AvgDecimalAccumulator { + fn state(&self) -> Result<Vec<ScalarValue>> { + Ok(vec![ + ScalarValue::Decimal128(self.sum, self.sum_precision, self.sum_scale), + ScalarValue::from(self.count), + ]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if !self.is_empty && !self.is_not_null { + // This means there's a overflow in decimal, so we will just skip the rest + // of the computation + return Ok(()); + } + + let values = &values[0]; + let data = values.as_primitive::<Decimal128Type>(); + + self.is_empty = self.is_empty && values.len() == values.null_count(); + + if values.null_count() == 0 { + for i in 0..data.len() { + self.update_single(data, i); + } + } else { + for i in 0..data.len() { + if data.is_null(i) { + continue; + } + self.update_single(data, i); + } + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + // counts are summed + self.count += sum(states[1].as_primitive::<Int64Type>()).unwrap_or_default(); + + // sums are summed + if let Some(x) = sum(states[0].as_primitive::<Decimal128Type>()) { + let v = self.sum.get_or_insert(0); + let (result, overflowed) = v.overflowing_add(x); + if overflowed { + // Set to None if overflow happens + self.sum = None; + } else { + *v = result; + } + } + Ok(()) + } + + fn evaluate(&self) -> Result<ScalarValue> { + fn make_decimal128(value: Option<i128>, precision: u8, scale: i8) -> ScalarValue { + ScalarValue::Decimal128(value, precision, scale) + } + + let scaler = 10_i128.pow(self.target_scale.saturating_sub(self.sum_scale) as u32); + let target_min = MIN_DECIMAL_FOR_EACH_PRECISION[self.target_precision as usize - 1]; + let target_max = MAX_DECIMAL_FOR_EACH_PRECISION[self.target_precision as usize - 1]; + + let result = self + .sum + .map(|v| avg(v, self.count as i128, target_min, target_max, scaler)); + + match result { + Some(value) => Ok(make_decimal128( + value, + self.target_precision, + self.target_scale, + )), + _ => Ok(make_decimal128( + None, + self.target_precision, + self.target_scale, + )), + } + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } +} + +#[derive(Debug)] +struct AvgDecimalGroupsAccumulator { + /// Tracks if the value is null + is_not_null: BooleanBufferBuilder, + + // Tracks if the value is empty + is_empty: BooleanBufferBuilder, + + /// The type of the avg return type + return_data_type: DataType, + target_precision: u8, + target_scale: i8, + + /// Count per group (use i64 to make Int64Array) + counts: Vec<i64>, + + /// Sums per group, stored as i128 + sums: Vec<i128>, + + /// The type of the sum + sum_data_type: DataType, + /// This is input_precision + 10 to be consistent with Spark + sum_precision: u8, + sum_scale: i8, +} + +impl AvgDecimalGroupsAccumulator { + pub fn new( + return_data_type: &DataType, + sum_data_type: &DataType, + target_precision: u8, + target_scale: i8, + sum_precision: u8, + sum_scale: i8, + ) -> Self { + Self { + is_not_null: BooleanBufferBuilder::new(0), + is_empty: BooleanBufferBuilder::new(0), + return_data_type: return_data_type.clone(), + target_precision, + target_scale, + sum_data_type: sum_data_type.clone(), + sum_precision, + sum_scale, + counts: vec![], + sums: vec![], + } + } + + fn is_overflow(&self, index: usize) -> bool { + !self.is_empty.get_bit(index) && !self.is_not_null.get_bit(index) + } + + fn update_single(&mut self, group_index: usize, value: i128) { + if self.is_overflow(group_index) { + // This means there's a overflow in decimal, so we will just skip the rest + // of the computation + return; + } + + self.is_empty.set_bit(group_index, false); + let (new_sum, is_overflow) = self.sums[group_index].overflowing_add(value); + self.counts[group_index] += 1; + + if is_overflow || validate_decimal_precision(new_sum, self.sum_precision).is_err() { + // Overflow: set buffer accumulator to null + self.is_not_null.set_bit(group_index, false); + return; + } + + self.sums[group_index] = new_sum; + self.is_not_null.set_bit(group_index, true) + } +} + +fn ensure_bit_capacity(builder: &mut BooleanBufferBuilder, capacity: usize) { + if builder.len() < capacity { + let additional = capacity - builder.len(); + builder.append_n(additional, true); + } +} + +impl GroupsAccumulator for AvgDecimalGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::<Decimal128Type>(); + let data = values.values(); + + // increment counts, update sums + self.counts.resize(total_num_groups, 0); + self.sums.resize(total_num_groups, 0); + ensure_bit_capacity(&mut self.is_empty, total_num_groups); + ensure_bit_capacity(&mut self.is_not_null, total_num_groups); + + let iter = group_indices.iter().zip(data.iter()); + if values.null_count() == 0 { + for (&group_index, &value) in iter { + self.update_single(group_index, value); + } + } else { + for (idx, (&group_index, &value)) in iter.enumerate() { + if values.is_null(idx) { + continue; + } + self.update_single(group_index, value); + } + } + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&arrow_array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to merge_batch"); + // first batch is partial sums, second is counts + let partial_sums = values[0].as_primitive::<Decimal128Type>(); + let partial_counts = values[1].as_primitive::<Int64Type>(); + // update counts with partial counts + self.counts.resize(total_num_groups, 0); + let iter1 = group_indices.iter().zip(partial_counts.values().iter()); + for (&group_index, &partial_count) in iter1 { + self.counts[group_index] += partial_count; + } + + // update sums + self.sums.resize(total_num_groups, 0); + let iter2 = group_indices.iter().zip(partial_sums.values().iter()); + for (&group_index, &new_value) in iter2 { + let sum = &mut self.sums[group_index]; + *sum = sum.add_wrapping(new_value); + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result<ArrayRef> { + let counts = emit_to.take_needed(&mut self.counts); + let sums = emit_to.take_needed(&mut self.sums); + + let mut builder = PrimitiveBuilder::<Decimal128Type>::with_capacity(sums.len()) + .with_data_type(self.return_data_type.clone()); + let iter = sums.into_iter().zip(counts); + + let scaler = 10_i128.pow(self.target_scale.saturating_sub(self.sum_scale) as u32); + let target_min = MIN_DECIMAL_FOR_EACH_PRECISION[self.target_precision as usize - 1]; + let target_max = MAX_DECIMAL_FOR_EACH_PRECISION[self.target_precision as usize - 1]; + + for (sum, count) in iter { + if count != 0 { + match avg(sum, count as i128, target_min, target_max, scaler) { + Some(value) => { + builder.append_value(value); + } + _ => { + builder.append_null(); + } + } + } else { + builder.append_null(); + } + } + let array: PrimitiveArray<Decimal128Type> = builder.finish(); + + Ok(Arc::new(array)) + } + + // return arrays for sums and counts + fn state(&mut self, emit_to: EmitTo) -> Result<Vec<ArrayRef>> { + assert!( + matches!(emit_to, EmitTo::All), + "EmitTo::First is not supported" + ); + + let nulls = self.is_not_null.finish(); + let nulls = Some(NullBuffer::new(nulls)); + + let counts = emit_to.take_needed(&mut self.counts); + let counts = Int64Array::new(counts.into(), nulls.clone()); + + let sums = emit_to.take_needed(&mut self.sums); + let sums = + Decimal128Array::new(sums.into(), nulls).with_data_type(self.sum_data_type.clone()); + + Ok(vec![ + Arc::new(sums) as ArrayRef, + Arc::new(counts) as ArrayRef, + ]) + } + + fn size(&self) -> usize { + self.counts.capacity() * std::mem::size_of::<i64>() + + self.sums.capacity() * std::mem::size_of::<i128>() + } +} + +/// Returns the `sum`/`count` as a i128 Decimal128 with +/// target_scale and target_precision and return None if overflows. +/// +/// * sum: The total sum value stored as Decimal128 with sum_scale +/// * count: total count, stored as a i128 (*NOT* a Decimal128 value) +/// * target_min: The minimum output value possible to represent with the target precision +/// * target_max: The maximum output value possible to represent with the target precision +/// * scaler: scale factor for avg +#[inline(always)] +fn avg(sum: i128, count: i128, target_min: i128, target_max: i128, scaler: i128) -> Option<i128> { + if let Some(value) = sum.checked_mul(scaler) { + // `sum / count` with ROUND_HALF_UP + let (div, rem) = value.div_rem(&count); + let half = count.div_ceil(2); + let half_neg = half.neg_wrapping(); + let new_value = match value >= 0 { + true if rem >= half => div.add_wrapping(1), + false if rem <= half_neg => div.sub_wrapping(1), + _ => div, + }; + if new_value >= target_min && new_value <= target_max { + Some(new_value) + } else { + None + } + } else { + None + } +} diff --git a/core/src/execution/datafusion/expressions/bitwise_not.rs b/core/src/execution/datafusion/expressions/bitwise_not.rs new file mode 100644 index 0000000000..f9f8ee392c --- /dev/null +++ b/core/src/execution/datafusion/expressions/bitwise_not.rs @@ -0,0 +1,194 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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, + hash::{Hash, Hasher}, + sync::Arc, +}; + +use arrow::{ + array::*, + datatypes::{DataType, Schema}, + record_batch::RecordBatch, +}; +use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_physical_expr::PhysicalExpr; + +use crate::execution::datafusion::expressions::utils::down_cast_any_ref; + +macro_rules! compute_op { + ($OPERAND:expr, $DT:ident) => {{ + let operand = $OPERAND + .as_any() + .downcast_ref::<$DT>() + .expect("compute_op failed to downcast array"); + let result: $DT = operand.iter().map(|x| x.map(|y| !y)).collect(); + Ok(Arc::new(result)) + }}; +} + +/// BitwiseNot expression +#[derive(Debug, Hash)] +pub struct BitwiseNotExpr { + /// Input expression + arg: Arc<dyn PhysicalExpr>, +} + +impl BitwiseNotExpr { + /// Create new bitwise not expression + pub fn new(arg: Arc<dyn PhysicalExpr>) -> Self { + Self { arg } + } + + /// Get the input expression + pub fn arg(&self) -> &Arc<dyn PhysicalExpr> { + &self.arg + } +} + +impl std::fmt::Display for BitwiseNotExpr { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "(~ {})", self.arg) + } +} + +impl PhysicalExpr for BitwiseNotExpr { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> Result<DataType> { + self.arg.data_type(input_schema) + } + + fn nullable(&self, input_schema: &Schema) -> Result<bool> { + self.arg.nullable(input_schema) + } + + fn evaluate(&self, batch: &RecordBatch) -> Result<ColumnarValue> { + let arg = self.arg.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let result: Result<ArrayRef> = match array.data_type() { + DataType::Int8 => compute_op!(array, Int8Array), + DataType::Int16 => compute_op!(array, Int16Array), + DataType::Int32 => compute_op!(array, Int32Array), + DataType::Int64 => compute_op!(array, Int64Array), + _ => Err(DataFusionError::Execution(format!( + "(- '{:?}') can't be evaluated because the expression's type is {:?}, not signed int", + self, + array.data_type(), + ))), + }; + result.map(ColumnarValue::Array) + } + ColumnarValue::Scalar(_) => Err(DataFusionError::Internal( + "shouldn't go to bitwise not scalar path".to_string(), + )), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.arg.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(BitwiseNotExpr::new(children[0].clone()))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.arg.hash(&mut s); + self.hash(&mut s); + } +} + +impl PartialEq<dyn Any> for BitwiseNotExpr { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.arg.eq(&x.arg)) + .unwrap_or(false) + } +} + +pub fn bitwise_not(arg: Arc<dyn PhysicalExpr>) -> Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(BitwiseNotExpr::new(arg))) +} + +fn scalar_bitwise_not(scalar: ScalarValue) -> Result<ScalarValue> { + match scalar { + ScalarValue::Int8(None) + | ScalarValue::Int16(None) + | ScalarValue::Int32(None) + | ScalarValue::Int64(None) => Ok(scalar), + ScalarValue::Int8(Some(v)) => Ok(ScalarValue::Int8(Some(!v))), + ScalarValue::Int16(Some(v)) => Ok(ScalarValue::Int16(Some(!v))), + ScalarValue::Int32(Some(v)) => Ok(ScalarValue::Int32(Some(!v))), + ScalarValue::Int64(Some(v)) => Ok(ScalarValue::Int64(Some(!v))), + value => Err(DataFusionError::Internal(format!( + "Can not run ! on scalar value {value:?}" + ))), + } +} + +#[cfg(test)] +mod tests { + use arrow::datatypes::*; + use datafusion_common::{cast::as_int32_array, Result}; + use datafusion_physical_expr::expressions::col; + + use super::*; + + #[test] + fn bitwise_not_op() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, true)]); + + let expr = bitwise_not(col("a", &schema)?)?; + + let input = Int32Array::from(vec![ + Some(1), + Some(2), + None, + Some(12345), + Some(89), + Some(-3456), + ]); + let expected = &Int32Array::from(vec![ + Some(-2), + Some(-3), + None, + Some(-12346), + Some(-90), + Some(3455), + ]); + + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(input)])?; + + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_int32_array(&result).expect("failed to downcast to In32Array"); + assert_eq!(result, expected); + + Ok(()) + } +} diff --git a/core/src/execution/datafusion/expressions/cast.rs b/core/src/execution/datafusion/expressions/cast.rs new file mode 100644 index 0000000000..d8450686df --- /dev/null +++ b/core/src/execution/datafusion/expressions/cast.rs @@ -0,0 +1,156 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{Display, Formatter}, + hash::{Hash, Hasher}, + sync::Arc, +}; + +use arrow::{ + compute::{cast_with_options, CastOptions}, + record_batch::RecordBatch, + util::display::FormatOptions, +}; +use arrow_array::ArrayRef; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{Result as DataFusionResult, ScalarValue}; +use datafusion_physical_expr::PhysicalExpr; + +use crate::execution::datafusion::expressions::utils::{ + array_with_timezone, down_cast_any_ref, spark_cast, +}; + +static TIMESTAMP_FORMAT: Option<&str> = Some("%Y-%m-%d %H:%M:%S%.f"); +static CAST_OPTIONS: CastOptions = CastOptions { + safe: true, + format_options: FormatOptions::new() + .with_timestamp_tz_format(TIMESTAMP_FORMAT) + .with_timestamp_format(TIMESTAMP_FORMAT), +}; + +#[derive(Debug, Hash)] +pub struct Cast { + pub child: Arc<dyn PhysicalExpr>, + pub data_type: DataType, + + /// When cast from/to timezone related types, we need timezone, which will be resolved with + /// session local timezone by an analyzer in Spark. + pub timezone: String, +} + +impl Cast { + pub fn new(child: Arc<dyn PhysicalExpr>, data_type: DataType, timezone: String) -> Self { + Self { + child, + data_type, + timezone, + } + } + + pub fn new_without_timezone(child: Arc<dyn PhysicalExpr>, data_type: DataType) -> Self { + Self { + child, + data_type, + timezone: "".to_string(), + } + } + + fn cast_array(&self, array: ArrayRef) -> DataFusionResult<ArrayRef> { + let array = array_with_timezone(array, self.timezone.clone(), Some(&self.data_type)); + let from_type = array.data_type(); + let cast_result = cast_with_options(&array, &self.data_type, &CAST_OPTIONS)?; + Ok(spark_cast(cast_result, from_type, &self.data_type)) + } +} + +impl Display for Cast { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Cast [data_type: {}, timezone: {}, child: {}]", + self.data_type, self.timezone, self.child + ) + } +} + +impl PartialEq<dyn Any> for Cast { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| { + self.child.eq(&x.child) + && self.timezone.eq(&x.timezone) + && self.data_type.eq(&x.data_type) + }) + .unwrap_or(false) + } +} + +impl PhysicalExpr for Cast { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _: &Schema) -> DataFusionResult<DataType> { + Ok(self.data_type.clone()) + } + + fn nullable(&self, _: &Schema) -> DataFusionResult<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> DataFusionResult<ColumnarValue> { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => Ok(ColumnarValue::Array(self.cast_array(array)?)), + ColumnarValue::Scalar(scalar) => { + // Note that normally CAST(scalar) should be fold in Spark JVM side. However, for + // some cases e.g., scalar subquery, Spark will not fold it, so we need to handle it + // here. + let array = scalar.to_array()?; + let scalar = ScalarValue::try_from_array(&self.cast_array(array)?, 0)?; + Ok(ColumnarValue::Scalar(scalar)) + } + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> datafusion_common::Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(Cast::new( + children[0].clone(), + self.data_type.clone(), + self.timezone.clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.data_type.hash(&mut s); + self.timezone.hash(&mut s); + self.hash(&mut s); + } +} diff --git a/core/src/execution/datafusion/expressions/checkoverflow.rs b/core/src/execution/datafusion/expressions/checkoverflow.rs new file mode 100644 index 0000000000..1e4b5f3339 --- /dev/null +++ b/core/src/execution/datafusion/expressions/checkoverflow.rs @@ -0,0 +1,190 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{Display, Formatter}, + hash::{Hash, Hasher}, + sync::Arc, +}; + +use arrow::{ + array::{as_primitive_array, Array, ArrayRef, Decimal128Array, PrimitiveArray}, + datatypes::{Decimal128Type, DecimalType}, + record_batch::RecordBatch, +}; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{DataFusionError, ScalarValue}; +use datafusion_physical_expr::PhysicalExpr; + +use crate::execution::datafusion::expressions::utils::down_cast_any_ref; + +/// This is from Spark `CheckOverflow` expression. Spark `CheckOverflow` expression rounds decimals +/// to given scale and check if the decimals can fit in given precision. As `cast` kernel rounds +/// decimals already, Comet `CheckOverflow` expression only checks if the decimals can fit in the +/// precision. +#[derive(Debug, Hash)] +pub struct CheckOverflow { + pub child: Arc<dyn PhysicalExpr>, + pub data_type: DataType, + pub fail_on_error: bool, +} + +impl CheckOverflow { + pub fn new(child: Arc<dyn PhysicalExpr>, data_type: DataType, fail_on_error: bool) -> Self { + Self { + child, + data_type, + fail_on_error, + } + } +} + +impl Display for CheckOverflow { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "CheckOverflow [datatype: {}, fail_on_error: {}, child: {}]", + self.data_type, self.fail_on_error, self.child + ) + } +} + +impl PartialEq<dyn Any> for CheckOverflow { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| { + self.child.eq(&x.child) + && self.data_type.eq(&x.data_type) + && self.fail_on_error.eq(&x.fail_on_error) + }) + .unwrap_or(false) + } +} + +impl PhysicalExpr for CheckOverflow { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _: &Schema) -> datafusion_common::Result<DataType> { + Ok(self.data_type.clone()) + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) + if matches!(array.data_type(), DataType::Decimal128(_, _)) => + { + let (precision, scale) = match &self.data_type { + DataType::Decimal128(p, s) => (p, s), + dt => { + return Err(DataFusionError::Execution(format!( + "CheckOverflow expects only Decimal128, but got {:?}", + dt + ))) + } + }; + + let decimal_array = as_primitive_array::<Decimal128Type>(&array); + + let casted_array = if self.fail_on_error { + // Returning error if overflow + let iter = decimal_array + .iter() + .map(|v| { + v.map(|v| { + Decimal128Type::validate_decimal_precision(v, *precision).map(|_| v) + }) + .map_or(Ok(None), |r| r.map(Some)) + }) + .collect::<Result<Vec<_>, _>>()? + .into_iter(); + unsafe { PrimitiveArray::<Decimal128Type>::from_trusted_len_iter(iter) } + } else { + // Overflowing gets null value + let iter = decimal_array.iter().map(|v| { + v.and_then(|v| { + Decimal128Type::validate_decimal_precision(v, *precision) + .map(|_| v) + .ok() + }) + }); + unsafe { PrimitiveArray::<Decimal128Type>::from_trusted_len_iter(iter) } + }; + + let new_array = Decimal128Array::from(casted_array.to_data()) + .with_precision_and_scale(*precision, *scale) + .map(|a| Arc::new(a) as ArrayRef)?; + + Ok(ColumnarValue::Array(new_array)) + } + ColumnarValue::Scalar(ScalarValue::Decimal128(v, precision, scale)) => { + // `fail_on_error` is only true when ANSI is enabled, which we don't support yet + // (Java side will simply fallback to Spark when it is enabled) + assert!( + !self.fail_on_error, + "fail_on_error (ANSI mode) is not supported yet" + ); + + let new_v: Option<i128> = v.and_then(|v| { + Decimal128Type::validate_decimal_precision(v, precision) + .map(|_| v) + .ok() + }); + + Ok(ColumnarValue::Scalar(ScalarValue::Decimal128( + new_v, precision, scale, + ))) + } + v => Err(DataFusionError::Execution(format!( + "CheckOverflow's child expression should be decimal array, but found {:?}", + v + ))), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> datafusion_common::Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(CheckOverflow::new( + children[0].clone(), + self.data_type.clone(), + self.fail_on_error, + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.data_type.hash(&mut s); + self.fail_on_error.hash(&mut s); + self.hash(&mut s); + } +} diff --git a/core/src/execution/datafusion/expressions/if_expr.rs b/core/src/execution/datafusion/expressions/if_expr.rs new file mode 100644 index 0000000000..826f017c12 --- /dev/null +++ b/core/src/execution/datafusion/expressions/if_expr.rs @@ -0,0 +1,221 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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, + hash::{Hash, Hasher}, + sync::Arc, +}; + +use arrow::{ + array::*, + compute::{and, is_null, kernels::zip::zip, not, or_kleene}, + datatypes::{DataType, Schema}, + record_batch::RecordBatch, +}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{cast::as_boolean_array, Result}; +use datafusion_physical_expr::PhysicalExpr; + +use crate::execution::datafusion::expressions::utils::down_cast_any_ref; + +#[derive(Debug, Hash)] +pub struct IfExpr { + if_expr: Arc<dyn PhysicalExpr>, + true_expr: Arc<dyn PhysicalExpr>, + false_expr: Arc<dyn PhysicalExpr>, +} + +impl std::fmt::Display for IfExpr { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!( + f, + "If [if: {}, true_expr: {}, false_expr: {}]", + self.if_expr, self.true_expr, self.false_expr + ) + } +} + +impl IfExpr { + /// Create a new IF expression + pub fn new( + if_expr: Arc<dyn PhysicalExpr>, + true_expr: Arc<dyn PhysicalExpr>, + false_expr: Arc<dyn PhysicalExpr>, + ) -> Self { + Self { + if_expr, + true_expr, + false_expr, + } + } +} + +impl PhysicalExpr for IfExpr { + /// Return a reference to Any that can be used for down-casting + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> Result<DataType> { + let data_type = self.true_expr.data_type(input_schema)?; + Ok(data_type) + } + + fn nullable(&self, _input_schema: &Schema) -> Result<bool> { + if self.true_expr.nullable(_input_schema)? || self.true_expr.nullable(_input_schema)? { + Ok(true) + } else { + Ok(false) + } + } + + fn evaluate(&self, batch: &RecordBatch) -> Result<ColumnarValue> { + let mut remainder = BooleanArray::from(vec![true; batch.num_rows()]); + + // evaluate if condition on batch + let if_value = self.if_expr.evaluate_selection(batch, &remainder)?; + let if_value = if_value.into_array(batch.num_rows())?; + let if_value = + as_boolean_array(&if_value).expect("if expression did not return a BooleanArray"); + + let true_value = self.true_expr.evaluate_selection(batch, if_value)?; + let true_value = true_value.into_array(batch.num_rows())?; + + remainder = and( + &remainder, + &or_kleene(¬(if_value)?, &is_null(if_value)?)?, + )?; + + let false_value = self + .false_expr + .evaluate_selection(batch, &remainder)? + .into_array(batch.num_rows())?; + let current_value = zip(&remainder, &false_value, &true_value)?; + + Ok(ColumnarValue::Array(current_value)) + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![ + self.true_expr.clone(), + self.if_expr.clone(), + self.false_expr.clone(), + ] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(IfExpr::new( + children[0].clone(), + children[1].clone(), + children[2].clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.if_expr.hash(&mut s); + self.true_expr.hash(&mut s); + self.false_expr.hash(&mut s); + self.hash(&mut s); + } +} + +impl PartialEq<dyn Any> for IfExpr { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| { + self.if_expr.eq(&x.if_expr) + && self.true_expr.eq(&x.true_expr) + && self.false_expr.eq(&x.false_expr) + }) + .unwrap_or(false) + } +} + +/// Create an If expression +pub fn if_fn( + if_expr: Arc<dyn PhysicalExpr>, + true_expr: Arc<dyn PhysicalExpr>, + false_expr: Arc<dyn PhysicalExpr>, +) -> Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(IfExpr::new(if_expr, true_expr, false_expr))) +} + +#[cfg(test)] +mod tests { + use arrow::{array::StringArray, datatypes::*}; + use datafusion::logical_expr::Operator; + use datafusion_common::cast::as_int32_array; + use datafusion_physical_expr::expressions::{binary, col, lit}; + + use super::*; + + #[test] + fn test_if_1() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]); + let a = StringArray::from(vec![Some("foo"), Some("baz"), None, Some("bar")]); + let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)])?; + let schema_ref = batch.schema(); + + // if a = 'foo' 123 else 999 + let if_expr = binary( + col("a", &schema_ref)?, + Operator::Eq, + lit("foo"), + &schema_ref, + )?; + let true_expr = lit(123i32); + let false_expr = lit(999i32); + + let expr = if_fn(if_expr, true_expr, false_expr); + let result = expr?.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_int32_array(&result)?; + + let expected = &Int32Array::from(vec![Some(123), Some(999), Some(999), Some(999)]); + + assert_eq!(expected, result); + + Ok(()) + } + + #[test] + fn test_if_2() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, true)]); + let a = Int32Array::from(vec![Some(1), Some(0), None, Some(5)]); + let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)])?; + let schema_ref = batch.schema(); + + // if a >=1 123 else 999 + let if_expr = binary(col("a", &schema_ref)?, Operator::GtEq, lit(1), &schema_ref)?; + let true_expr = lit(123i32); + let false_expr = lit(999i32); + + let expr = if_fn(if_expr, true_expr, false_expr); + let result = expr?.evaluate(&batch)?.into_array(batch.num_rows())?; + let result = as_int32_array(&result)?; + + let expected = &Int32Array::from(vec![Some(123), Some(999), Some(999), Some(123)]); + assert_eq!(expected, result); + + Ok(()) + } +} diff --git a/core/src/execution/datafusion/expressions/mod.rs b/core/src/execution/datafusion/expressions/mod.rs new file mode 100644 index 0000000000..cfc312510b --- /dev/null +++ b/core/src/execution/datafusion/expressions/mod.rs @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Native DataFusion expressions + +pub mod bitwise_not; +pub mod cast; +pub mod checkoverflow; +pub mod if_expr; +mod normalize_nan; +pub mod scalar_funcs; +pub use normalize_nan::NormalizeNaNAndZero; +pub mod avg; +pub mod avg_decimal; +pub mod strings; +pub mod subquery; +pub mod sum_decimal; +pub mod temporal; +mod utils; diff --git a/core/src/execution/datafusion/expressions/normalize_nan.rs b/core/src/execution/datafusion/expressions/normalize_nan.rs new file mode 100644 index 0000000000..111a34d5d6 --- /dev/null +++ b/core/src/execution/datafusion/expressions/normalize_nan.rs @@ -0,0 +1,167 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{Display, Formatter}, + hash::{Hash, Hasher}, + sync::Arc, +}; + +use arrow::{ + array::{as_primitive_array, ArrayAccessor, ArrayIter, Float32Array, Float64Array}, + datatypes::{ArrowNativeType, Float32Type, Float64Type}, + record_batch::RecordBatch, +}; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_physical_expr::PhysicalExpr; + +use crate::execution::datafusion::expressions::utils::down_cast_any_ref; + +#[derive(Debug, Hash)] +pub struct NormalizeNaNAndZero { + pub data_type: DataType, + pub child: Arc<dyn PhysicalExpr>, +} + +impl NormalizeNaNAndZero { + pub fn new(data_type: DataType, child: Arc<dyn PhysicalExpr>) -> Self { + Self { data_type, child } + } +} + +impl PhysicalExpr for NormalizeNaNAndZero { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result<DataType> { + self.child.data_type(input_schema) + } + + fn nullable(&self, input_schema: &Schema) -> datafusion_common::Result<bool> { + self.child.nullable(input_schema) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let cv = self.child.evaluate(batch)?; + let array = cv.into_array(batch.num_rows())?; + + match &self.data_type { + DataType::Float32 => { + let v = eval_typed(as_primitive_array::<Float32Type>(&array)); + let new_array = Float32Array::from(v); + Ok(ColumnarValue::Array(Arc::new(new_array))) + } + DataType::Float64 => { + let v = eval_typed(as_primitive_array::<Float64Type>(&array)); + let new_array = Float64Array::from(v); + Ok(ColumnarValue::Array(Arc::new(new_array))) + } + dt => panic!("Unexpected data type {:?}", dt), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + self.child.children() + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> datafusion_common::Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(NormalizeNaNAndZero::new( + self.data_type.clone(), + children[0].clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.data_type.hash(&mut s); + self.hash(&mut s); + } +} + +fn eval_typed<V: FloatDouble, T: ArrayAccessor<Item = V>>(input: T) -> Vec<Option<V>> { + let iter = ArrayIter::new(input); + iter.map(|o| { + o.map(|v| { + if v.is_nan() { + v.nan() + } else if v.is_neg_zero() { + v.zero() + } else { + v + } + }) + }) + .collect() +} + +impl Display for NormalizeNaNAndZero { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "FloatNormalize [child: {}]", self.child) + } +} + +impl PartialEq<dyn Any> for NormalizeNaNAndZero { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.child.eq(&x.child) && self.data_type.eq(&x.data_type)) + .unwrap_or(false) + } +} + +trait FloatDouble: ArrowNativeType { + fn is_nan(&self) -> bool; + fn nan(&self) -> Self; + fn is_neg_zero(&self) -> bool; + fn zero(&self) -> Self; +} + +impl FloatDouble for f32 { + fn is_nan(&self) -> bool { + f32::is_nan(*self) + } + fn nan(&self) -> Self { + f32::NAN + } + fn is_neg_zero(&self) -> bool { + *self == -0.0 + } + fn zero(&self) -> Self { + 0.0 + } +} +impl FloatDouble for f64 { + fn is_nan(&self) -> bool { + f64::is_nan(*self) + } + fn nan(&self) -> Self { + f64::NAN + } + fn is_neg_zero(&self) -> bool { + *self == -0.0 + } + fn zero(&self) -> Self { + 0.0 + } +} diff --git a/core/src/execution/datafusion/expressions/scalar_funcs.rs b/core/src/execution/datafusion/expressions/scalar_funcs.rs new file mode 100644 index 0000000000..875956621f --- /dev/null +++ b/core/src/execution/datafusion/expressions/scalar_funcs.rs @@ -0,0 +1,490 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::min, str::FromStr, sync::Arc}; + +use arrow::{ + array::{ + ArrayRef, AsArray, Decimal128Builder, Float32Array, Float64Array, GenericStringArray, + Int16Array, Int32Array, Int64Array, Int64Builder, Int8Array, OffsetSizeTrait, + }, + datatypes::{validate_decimal_precision, Decimal128Type, Int64Type}, +}; +use arrow_array::{Array, ArrowNativeTypeOp, Decimal128Array}; +use arrow_schema::DataType; +use datafusion::{ + logical_expr::{BuiltinScalarFunction, ScalarFunctionImplementation}, + physical_plan::ColumnarValue, +}; +use datafusion_common::{ + cast::as_generic_string_array, internal_err, DataFusionError, Result as DataFusionResult, + ScalarValue, +}; +use datafusion_physical_expr::{ + execution_props::ExecutionProps, + functions::{create_physical_fun, make_scalar_function}, + math_expressions, +}; +use num::{BigInt, Signed, ToPrimitive}; +use unicode_segmentation::UnicodeSegmentation; + +/// Create a physical scalar function. +pub fn create_comet_physical_fun( + fun_name: &str, + execution_props: &ExecutionProps, + data_type: DataType, +) -> Result<ScalarFunctionImplementation, DataFusionError> { + match fun_name { + "ceil" => Ok(Arc::new(move |x| spark_ceil(x, &data_type))), + "floor" => Ok(Arc::new(move |x| spark_floor(x, &data_type))), + "rpad" => Ok(Arc::new(spark_rpad)), + "round" => Ok(Arc::new(move |x| spark_round(x, &data_type))), + "unscaled_value" => Ok(Arc::new(spark_unscaled_value)), + "make_decimal" => Ok(Arc::new(move |x| spark_make_decimal(x, &data_type))), + "decimal_div" => Ok(Arc::new(move |x| spark_decimal_div(x, &data_type))), + _ => { + let fun = &BuiltinScalarFunction::from_str(fun_name)?; + create_physical_fun(fun, execution_props) + } + } +} + +#[inline] +fn get_precision_scale(data_type: &DataType) -> (u8, i8) { + let DataType::Decimal128(precision, scale) = data_type else { + unreachable!() + }; + (*precision, *scale) +} + +macro_rules! downcast_compute_op { + ($ARRAY:expr, $NAME:expr, $FUNC:ident, $TYPE:ident, $RESULT:ident) => {{ + let n = $ARRAY.as_any().downcast_ref::<$TYPE>(); + match n { + Some(array) => { + let res: $RESULT = + arrow::compute::kernels::arity::unary(array, |x| x.$FUNC() as i64); + Ok(Arc::new(res)) + } + _ => Err(DataFusionError::Internal(format!( + "Invalid data type for {}", + $NAME + ))), + } + }}; +} + +/// `ceil` function that simulates Spark `ceil` expression +pub fn spark_ceil( + args: &[ColumnarValue], + data_type: &DataType, +) -> Result<ColumnarValue, DataFusionError> { + let value = &args[0]; + match value { + ColumnarValue::Array(array) => match array.data_type() { + DataType::Float32 => { + let result = downcast_compute_op!(array, "ceil", ceil, Float32Array, Int64Array); + Ok(ColumnarValue::Array(result?)) + } + DataType::Float64 => { + let result = downcast_compute_op!(array, "ceil", ceil, Float64Array, Int64Array); + Ok(ColumnarValue::Array(result?)) + } + DataType::Int64 => { + let result = array.as_any().downcast_ref::<Int64Array>().unwrap(); + Ok(ColumnarValue::Array(Arc::new(result.clone()))) + } + DataType::Decimal128(_, scale) if *scale > 0 => { + let f = decimal_ceil_f(scale); + let (precision, scale) = get_precision_scale(data_type); + make_decimal_array(array, precision, scale, &f) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function ceil", + other, + ))), + }, + ColumnarValue::Scalar(a) => match a { + ScalarValue::Float32(a) => Ok(ColumnarValue::Scalar(ScalarValue::Int64( + a.map(|x| x.ceil() as i64), + ))), + ScalarValue::Float64(a) => Ok(ColumnarValue::Scalar(ScalarValue::Int64( + a.map(|x| x.ceil() as i64), + ))), + ScalarValue::Int64(a) => Ok(ColumnarValue::Scalar(ScalarValue::Int64(a.map(|x| x)))), + ScalarValue::Decimal128(a, _, scale) if *scale > 0 => { + let f = decimal_ceil_f(scale); + let (precision, scale) = get_precision_scale(data_type); + make_decimal_scalar(a, precision, scale, &f) + } + _ => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function ceil", + value.data_type(), + ))), + }, + } +} + +/// `floor` function that simulates Spark `floor` expression +pub fn spark_floor( + args: &[ColumnarValue], + data_type: &DataType, +) -> Result<ColumnarValue, DataFusionError> { + let value = &args[0]; + match value { + ColumnarValue::Array(array) => match array.data_type() { + DataType::Float32 => { + let result = downcast_compute_op!(array, "floor", floor, Float32Array, Int64Array); + Ok(ColumnarValue::Array(result?)) + } + DataType::Float64 => { + let result = downcast_compute_op!(array, "floor", floor, Float64Array, Int64Array); + Ok(ColumnarValue::Array(result?)) + } + DataType::Int64 => { + let result = array.as_any().downcast_ref::<Int64Array>().unwrap(); + Ok(ColumnarValue::Array(Arc::new(result.clone()))) + } + DataType::Decimal128(_, scale) if *scale > 0 => { + let f = decimal_floor_f(scale); + let (precision, scale) = get_precision_scale(data_type); + make_decimal_array(array, precision, scale, &f) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function floor", + other, + ))), + }, + ColumnarValue::Scalar(a) => match a { + ScalarValue::Float32(a) => Ok(ColumnarValue::Scalar(ScalarValue::Int64( + a.map(|x| x.floor() as i64), + ))), + ScalarValue::Float64(a) => Ok(ColumnarValue::Scalar(ScalarValue::Int64( + a.map(|x| x.floor() as i64), + ))), + ScalarValue::Int64(a) => Ok(ColumnarValue::Scalar(ScalarValue::Int64(a.map(|x| x)))), + ScalarValue::Decimal128(a, _, scale) if *scale > 0 => { + let f = decimal_floor_f(scale); + let (precision, scale) = get_precision_scale(data_type); + make_decimal_scalar(a, precision, scale, &f) + } + _ => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function floor", + value.data_type(), + ))), + }, + } +} + +pub fn spark_unscaled_value(args: &[ColumnarValue]) -> DataFusionResult<ColumnarValue> { + match &args[0] { + ColumnarValue::Scalar(v) => match v { + ScalarValue::Decimal128(d, _, _) => Ok(ColumnarValue::Scalar(ScalarValue::Int64( + d.map(|n| n as i64), + ))), + dt => internal_err!("Expected Decimal128 but found {dt:}"), + }, + ColumnarValue::Array(a) => { + let arr = a.as_primitive::<Decimal128Type>(); + let mut result = Int64Builder::new(); + for v in arr.into_iter() { + result.append_option(v.map(|v| v as i64)); + } + Ok(ColumnarValue::Array(Arc::new(result.finish()))) + } + } +} + +pub fn spark_make_decimal( + args: &[ColumnarValue], + data_type: &DataType, +) -> DataFusionResult<ColumnarValue> { + let (precision, scale) = get_precision_scale(data_type); + match &args[0] { + ColumnarValue::Scalar(v) => match v { + ScalarValue::Int64(n) => Ok(ColumnarValue::Scalar(ScalarValue::Decimal128( + long_to_decimal(n, precision), + precision, + scale, + ))), + sv => internal_err!("Expected Int64 but found {sv:?}"), + }, + ColumnarValue::Array(a) => { + let arr = a.as_primitive::<Int64Type>(); + let mut result = Decimal128Builder::new(); + for v in arr.into_iter() { + result.append_option(long_to_decimal(&v, precision)) + } + let result_type = DataType::Decimal128(precision, scale); + + Ok(ColumnarValue::Array(Arc::new( + result.finish().with_data_type(result_type), + ))) + } + } +} + +/// Convert the input long to decimal with the given maximum precision. If overflows, returns null +/// instead. +#[inline] +fn long_to_decimal(v: &Option<i64>, precision: u8) -> Option<i128> { + match v { + Some(v) if validate_decimal_precision(*v as i128, precision).is_ok() => Some(*v as i128), + _ => None, + } +} + +#[inline] +fn decimal_ceil_f(scale: &i8) -> impl Fn(i128) -> i128 { + let div = 10_i128.pow_wrapping(*scale as u32); + move |x: i128| x.div_ceil(div) +} + +#[inline] +fn decimal_floor_f(scale: &i8) -> impl Fn(i128) -> i128 { + let div = 10_i128.pow_wrapping(*scale as u32); + move |x: i128| x.div_floor(div) +} + +// Spark uses BigDecimal. See RoundBase implementation in Spark. Instead, we do the same by +// 1) add the half of divisor, 2) round down by division, 3) adjust precision by multiplication +#[inline] +fn decimal_round_f(scale: &i8, point: &i64) -> Box<dyn Fn(i128) -> i128> { + if *point < 0 { + if let Some(div) = 10_i128.checked_pow((-(*point) as u32) + (*scale as u32)) { + let half = div / 2; + let mul = 10_i128.pow_wrapping((-(*point)) as u32); + // i128 can hold 39 digits of a base 10 number, adding half will not cause overflow + Box::new(move |x: i128| (x + x.signum() * half) / div * mul) + } else { + Box::new(move |_: i128| 0) + } + } else { + let div = 10_i128.pow_wrapping((*scale as u32) - min(*scale as u32, *point as u32)); + let half = div / 2; + Box::new(move |x: i128| (x + x.signum() * half) / div) + } +} + +#[inline] +fn make_decimal_array( + array: &ArrayRef, + precision: u8, + scale: i8, + f: &dyn Fn(i128) -> i128, +) -> Result<ColumnarValue, DataFusionError> { + let array = array.as_primitive::<Decimal128Type>(); + let result: Decimal128Array = arrow::compute::kernels::arity::unary(array, f); + let result = result.with_data_type(DataType::Decimal128(precision, scale)); + Ok(ColumnarValue::Array(Arc::new(result))) +} + +#[inline] +fn make_decimal_scalar( + a: &Option<i128>, + precision: u8, + scale: i8, + f: &dyn Fn(i128) -> i128, +) -> Result<ColumnarValue, DataFusionError> { + let result = ScalarValue::Decimal128(a.map(f), precision, scale); + Ok(ColumnarValue::Scalar(result)) +} + +macro_rules! integer_round { + ($X:expr, $DIV:expr, $HALF:expr) => {{ + let rem = $X % $DIV; + if rem <= -$HALF { + ($X - rem).sub_wrapping($DIV) + } else if rem >= $HALF { + ($X - rem).add_wrapping($DIV) + } else { + $X - rem + } + }}; +} + +macro_rules! round_integer_array { + ($ARRAY:expr, $POINT:expr, $TYPE:ty, $NATIVE:ty) => {{ + let array = $ARRAY.as_any().downcast_ref::<$TYPE>().unwrap(); + let ten: $NATIVE = 10; + let result: $TYPE = if let Some(div) = ten.checked_pow((-(*$POINT)) as u32) { + let half = div / 2; + arrow::compute::kernels::arity::unary(array, |x| integer_round!(x, div, half)) + } else { + arrow::compute::kernels::arity::unary(array, |_| 0) + }; + Ok(ColumnarValue::Array(Arc::new(result))) + }}; +} + +macro_rules! round_integer_scalar { + ($SCALAR:expr, $POINT:expr, $TYPE:expr, $NATIVE:ty) => {{ + let ten: $NATIVE = 10; + if let Some(div) = ten.checked_pow((-(*$POINT)) as u32) { + let half = div / 2; + Ok(ColumnarValue::Scalar($TYPE( + $SCALAR.map(|x| integer_round!(x, div, half)), + ))) + } else { + Ok(ColumnarValue::Scalar($TYPE(Some(0)))) + } + }}; +} + +/// `round` function that simulates Spark `round` expression +fn spark_round( + args: &[ColumnarValue], + data_type: &DataType, +) -> Result<ColumnarValue, DataFusionError> { + let value = &args[0]; + let point = &args[1]; + let ColumnarValue::Scalar(ScalarValue::Int64(Some(point))) = point else { + return internal_err!("Invalid point argument for Round(): {:#?}", point); + }; + match value { + ColumnarValue::Array(array) => match array.data_type() { + DataType::Int64 if *point < 0 => round_integer_array!(array, point, Int64Array, i64), + DataType::Int32 if *point < 0 => round_integer_array!(array, point, Int32Array, i32), + DataType::Int16 if *point < 0 => round_integer_array!(array, point, Int16Array, i16), + DataType::Int8 if *point < 0 => round_integer_array!(array, point, Int8Array, i8), + DataType::Decimal128(_, scale) if *scale > 0 => { + let f = decimal_round_f(scale, point); + let (precision, scale) = get_precision_scale(data_type); + make_decimal_array(array, precision, scale, &f) + } + _ => make_scalar_function(math_expressions::round)(args), + }, + ColumnarValue::Scalar(a) => match a { + ScalarValue::Int64(a) if *point < 0 => { + round_integer_scalar!(a, point, ScalarValue::Int64, i64) + } + ScalarValue::Int32(a) if *point < 0 => { + round_integer_scalar!(a, point, ScalarValue::Int32, i32) + } + ScalarValue::Int16(a) if *point < 0 => { + round_integer_scalar!(a, point, ScalarValue::Int16, i16) + } + ScalarValue::Int8(a) if *point < 0 => { + round_integer_scalar!(a, point, ScalarValue::Int8, i8) + } + ScalarValue::Decimal128(a, _, scale) if *scale >= 0 => { + let f = decimal_round_f(scale, point); + let (precision, scale) = get_precision_scale(data_type); + make_decimal_scalar(a, precision, scale, &f) + } + _ => make_scalar_function(math_expressions::round)(args), + }, + } +} + +/// Similar to DataFusion `rpad`, but not to truncate when the string is already longer than length +fn spark_rpad(args: &[ColumnarValue]) -> Result<ColumnarValue, DataFusionError> { + match args { + [ColumnarValue::Array(array), ColumnarValue::Scalar(ScalarValue::Int32(Some(length)))] => { + match args[0].data_type() { + DataType::Utf8 => spark_rpad_internal::<i32>(array, *length), + DataType::LargeUtf8 => spark_rpad_internal::<i64>(array, *length), + // TODO: handle Dictionary types + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {other:?} for function rpad", + ))), + } + } + other => Err(DataFusionError::Internal(format!( + "Unsupported arguments {other:?} for function rpad", + ))), + } +} + +fn spark_rpad_internal<T: OffsetSizeTrait>( + array: &ArrayRef, + length: i32, +) -> Result<ColumnarValue, DataFusionError> { + let string_array = as_generic_string_array::<T>(array)?; + + let result = string_array + .iter() + .map(|string| match string { + Some(string) => { + let length = if length < 0 { 0 } else { length as usize }; + if length == 0 { + Ok(Some("".to_string())) + } else { + let graphemes = string.graphemes(true).collect::<Vec<&str>>(); + if length < graphemes.len() { + Ok(Some(string.to_string())) + } else { + let mut s = string.to_string(); + s.push_str(" ".repeat(length - graphemes.len()).as_str()); + Ok(Some(s)) + } + } + } + _ => Ok(None), + }) + .collect::<Result<GenericStringArray<T>, DataFusionError>>()?; + Ok(ColumnarValue::Array(Arc::new(result))) +} + +// Let Decimal(p3, s3) as return type i.e. Decimal(p1, s1) / Decimal(p2, s2) = Decimal(p3, s3). +// Conversely, Decimal(p1, s1) = Decimal(p2, s2) * Decimal(p3, s3). This means that, in order to +// get enough scale that matches with Spark behavior, it requires to widen s1 to s2 + s3 + 1. Since +// both s2 and s3 are 38 at max., s1 is 77 at max. DataFusion division cannot handle such scale > +// Decimal256Type::MAX_SCALE. Therefore, we need to implement this decimal division using BigInt. +fn spark_decimal_div( + args: &[ColumnarValue], + data_type: &DataType, +) -> Result<ColumnarValue, DataFusionError> { + let left = &args[0]; + let right = &args[1]; + let (p3, s3) = get_precision_scale(data_type); + + let (left, right): (ArrayRef, ArrayRef) = match (left, right) { + (ColumnarValue::Array(l), ColumnarValue::Array(r)) => (l.clone(), r.clone()), + (ColumnarValue::Scalar(l), ColumnarValue::Array(r)) => { + (l.to_array_of_size(r.len())?, r.clone()) + } + (ColumnarValue::Array(l), ColumnarValue::Scalar(r)) => { + (l.clone(), r.to_array_of_size(l.len())?) + } + (ColumnarValue::Scalar(l), ColumnarValue::Scalar(r)) => (l.to_array()?, r.to_array()?), + }; + let left = left.as_primitive::<Decimal128Type>(); + let right = right.as_primitive::<Decimal128Type>(); + let (_, s1) = get_precision_scale(left.data_type()); + let (_, s2) = get_precision_scale(right.data_type()); + + let ten = BigInt::from(10); + let l_exp = ((s2 + s3 + 1) as u32).saturating_sub(s1 as u32); + let r_exp = (s1 as u32).saturating_sub((s2 + s3 + 1) as u32); + let l_mul = ten.pow(l_exp); + let r_mul = ten.pow(r_exp); + let five = BigInt::from(5); + let result: Decimal128Array = arrow::compute::kernels::arity::binary(left, right, |l, r| { + let l = BigInt::from(l) * &l_mul; + let r = BigInt::from(r) * &r_mul; + let div = &l / &r; + let res = if div.is_negative() { + div - &five + } else { + div + &five + } / &ten; + res.to_i128().unwrap_or(i128::MAX) + })?; + let result = result.with_data_type(DataType::Decimal128(p3, s3)); + Ok(ColumnarValue::Array(Arc::new(result))) +} diff --git a/core/src/execution/datafusion/expressions/strings.rs b/core/src/execution/datafusion/expressions/strings.rs new file mode 100644 index 0000000000..ee9a22212b --- /dev/null +++ b/core/src/execution/datafusion/expressions/strings.rs @@ -0,0 +1,305 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#![allow(deprecated)] + +use crate::execution::{ + datafusion::expressions::utils::down_cast_any_ref, + kernels::strings::{string_space, substring}, +}; +use arrow::{ + compute::{ + contains_dyn, contains_utf8_scalar_dyn, ends_with_dyn, ends_with_utf8_scalar_dyn, like_dyn, + like_utf8_scalar_dyn, starts_with_dyn, starts_with_utf8_scalar_dyn, + }, + record_batch::RecordBatch, +}; +use arrow_schema::{DataType, Schema}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{DataFusionError, ScalarValue::Utf8}; +use datafusion_physical_expr::PhysicalExpr; +use std::{ + any::Any, + fmt::{Display, Formatter}, + hash::{Hash, Hasher}, + sync::Arc, +}; + +macro_rules! make_predicate_function { + ($name: ident, $kernel: ident, $str_scalar_kernel: ident) => { + #[derive(Debug, Hash)] + pub struct $name { + left: Arc<dyn PhysicalExpr>, + right: Arc<dyn PhysicalExpr>, + } + + impl $name { + pub fn new(left: Arc<dyn PhysicalExpr>, right: Arc<dyn PhysicalExpr>) -> Self { + Self { left, right } + } + } + + impl Display for $name { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "$name [left: {}, right: {}]", self.left, self.right) + } + } + + impl PartialEq<dyn Any> for $name { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.left.eq(&x.left) && self.right.eq(&x.right)) + .unwrap_or(false) + } + } + + impl PhysicalExpr for $name { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _: &Schema) -> datafusion_common::Result<DataType> { + Ok(DataType::Boolean) + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let left_arg = self.left.evaluate(batch)?; + let right_arg = self.right.evaluate(batch)?; + + let array = match (left_arg, right_arg) { + // array (op) scalar + (ColumnarValue::Array(array), ColumnarValue::Scalar(Utf8(Some(string)))) => { + $str_scalar_kernel(&array, string.as_str()) + } + (ColumnarValue::Array(_), ColumnarValue::Scalar(other)) => { + return Err(DataFusionError::Execution(format!( + "Should be String but got: {:?}", + other + ))) + } + // array (op) array + (ColumnarValue::Array(array1), ColumnarValue::Array(array2)) => { + $kernel(&array1, &array2) + } + // scalar (op) scalar should be folded at Spark optimizer + _ => { + return Err(DataFusionError::Execution( + "Predicate on two literals should be folded at Spark".to_string(), + )) + } + }?; + + Ok(ColumnarValue::Array(Arc::new(array))) + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.left.clone(), self.right.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> datafusion_common::Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new($name::new( + children[0].clone(), + children[1].clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.left.hash(&mut s); + self.right.hash(&mut s); + self.hash(&mut s); + } + } + }; +} + +make_predicate_function!(Like, like_dyn, like_utf8_scalar_dyn); + +make_predicate_function!(StartsWith, starts_with_dyn, starts_with_utf8_scalar_dyn); + +make_predicate_function!(EndsWith, ends_with_dyn, ends_with_utf8_scalar_dyn); + +make_predicate_function!(Contains, contains_dyn, contains_utf8_scalar_dyn); + +// make_predicate_function!(RLike, rlike_dyn, rlike_utf8_scalar_dyn); + +#[derive(Debug, Hash)] +pub struct SubstringExec { + pub child: Arc<dyn PhysicalExpr>, + pub start: i64, + pub len: u64, +} + +#[derive(Debug, Hash)] +pub struct StringSpaceExec { + pub child: Arc<dyn PhysicalExpr>, +} + +impl SubstringExec { + pub fn new(child: Arc<dyn PhysicalExpr>, start: i64, len: u64) -> Self { + Self { child, start, len } + } +} + +impl StringSpaceExec { + pub fn new(child: Arc<dyn PhysicalExpr>) -> Self { + Self { child } + } +} + +impl Display for SubstringExec { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "StringSpace [start: {}, len: {}, child: {}]", + self.start, self.len, self.child + ) + } +} + +impl Display for StringSpaceExec { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "StringSpace [child: {}] ", self.child) + } +} + +impl PartialEq<dyn Any> for SubstringExec { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.child.eq(&x.child) && self.start.eq(&x.start) && self.len.eq(&x.len)) + .unwrap_or(false) + } +} + +impl PhysicalExpr for SubstringExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result<DataType> { + self.child.data_type(input_schema) + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let result = substring(&array, self.start, self.len)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Substring(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> datafusion_common::Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(SubstringExec::new( + children[0].clone(), + self.start, + self.len, + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.start.hash(&mut s); + self.len.hash(&mut s); + self.hash(&mut s); + } +} + +impl PartialEq<dyn Any> for StringSpaceExec { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.child.eq(&x.child)) + .unwrap_or(false) + } +} + +impl PhysicalExpr for StringSpaceExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result<DataType> { + match self.child.data_type(input_schema)? { + DataType::Dictionary(key_type, _) => { + Ok(DataType::Dictionary(key_type, Box::new(DataType::Utf8))) + } + _ => Ok(DataType::Utf8), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let result = string_space(&array)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "StringSpace(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> datafusion_common::Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(StringSpaceExec::new(children[0].clone()))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.hash(&mut s); + } +} diff --git a/core/src/execution/datafusion/expressions/subquery.rs b/core/src/execution/datafusion/expressions/subquery.rs new file mode 100644 index 0000000000..a82fb357c4 --- /dev/null +++ b/core/src/execution/datafusion/expressions/subquery.rs @@ -0,0 +1,211 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::RecordBatch; +use arrow_schema::{DataType, Schema, TimeUnit}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{internal_err, DataFusionError, ScalarValue}; +use datafusion_physical_expr::PhysicalExpr; +use jni::sys::{jboolean, jbyte, jint, jlong, jshort}; +use std::{ + any::Any, + fmt::{Display, Formatter}, + hash::{Hash, Hasher}, + sync::Arc, +}; + +use crate::{ + execution::{datafusion::expressions::utils::down_cast_any_ref, utils::bytes_to_i128}, + jvm_bridge::{jni_static_call, BinaryWrapper, JVMClasses, StringWrapper}, +}; + +#[derive(Debug, Hash)] +pub struct Subquery { + /// The ID of the execution context that owns this subquery. We use this ID to retrieve the + /// subquery result. + exec_context_id: i64, + /// The ID of the subquery, we retrieve the subquery result from JVM using this ID. + pub id: i64, + /// The data type of the subquery result. + pub data_type: DataType, +} + +impl Subquery { + pub fn new(exec_context_id: i64, id: i64, data_type: DataType) -> Self { + Self { + exec_context_id, + id, + data_type, + } + } +} + +impl Display for Subquery { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "Subquery [id: {}]", self.id) + } +} + +impl PartialEq<dyn Any> for Subquery { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| { + self.id.eq(&x.id) + && self.data_type.eq(&x.data_type) + && self.exec_context_id.eq(&x.exec_context_id) + }) + .unwrap_or(false) + } +} + +impl PhysicalExpr for Subquery { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _: &Schema) -> datafusion_common::Result<DataType> { + Ok(self.data_type.clone()) + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, _: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let env = JVMClasses::get_env(); + + let is_null = + jni_static_call!(env, comet_exec.is_null(self.exec_context_id, self.id) -> jboolean)?; + + if is_null > 0 { + return Ok(ColumnarValue::Scalar(ScalarValue::try_from( + &self.data_type, + )?)); + } + + match &self.data_type { + DataType::Boolean => { + let r = jni_static_call!(env, + comet_exec.get_bool(self.exec_context_id, self.id) -> jboolean + )?; + Ok(ColumnarValue::Scalar(ScalarValue::Boolean(Some(r > 0)))) + } + DataType::Int8 => { + let r = jni_static_call!(env, + comet_exec.get_byte(self.exec_context_id, self.id) -> jbyte + )?; + Ok(ColumnarValue::Scalar(ScalarValue::Int8(Some(r)))) + } + DataType::Int16 => { + let r = jni_static_call!(env, + comet_exec.get_short(self.exec_context_id, self.id) -> jshort + )?; + Ok(ColumnarValue::Scalar(ScalarValue::Int16(Some(r)))) + } + DataType::Int32 => { + let r = jni_static_call!(env, + comet_exec.get_int(self.exec_context_id, self.id) -> jint + )?; + Ok(ColumnarValue::Scalar(ScalarValue::Int32(Some(r)))) + } + DataType::Int64 => { + let r = jni_static_call!(env, + comet_exec.get_long(self.exec_context_id, self.id) -> jlong + )?; + Ok(ColumnarValue::Scalar(ScalarValue::Int64(Some(r)))) + } + DataType::Float32 => { + let r = jni_static_call!(env, + comet_exec.get_float(self.exec_context_id, self.id) -> f32 + )?; + Ok(ColumnarValue::Scalar(ScalarValue::Float32(Some(r)))) + } + DataType::Float64 => { + let r = jni_static_call!(env, + comet_exec.get_double(self.exec_context_id, self.id) -> f64 + )?; + + Ok(ColumnarValue::Scalar(ScalarValue::Float64(Some(r)))) + } + DataType::Decimal128(p, s) => { + let bytes = jni_static_call!(env, + comet_exec.get_decimal(self.exec_context_id, self.id) -> BinaryWrapper + )?; + + let slice = env.convert_byte_array((*bytes.get()).into_inner()).unwrap(); + + Ok(ColumnarValue::Scalar(ScalarValue::Decimal128( + Some(bytes_to_i128(&slice)), + *p, + *s, + ))) + } + DataType::Date32 => { + let r = jni_static_call!(env, + comet_exec.get_int(self.exec_context_id, self.id) -> jint + )?; + + Ok(ColumnarValue::Scalar(ScalarValue::Date32(Some(r)))) + } + DataType::Timestamp(TimeUnit::Microsecond, timezone) => { + let r = jni_static_call!(env, + comet_exec.get_long(self.exec_context_id, self.id) -> jlong + )?; + + Ok(ColumnarValue::Scalar(ScalarValue::TimestampMicrosecond( + Some(r), + timezone.clone(), + ))) + } + DataType::Utf8 => { + let string = jni_static_call!(env, + comet_exec.get_string(self.exec_context_id, self.id) -> StringWrapper + )?; + + let string = env.get_string(*string.get()).unwrap().into(); + Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(string)))) + } + DataType::Binary => { + let bytes = jni_static_call!(env, + comet_exec.get_binary(self.exec_context_id, self.id) -> BinaryWrapper + )?; + + let slice = env.convert_byte_array((*bytes.get()).into_inner()).unwrap(); + + Ok(ColumnarValue::Scalar(ScalarValue::Binary(Some(slice)))) + } + _ => internal_err!("Unsupported scalar subquery data type {:?}", self.data_type), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![] + } + + fn with_new_children( + self: Arc<Self>, + _: Vec<Arc<dyn PhysicalExpr>>, + ) -> datafusion_common::Result<Arc<dyn PhysicalExpr>> { + Ok(self) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.hash(&mut s) + } +} diff --git a/core/src/execution/datafusion/expressions/sum_decimal.rs b/core/src/execution/datafusion/expressions/sum_decimal.rs new file mode 100644 index 0000000000..d7dc79e6fd --- /dev/null +++ b/core/src/execution/datafusion/expressions/sum_decimal.rs @@ -0,0 +1,455 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::BooleanBufferBuilder, buffer::NullBuffer}; +use arrow_array::{ + cast::AsArray, types::Decimal128Type, Array, ArrayRef, BooleanArray, Decimal128Array, +}; +use arrow_data::decimal::validate_decimal_precision; +use arrow_schema::{DataType, Field}; +use datafusion::logical_expr::Accumulator; +use datafusion_common::{Result as DFResult, ScalarValue}; +use datafusion_physical_expr::{ + aggregate::utils::down_cast_any_ref, AggregateExpr, EmitTo, GroupsAccumulator, PhysicalExpr, +}; +use std::{any::Any, ops::BitAnd, sync::Arc}; + +use crate::unlikely; + +#[derive(Debug)] +pub struct SumDecimal { + name: String, + expr: Arc<dyn PhysicalExpr>, + + /// The data type of the SUM result + result_type: DataType, + + /// Decimal precision and scale + precision: u8, + scale: i8, + + /// Whether the result is nullable + nullable: bool, +} + +impl SumDecimal { + pub fn new(name: impl Into<String>, expr: Arc<dyn PhysicalExpr>, data_type: DataType) -> Self { + // The `data_type` is the SUM result type passed from Spark side + let (precision, scale) = match data_type { + DataType::Decimal128(p, s) => (p, s), + _ => unreachable!(), + }; + Self { + name: name.into(), + expr, + result_type: data_type, + precision, + scale, + nullable: true, + } + } +} + +impl AggregateExpr for SumDecimal { + fn as_any(&self) -> &dyn Any { + self + } + + fn field(&self) -> DFResult<Field> { + Ok(Field::new( + &self.name, + self.result_type.clone(), + self.nullable, + )) + } + + fn create_accumulator(&self) -> DFResult<Box<dyn Accumulator>> { + Ok(Box::new(SumDecimalAccumulator::new( + self.precision, + self.scale, + ))) + } + + fn state_fields(&self) -> DFResult<Vec<Field>> { + let fields = vec![ + Field::new(&self.name, self.result_type.clone(), self.nullable), + Field::new("is_empty", DataType::Boolean, false), + ]; + Ok(fields) + } + + fn expressions(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.expr.clone()] + } + + fn name(&self) -> &str { + &self.name + } + + fn groups_accumulator_supported(&self) -> bool { + true + } + + fn create_groups_accumulator(&self) -> DFResult<Box<dyn GroupsAccumulator>> { + Ok(Box::new(SumDecimalGroupsAccumulator::new( + self.result_type.clone(), + self.precision, + self.scale, + ))) + } +} + +impl PartialEq<dyn Any> for SumDecimal { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| { + self.name == x.name + && self.precision == x.precision + && self.scale == x.scale + && self.nullable == x.nullable + && self.result_type == x.result_type + && self.expr.eq(&x.expr) + }) + .unwrap_or(false) + } +} + +#[derive(Debug)] +struct SumDecimalAccumulator { + sum: i128, + is_empty: bool, + is_not_null: bool, + + precision: u8, + scale: i8, +} + +impl SumDecimalAccumulator { + fn new(precision: u8, scale: i8) -> Self { + Self { + sum: 0, + is_empty: true, + is_not_null: true, + precision, + scale, + } + } + + fn update_single(&mut self, values: &Decimal128Array, idx: usize) { + let v = unsafe { values.value_unchecked(idx) }; + let (new_sum, is_overflow) = self.sum.overflowing_add(v); + + if is_overflow || validate_decimal_precision(new_sum, self.precision).is_err() { + // Overflow: set buffer accumulator to null + self.is_not_null = false; + return; + } + + self.sum = new_sum; + self.is_not_null = true; + } +} + +impl Accumulator for SumDecimalAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> DFResult<()> { + assert_eq!( + values.len(), + 1, + "Expect only one element in 'values' but found {}", + values.len() + ); + + if !self.is_empty && !self.is_not_null { + // This means there's a overflow in decimal, so we will just skip the rest + // of the computation + return Ok(()); + } + + let values = &values[0]; + let data = values.as_primitive::<Decimal128Type>(); + + self.is_empty = self.is_empty && values.len() == values.null_count(); + + if values.null_count() == 0 { + for i in 0..data.len() { + self.update_single(data, i); + } + } else { + for i in 0..data.len() { + if data.is_null(i) { + continue; + } + self.update_single(data, i); + } + } + + Ok(()) + } + + fn evaluate(&self) -> DFResult<ScalarValue> { + // For each group: + // 1. if `is_empty` is true, it means either there is no value or all values for the group + // are null, in this case we'll return null + // 2. if `is_empty` is false, but `null_state` is true, it means there's an overflow. In + // non-ANSI mode Spark returns null. + if self.is_empty || !self.is_not_null { + ScalarValue::new_primitive::<Decimal128Type>( + None, + &DataType::Decimal128(self.precision, self.scale), + ) + } else { + ScalarValue::try_new_decimal128(self.sum, self.precision, self.scale) + } + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&self) -> DFResult<Vec<ScalarValue>> { + let sum = if self.is_not_null { + ScalarValue::try_new_decimal128(self.sum, self.precision, self.scale)? + } else { + ScalarValue::new_primitive::<Decimal128Type>( + None, + &DataType::Decimal128(self.precision, self.scale), + )? + }; + Ok(vec![sum, ScalarValue::from(self.is_empty)]) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> DFResult<()> { + assert_eq!( + states.len(), + 2, + "Expect two element in 'states' but found {}", + states.len() + ); + assert_eq!(states[0].len(), 1); + assert_eq!(states[1].len(), 1); + + let that_sum = states[0].as_primitive::<Decimal128Type>(); + let that_is_empty = states[1].as_any().downcast_ref::<BooleanArray>().unwrap(); + + let this_overflow = !self.is_empty && !self.is_not_null; + let that_overflow = !that_is_empty.value(0) && that_sum.is_null(0); + + self.is_not_null = !this_overflow && !that_overflow; + self.is_empty = self.is_empty && that_is_empty.value(0); + + if self.is_not_null { + self.sum += that_sum.value(0); + } + + Ok(()) + } +} + +struct SumDecimalGroupsAccumulator { + // Whether aggregate buffer for a particular group is null. True indicates it is not null. + is_not_null: BooleanBufferBuilder, + is_empty: BooleanBufferBuilder, + sum: Vec<i128>, + result_type: DataType, + precision: u8, + scale: i8, +} + +impl SumDecimalGroupsAccumulator { + fn new(result_type: DataType, precision: u8, scale: i8) -> Self { + Self { + is_not_null: BooleanBufferBuilder::new(0), + is_empty: BooleanBufferBuilder::new(0), + sum: Vec::new(), + result_type, + precision, + scale, + } + } + + fn is_overflow(&self, index: usize) -> bool { + !self.is_empty.get_bit(index) && !self.is_not_null.get_bit(index) + } + + fn update_single(&mut self, group_index: usize, value: i128) { + if unlikely(self.is_overflow(group_index)) { + // This means there's a overflow in decimal, so we will just skip the rest + // of the computation + return; + } + + self.is_empty.set_bit(group_index, false); + let (new_sum, is_overflow) = self.sum[group_index].overflowing_add(value); + + if is_overflow || validate_decimal_precision(new_sum, self.precision).is_err() { + // Overflow: set buffer accumulator to null + self.is_not_null.set_bit(group_index, false); + return; + } + + self.sum[group_index] = new_sum; + self.is_not_null.set_bit(group_index, true) + } +} + +fn ensure_bit_capacity(builder: &mut BooleanBufferBuilder, capacity: usize) { + if builder.len() < capacity { + let additional = capacity - builder.len(); + builder.append_n(additional, true); + } +} + +impl GroupsAccumulator for SumDecimalGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> DFResult<()> { + assert!(opt_filter.is_none(), "opt_filter is not supported yet"); + assert_eq!(values.len(), 1); + let values = values[0].as_primitive::<Decimal128Type>(); + let data = values.values(); + + // Update size for the accumulate states + self.sum.resize(total_num_groups, 0); + ensure_bit_capacity(&mut self.is_empty, total_num_groups); + ensure_bit_capacity(&mut self.is_not_null, total_num_groups); + + let iter = group_indices.iter().zip(data.iter()); + if values.null_count() == 0 { + for (&group_index, &value) in iter { + self.update_single(group_index, value); + } + } else { + for (idx, (&group_index, &value)) in iter.enumerate() { + if values.is_null(idx) { + continue; + } + self.update_single(group_index, value); + } + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> DFResult<ArrayRef> { + // TODO: we do not support group-by ordering yet, but should fix here once it is supported + assert!( + matches!(emit_to, EmitTo::All), + "EmitTo::First is not supported" + ); + // For each group: + // 1. if `is_empty` is true, it means either there is no value or all values for the group + // are null, in this case we'll return null + // 2. if `is_empty` is false, but `null_state` is true, it means there's an overflow. In + // non-ANSI mode Spark returns null. + let nulls = self.is_not_null.finish(); + let is_empty = self.is_empty.finish(); + let x = (!&is_empty).bitand(&nulls); + + let result = emit_to.take_needed(&mut self.sum); + let result = Decimal128Array::new(result.into(), Some(NullBuffer::new(x))) + .with_data_type(self.result_type.clone()); + + Ok(Arc::new(result)) + } + + fn state(&mut self, emit_to: EmitTo) -> DFResult<Vec<ArrayRef>> { + // TODO: we do not support group-by ordering yet, but should fix here once it is supported + assert!( + matches!(emit_to, EmitTo::All), + "EmitTo::First is not supported" + ); + let nulls = self.is_not_null.finish(); + let nulls = Some(NullBuffer::new(nulls)); + + let sum = emit_to.take_needed(&mut self.sum); + let sum = Decimal128Array::new(sum.into(), nulls.clone()) + .with_data_type(self.result_type.clone()); + + let is_empty = self.is_empty.finish(); + let is_empty = BooleanArray::new(is_empty, None); + + Ok(vec![ + Arc::new(sum) as ArrayRef, + Arc::new(is_empty) as ArrayRef, + ]) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> DFResult<()> { + assert_eq!( + values.len(), + 2, + "Expected two arrays: 'sum' and 'is_empty', but found {}", + values.len() + ); + assert!(opt_filter.is_none(), "opt_filter is not supported yet"); + + // Make sure we have enough capacity for the additional groups + self.sum.resize(total_num_groups, 0); + ensure_bit_capacity(&mut self.is_empty, total_num_groups); + ensure_bit_capacity(&mut self.is_not_null, total_num_groups); + + let that_sum = &values[0]; + let that_sum = that_sum.as_primitive::<Decimal128Type>(); + let that_is_empty = &values[1]; + let that_is_empty = that_is_empty + .as_any() + .downcast_ref::<BooleanArray>() + .unwrap(); + + group_indices + .iter() + .enumerate() + .for_each(|(idx, &group_index)| unsafe { + let this_overflow = self.is_overflow(group_index); + let that_is_empty = that_is_empty.value_unchecked(idx); + let that_overflow = !that_is_empty && that_sum.is_null(idx); + let is_overflow = this_overflow || that_overflow; + + // This part follows the logic in Spark: + // `org.apache.spark.sql.catalyst.expressions.aggregate.Sum` + self.is_not_null.set_bit(group_index, !is_overflow); + self.is_empty.set_bit( + group_index, + self.is_empty.get_bit(group_index) && that_is_empty, + ); + if !is_overflow { + // .. otherwise, the sum value for this particular index must not be null, + // and thus we merge both values and update this sum. + self.sum[group_index] += that_sum.value_unchecked(idx); + } + }); + + Ok(()) + } + + fn size(&self) -> usize { + self.sum.capacity() * std::mem::size_of::<i128>() + + self.is_empty.capacity() / 8 + + self.is_not_null.capacity() / 8 + } +} diff --git a/core/src/execution/datafusion/expressions/temporal.rs b/core/src/execution/datafusion/expressions/temporal.rs new file mode 100644 index 0000000000..3654a4ed9f --- /dev/null +++ b/core/src/execution/datafusion/expressions/temporal.rs @@ -0,0 +1,518 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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, Display, Formatter}, + hash::{Hash, Hasher}, + sync::Arc, +}; + +use arrow::{ + compute::{hour_dyn, minute_dyn, second_dyn}, + record_batch::RecordBatch, +}; +use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{DataFusionError, ScalarValue::Utf8}; +use datafusion_physical_expr::PhysicalExpr; + +use crate::execution::{ + datafusion::expressions::utils::{array_with_timezone, down_cast_any_ref}, + kernels::temporal::{date_trunc_dyn, timestamp_trunc_dyn}, +}; + +#[derive(Debug, Hash)] +pub struct HourExec { + /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) + child: Arc<dyn PhysicalExpr>, + timezone: String, +} + +impl HourExec { + pub fn new(child: Arc<dyn PhysicalExpr>, timezone: String) -> Self { + HourExec { child, timezone } + } +} + +impl Display for HourExec { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Hour [timezone:{}, child: {}]", + self.timezone, self.child + ) + } +} + +impl PartialEq<dyn Any> for HourExec { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) + .unwrap_or(false) + } +} + +impl PhysicalExpr for HourExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result<DataType> { + match self.child.data_type(input_schema).unwrap() { + DataType::Dictionary(key_type, _) => { + Ok(DataType::Dictionary(key_type, Box::new(DataType::Int32))) + } + _ => Ok(DataType::Int32), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let array = array_with_timezone( + array, + self.timezone.clone(), + Some(&DataType::Timestamp( + Microsecond, + Some(self.timezone.clone().into()), + )), + ); + let result = hour_dyn(&array)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Hour(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> Result<Arc<dyn PhysicalExpr>, DataFusionError> { + Ok(Arc::new(HourExec::new( + children[0].clone(), + self.timezone.clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.timezone.hash(&mut s); + self.hash(&mut s); + } +} + +#[derive(Debug, Hash)] +pub struct MinuteExec { + /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) + child: Arc<dyn PhysicalExpr>, + timezone: String, +} + +impl MinuteExec { + pub fn new(child: Arc<dyn PhysicalExpr>, timezone: String) -> Self { + MinuteExec { child, timezone } + } +} + +impl Display for MinuteExec { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Minute [timezone:{}, child: {}]", + self.timezone, self.child + ) + } +} + +impl PartialEq<dyn Any> for MinuteExec { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) + .unwrap_or(false) + } +} + +impl PhysicalExpr for MinuteExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result<DataType> { + match self.child.data_type(input_schema).unwrap() { + DataType::Dictionary(key_type, _) => { + Ok(DataType::Dictionary(key_type, Box::new(DataType::Int32))) + } + _ => Ok(DataType::Int32), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let array = array_with_timezone( + array, + self.timezone.clone(), + Some(&DataType::Timestamp( + Microsecond, + Some(self.timezone.clone().into()), + )), + ); + let result = minute_dyn(&array)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Minute(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> Result<Arc<dyn PhysicalExpr>, DataFusionError> { + Ok(Arc::new(MinuteExec::new( + children[0].clone(), + self.timezone.clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.timezone.hash(&mut s); + self.hash(&mut s); + } +} + +#[derive(Debug, Hash)] +pub struct SecondExec { + /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) + child: Arc<dyn PhysicalExpr>, + timezone: String, +} + +impl SecondExec { + pub fn new(child: Arc<dyn PhysicalExpr>, timezone: String) -> Self { + SecondExec { child, timezone } + } +} + +impl Display for SecondExec { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Second (timezone:{}, child: {}]", + self.timezone, self.child + ) + } +} + +impl PartialEq<dyn Any> for SecondExec { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) + .unwrap_or(false) + } +} + +impl PhysicalExpr for SecondExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result<DataType> { + match self.child.data_type(input_schema).unwrap() { + DataType::Dictionary(key_type, _) => { + Ok(DataType::Dictionary(key_type, Box::new(DataType::Int32))) + } + _ => Ok(DataType::Int32), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let arg = self.child.evaluate(batch)?; + match arg { + ColumnarValue::Array(array) => { + let array = array_with_timezone( + array, + self.timezone.clone(), + Some(&DataType::Timestamp( + Microsecond, + Some(self.timezone.clone().into()), + )), + ); + let result = second_dyn(&array)?; + + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Second(scalar) should be fold in Spark JVM side.".to_string(), + )), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> Result<Arc<dyn PhysicalExpr>, DataFusionError> { + Ok(Arc::new(SecondExec::new( + children[0].clone(), + self.timezone.clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.timezone.hash(&mut s); + self.hash(&mut s); + } +} + +#[derive(Debug, Hash)] +pub struct DateTruncExec { + /// An array with DataType::Date32 + child: Arc<dyn PhysicalExpr>, + /// Scalar UTF8 string matching the valid values in Spark SQL: https://spark.apache.org/docs/latest/api/sql/index.html#trunc + format: Arc<dyn PhysicalExpr>, +} + +impl DateTruncExec { + pub fn new(child: Arc<dyn PhysicalExpr>, format: Arc<dyn PhysicalExpr>) -> Self { + DateTruncExec { child, format } + } +} + +impl Display for DateTruncExec { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "DateTrunc [child:{}, format: {}]", + self.child, self.format + ) + } +} + +impl PartialEq<dyn Any> for DateTruncExec { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.child.eq(&x.child) && self.format.eq(&x.format)) + .unwrap_or(false) + } +} + +impl PhysicalExpr for DateTruncExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result<DataType> { + self.child.data_type(input_schema) + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let date = self.child.evaluate(batch)?; + let format = self.format.evaluate(batch)?; + match (date, format) { + (ColumnarValue::Array(date), ColumnarValue::Scalar(Utf8(Some(format)))) => { + let result = date_trunc_dyn(&date, format)?; + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Invalid input to function DateTrunc. Expected (PrimitiveArray<Date32>, Scalar)" + .to_string(), + )), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> Result<Arc<dyn PhysicalExpr>, DataFusionError> { + Ok(Arc::new(DateTruncExec::new( + children[0].clone(), + self.format.clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.format.hash(&mut s); + self.hash(&mut s); + } +} + +#[derive(Debug, Hash)] +pub struct TimestampTruncExec { + /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) + child: Arc<dyn PhysicalExpr>, + /// Scalar UTF8 string matching the valid values in Spark SQL: https://spark.apache.org/docs/latest/api/sql/index.html#date_trunc + format: Arc<dyn PhysicalExpr>, + /// String containing a timezone name. The name must be found in the standard timezone + /// database (https://en.wikipedia.org/wiki/List_of_tz_database_time_zones). The string is + /// later parsed into a chrono::TimeZone. + /// Timestamp arrays in this implementation are kept in arrays of UTC timestamps (in micros) + /// along with a single value for the associated TimeZone. The timezone offset is applied + /// just before any operations on the timestamp + timezone: String, +} + +impl TimestampTruncExec { + pub fn new( + child: Arc<dyn PhysicalExpr>, + format: Arc<dyn PhysicalExpr>, + timezone: String, + ) -> Self { + TimestampTruncExec { + child, + format, + timezone, + } + } +} + +impl Display for TimestampTruncExec { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "TimestampTrunc [child:{}, format:{}, timezone: {}]", + self.child, self.format, self.timezone + ) + } +} + +impl PartialEq<dyn Any> for TimestampTruncExec { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| { + self.child.eq(&x.child) + && self.format.eq(&x.format) + && self.timezone.eq(&x.timezone) + }) + .unwrap_or(false) + } +} + +impl PhysicalExpr for TimestampTruncExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> datafusion_common::Result<DataType> { + match self.child.data_type(input_schema)? { + DataType::Dictionary(key_type, _) => Ok(DataType::Dictionary( + key_type, + Box::new(DataType::Timestamp(Microsecond, None)), + )), + _ => Ok(DataType::Timestamp(Microsecond, None)), + } + } + + fn nullable(&self, _: &Schema) -> datafusion_common::Result<bool> { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> datafusion_common::Result<ColumnarValue> { + let timestamp = self.child.evaluate(batch)?; + let format = self.format.evaluate(batch)?; + let tz = self.timezone.clone(); + match (timestamp, format) { + (ColumnarValue::Array(ts), ColumnarValue::Scalar(Utf8(Some(format)))) => { + let ts = array_with_timezone( + ts, + tz.clone(), + Some(&DataType::Timestamp(Microsecond, Some(tz.into()))), + ); + let result = timestamp_trunc_dyn(&ts, format)?; + Ok(ColumnarValue::Array(result)) + } + _ => Err(DataFusionError::Execution( + "Invalid input to function TimestampTrunc. ".to_owned() + + "Expected (PrimitiveArray<TimestampMicrosecondType>, Scalar, String)", + )), + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> Result<Arc<dyn PhysicalExpr>, DataFusionError> { + Ok(Arc::new(TimestampTruncExec::new( + children[0].clone(), + self.format.clone(), + self.timezone.clone(), + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.format.hash(&mut s); + self.timezone.hash(&mut s); + self.hash(&mut s); + } +} diff --git a/core/src/execution/datafusion/expressions/utils.rs b/core/src/execution/datafusion/expressions/utils.rs new file mode 100644 index 0000000000..ec0cf2207d --- /dev/null +++ b/core/src/execution/datafusion/expressions/utils.rs @@ -0,0 +1,268 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 crate::execution::timezone::Tz; +use arrow::{ + array::{ + as_dictionary_array, as_primitive_array, Array, ArrayRef, GenericStringArray, + PrimitiveArray, + }, + compute::unary, + datatypes::{Int32Type, Int64Type, TimestampMicrosecondType}, + error::ArrowError, + temporal_conversions::as_datetime, +}; +use arrow_array::{cast::AsArray, types::ArrowPrimitiveType}; +use arrow_schema::DataType; +use chrono::{DateTime, Offset, TimeZone}; +use datafusion_common::cast::as_generic_string_array; +use datafusion_physical_expr::PhysicalExpr; +use std::{any::Any, sync::Arc}; + +/// An utility function from DataFusion. It is not exposed by DataFusion. +pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { + if any.is::<Arc<dyn PhysicalExpr>>() { + any.downcast_ref::<Arc<dyn PhysicalExpr>>() + .unwrap() + .as_any() + } else if any.is::<Box<dyn PhysicalExpr>>() { + any.downcast_ref::<Box<dyn PhysicalExpr>>() + .unwrap() + .as_any() + } else { + any + } +} + +/// Preprocesses input arrays to add timezone information from Spark to Arrow array datatype or +/// to apply timezone offset. +// +// We consider the following cases: +// +// | --------------------- | ------------ | ----------------- | -------------------------------- | +// | Conversion | Input array | Timezone | Output array | +// | --------------------- | ------------ | ----------------- | -------------------------------- | +// | Timestamp -> | Array in UTC | Timezone of input | A timestamp with the timezone | +// | Utf8 or Date32 | | | offset applied and timezone | +// | | | | removed | +// | --------------------- | ------------ | ----------------- | -------------------------------- | +// | Timestamp -> | Array in UTC | Timezone of input | Same as input array | +// | Timestamp w/Timezone| | | | +// | --------------------- | ------------ | ----------------- | -------------------------------- | +// | Timestamp_ntz -> | Array in | Timezone of input | Same as input array | +// | Utf8 or Date32 | timezone | | | +// | | session local| | | +// | | timezone | | | +// | --------------------- | ------------ | ----------------- | -------------------------------- | +// | Timestamp_ntz -> | Array in | Timezone of input | Array in UTC and timezone | +// | Timestamp w/Timezone | session local| | specified in input | +// | | timezone | | | +// | --------------------- | ------------ | ----------------- | -------------------------------- | +// | Timestamp(_ntz) -> | | +// | Any other type | Not Supported | +// | --------------------- | ------------ | ----------------- | -------------------------------- | +// +pub fn array_with_timezone( + array: ArrayRef, + timezone: String, + to_type: Option<&DataType>, +) -> ArrayRef { + match array.data_type() { + DataType::Timestamp(_, None) => { + assert!(!timezone.is_empty()); + match to_type { + Some(DataType::Utf8) | Some(DataType::Date32) => array, + Some(DataType::Timestamp(_, Some(_))) => { + timestamp_ntz_to_timestamp(array, timezone.as_str(), Some(timezone.as_str())) + } + _ => { + // Not supported + panic!( + "Cannot convert from {:?} to {:?}", + array.data_type(), + to_type.unwrap() + ) + } + } + } + DataType::Timestamp(_, Some(_)) => { + assert!(!timezone.is_empty()); + let array = as_primitive_array::<TimestampMicrosecondType>(&array); + let array_with_timezone = array.clone().with_timezone(timezone.clone()); + let array = Arc::new(array_with_timezone) as ArrayRef; + match to_type { + Some(DataType::Utf8) | Some(DataType::Date32) => { + pre_timestamp_cast(array, timezone) + } + _ => array, + } + } + DataType::Dictionary(_, value_type) + if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => + { + let dict = as_dictionary_array::<Int32Type>(&array); + let array = as_primitive_array::<TimestampMicrosecondType>(dict.values()); + let array_with_timezone = + array_with_timezone(Arc::new(array.clone()) as ArrayRef, timezone, to_type); + let dict = dict.with_values(array_with_timezone); + Arc::new(dict) as ArrayRef + } + _ => array, + } +} + +/// Takes in a Timestamp(Microsecond, None) array and a timezone id, and returns +/// a Timestamp(Microsecond, Some<_>) array. +/// The understanding is that the input array has time in the timezone specified in the second +/// argument. +/// Parameters: +/// array - input array of timestamp without timezone +/// tz - timezone of the values in the input array +/// to_timezone - timezone to change the input values to +fn timestamp_ntz_to_timestamp(array: ArrayRef, tz: &str, to_timezone: Option<&str>) -> ArrayRef { + assert!(!tz.is_empty()); + match array.data_type() { + DataType::Timestamp(_, None) => { + let array = as_primitive_array::<TimestampMicrosecondType>(&array); + let tz: Tz = tz.parse().unwrap(); + let values = array.iter().map(|v| { + v.map(|value| { + let local_datetime = as_datetime::<TimestampMicrosecondType>(value).unwrap(); + let datetime: DateTime<Tz> = tz.from_local_datetime(&local_datetime).unwrap(); + datetime.timestamp_micros() + }) + }); + let mut array: PrimitiveArray<TimestampMicrosecondType> = + unsafe { PrimitiveArray::from_trusted_len_iter(values) }; + array = if let Some(to_tz) = to_timezone { + array.with_timezone(to_tz) + } else { + array + }; + Arc::new(array) as ArrayRef + } + _ => array, + } +} + +const MICROS_PER_SECOND: i64 = 1000000; + +/// This takes for special pre-casting cases of Spark. E.g., Timestamp to String. +fn pre_timestamp_cast(array: ArrayRef, timezone: String) -> ArrayRef { + assert!(!timezone.is_empty()); + match array.data_type() { + DataType::Timestamp(_, _) => { + // Spark doesn't output timezone while casting timestamp to string, but arrow's cast + // kernel does if timezone exists. So we need to apply offset of timezone to array + // timestamp value and remove timezone from array datatype. + let array = as_primitive_array::<TimestampMicrosecondType>(&array); + + let tz: Tz = timezone.parse().unwrap(); + let values = array.iter().map(|v| { + v.map(|value| { + let datetime = as_datetime::<TimestampMicrosecondType>(value).unwrap(); + let offset = tz.offset_from_utc_datetime(&datetime).fix(); + let datetime = datetime + offset; + datetime.timestamp_micros() + }) + }); + + let array: PrimitiveArray<TimestampMicrosecondType> = + unsafe { PrimitiveArray::from_trusted_len_iter(values) }; + Arc::new(array) as ArrayRef + } + _ => array, + } +} + +/// This takes for special casting cases of Spark. E.g., Timestamp to Long. +/// This function runs as a post process of the DataFusion cast(). By the time it arrives here, +/// Dictionary arrays are already unpacked by the DataFusion cast() since Spark cannot specify +/// Dictionary as to_type. The from_type is taken before the DataFusion cast() runs in +/// expressions/cast.rs, so it can be still Dictionary. +pub(crate) fn spark_cast(array: ArrayRef, from_type: &DataType, to_type: &DataType) -> ArrayRef { + match (from_type, to_type) { + (DataType::Timestamp(_, _), DataType::Int64) => { + // See Spark's `Cast` expression + unary_dyn::<_, Int64Type>(&array, |v| v.div_floor(MICROS_PER_SECOND)).unwrap() + } + (DataType::Dictionary(_, value_type), DataType::Int64) + if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => + { + // See Spark's `Cast` expression + unary_dyn::<_, Int64Type>(&array, |v| v.div_floor(MICROS_PER_SECOND)).unwrap() + } + (DataType::Timestamp(_, _), DataType::Utf8) => remove_trailing_zeroes(array), + (DataType::Dictionary(_, value_type), DataType::Utf8) + if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => + { + remove_trailing_zeroes(array) + } + _ => array, + } +} + +/// A fork & modified version of Arrow's `unary_dyn` which is being deprecated +fn unary_dyn<F, T>(array: &ArrayRef, op: F) -> Result<ArrayRef, ArrowError> +where + T: ArrowPrimitiveType, + F: Fn(T::Native) -> T::Native, +{ + if let Some(d) = array.as_any_dictionary_opt() { + let new_values = unary_dyn::<F, T>(d.values(), op)?; + return Ok(Arc::new(d.with_values(Arc::new(new_values)))); + } + + match array.as_primitive_opt::<T>() { + Some(a) if PrimitiveArray::<T>::is_compatible(a.data_type()) => { + Ok(Arc::new(unary::<T, F, T>( + array.as_any().downcast_ref::<PrimitiveArray<T>>().unwrap(), + op, + ))) + } + _ => Err(ArrowError::NotYetImplemented(format!( + "Cannot perform unary operation of type {} on array of type {}", + T::DATA_TYPE, + array.data_type() + ))), + } +} + +/// Remove any trailing zeroes in the string if they occur after in the fractional seconds, +/// to match Spark behavior +/// example: +/// "1970-01-01 05:29:59.900" => "1970-01-01 05:29:59.9" +/// "1970-01-01 05:29:59.990" => "1970-01-01 05:29:59.99" +/// "1970-01-01 05:29:59.999" => "1970-01-01 05:29:59.999" +/// "1970-01-01 05:30:00" => "1970-01-01 05:30:00" +/// "1970-01-01 05:30:00.001" => "1970-01-01 05:30:00.001" +fn remove_trailing_zeroes(array: ArrayRef) -> ArrayRef { + let string_array = as_generic_string_array::<i32>(&array).unwrap(); + let result = string_array + .iter() + .map(|s| s.map(trim_end)) + .collect::<GenericStringArray<i32>>(); + Arc::new(result) as ArrayRef +} + +fn trim_end(s: &str) -> &str { + if s.rfind('.').is_some() { + s.trim_end_matches('0') + } else { + s + } +} diff --git a/core/src/execution/datafusion/mod.rs b/core/src/execution/datafusion/mod.rs new file mode 100644 index 0000000000..b74ee26385 --- /dev/null +++ b/core/src/execution/datafusion/mod.rs @@ -0,0 +1,23 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Native execution through DataFusion + +mod expressions; +mod operators; +pub mod planner; +mod spark_hash; diff --git a/core/src/execution/datafusion/operators/expand.rs b/core/src/execution/datafusion/operators/expand.rs new file mode 100644 index 0000000000..e3f681b77b --- /dev/null +++ b/core/src/execution/datafusion/operators/expand.rs @@ -0,0 +1,205 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::RecordBatch; +use arrow_schema::SchemaRef; +use datafusion::{ + execution::TaskContext, + physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, + }, +}; +use datafusion_common::DataFusionError; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use futures::{Stream, StreamExt}; +use std::{ + any::Any, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; + +/// A Comet native operator that expands a single row into multiple rows. This behaves as same as +/// Spark Expand operator. +#[derive(Debug)] +pub struct CometExpandExec { + projections: Vec<Vec<Arc<dyn PhysicalExpr>>>, + child: Arc<dyn ExecutionPlan>, + schema: SchemaRef, +} + +impl CometExpandExec { + /// Create a new ExpandExec + pub fn new( + projections: Vec<Vec<Arc<dyn PhysicalExpr>>>, + child: Arc<dyn ExecutionPlan>, + schema: SchemaRef, + ) -> Self { + Self { + projections, + child, + schema, + } + } +} + +impl DisplayAs for CometExpandExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "CometExpandExec")?; + write!(f, "Projections: [")?; + for projection in &self.projections { + write!(f, "[")?; + for expr in projection { + write!(f, "{}, ", expr)?; + } + write!(f, "], ")?; + } + write!(f, "]")?; + + Ok(()) + } + } + } +} + +impl ExecutionPlan for CometExpandExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(1) + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + None + } + + fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> { + vec![self.child.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn ExecutionPlan>>, + ) -> datafusion_common::Result<Arc<dyn ExecutionPlan>> { + let new_expand = CometExpandExec::new( + self.projections.clone(), + children[0].clone(), + self.schema.clone(), + ); + Ok(Arc::new(new_expand)) + } + + fn execute( + &self, + partition: usize, + context: Arc<TaskContext>, + ) -> datafusion_common::Result<SendableRecordBatchStream> { + let child_stream = self.child.execute(partition, context)?; + let expand_stream = + ExpandStream::new(self.projections.clone(), child_stream, self.schema.clone()); + Ok(Box::pin(expand_stream)) + } +} + +pub struct ExpandStream { + projections: Vec<Vec<Arc<dyn PhysicalExpr>>>, + child_stream: SendableRecordBatchStream, + schema: SchemaRef, + current_index: i32, + max_index: i32, + current_batch: Option<RecordBatch>, +} + +impl ExpandStream { + /// Create a new ExpandStream + pub fn new( + projections: Vec<Vec<Arc<dyn PhysicalExpr>>>, + child_stream: SendableRecordBatchStream, + schema: SchemaRef, + ) -> Self { + let max_index = projections.len() as i32; + Self { + projections, + child_stream, + schema, + current_index: -1, + max_index, + current_batch: None, + } + } + + fn expand( + &self, + batch: &RecordBatch, + projection: &[Arc<dyn PhysicalExpr>], + ) -> Result<RecordBatch, DataFusionError> { + let mut columns = vec![]; + + projection.iter().try_for_each(|expr| { + let column = expr.evaluate(batch)?; + columns.push(column.into_array(batch.num_rows())?); + + Ok::<(), DataFusionError>(()) + })?; + + RecordBatch::try_new(self.schema.clone(), columns).map_err(|e| e.into()) + } +} + +impl Stream for ExpandStream { + type Item = datafusion_common::Result<RecordBatch>; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { + if self.current_index == -1 { + let next = self.child_stream.poll_next_unpin(cx); + match next { + Poll::Ready(Some(Ok(batch))) => { + self.current_batch = Some(batch); + self.current_index = 0; + } + other => return other, + } + } + assert!(self.current_batch.is_some()); + + let projection = &self.projections[self.current_index as usize]; + let batch = self.expand(self.current_batch.as_ref().unwrap(), projection); + + self.current_index += 1; + + if self.current_index == self.max_index { + self.current_index = -1; + self.current_batch = None; + } + Poll::Ready(Some(batch)) + } +} + +impl RecordBatchStream for ExpandStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} diff --git a/core/src/execution/datafusion/operators/mod.rs b/core/src/execution/datafusion/operators/mod.rs new file mode 100644 index 0000000000..3d28a266a6 --- /dev/null +++ b/core/src/execution/datafusion/operators/mod.rs @@ -0,0 +1,18 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 expand; diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs new file mode 100644 index 0000000000..1a6aa5ac37 --- /dev/null +++ b/core/src/execution/datafusion/planner.rs @@ -0,0 +1,1192 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Converts Spark physical plan to DataFusion physical plan + +use std::{str::FromStr, sync::Arc}; + +use arrow_schema::{DataType, Field, Schema, TimeUnit}; +use datafusion::{ + arrow::{compute::SortOptions, datatypes::SchemaRef}, + common::DataFusionError, + logical_expr::{BuiltinScalarFunction, Operator as DataFusionOperator}, + physical_expr::{ + expressions::{BinaryExpr, Column, IsNotNullExpr, Literal as DataFusionLiteral}, + functions::create_physical_expr, + PhysicalExpr, PhysicalSortExpr, + }, + physical_plan::{ + aggregates::{AggregateMode as DFAggregateMode, PhysicalGroupBy}, + filter::FilterExec, + limit::LocalLimitExec, + projection::ProjectionExec, + sorts::sort::SortExec, + ExecutionPlan, Partitioning, + }, +}; +use datafusion_common::ScalarValue; +use datafusion_physical_expr::{ + execution_props::ExecutionProps, + expressions::{ + CaseExpr, CastExpr, Count, InListExpr, IsNullExpr, Max, Min, NegativeExpr, NotExpr, Sum, + }, + AggregateExpr, ScalarFunctionExpr, +}; +use itertools::Itertools; +use num::{BigInt, ToPrimitive}; + +use crate::{ + errors::ExpressionError, + execution::{ + datafusion::{ + expressions::{ + avg::Avg, + avg_decimal::AvgDecimal, + bitwise_not::BitwiseNotExpr, + cast::Cast, + checkoverflow::CheckOverflow, + if_expr::IfExpr, + scalar_funcs::create_comet_physical_fun, + strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExec, SubstringExec}, + subquery::Subquery, + sum_decimal::SumDecimal, + temporal::{DateTruncExec, HourExec, MinuteExec, SecondExec, TimestampTruncExec}, + NormalizeNaNAndZero, + }, + operators::expand::CometExpandExec, + }, + operators::{CopyExec, ExecutionError, InputBatch, ScanExec}, + serde::to_arrow_datatype, + spark_expression, + spark_expression::{ + agg_expr::ExprStruct as AggExprStruct, expr::ExprStruct, literal::Value, AggExpr, Expr, + ScalarFunc, + }, + spark_operator::{operator::OpStruct, Operator}, + spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, + }, +}; + +// For clippy error on type_complexity. +type ExecResult<T> = Result<T, ExecutionError>; +type PhyAggResult = Result<Vec<Arc<dyn AggregateExpr>>, ExecutionError>; +type PhyExprResult = Result<Vec<(Arc<dyn PhysicalExpr>, String)>, ExecutionError>; +type PartitionPhyExprResult = Result<Vec<Arc<dyn PhysicalExpr>>, ExecutionError>; + +/// The query planner for converting Spark query plans to DataFusion query plans. +pub struct PhysicalPlanner { + // The execution context id of this planner. + exec_context_id: i64, + execution_props: ExecutionProps, +} + +impl Default for PhysicalPlanner { + fn default() -> Self { + Self::new() + } +} + +impl PhysicalPlanner { + pub fn new() -> Self { + let execution_props = ExecutionProps::new(); + Self { + exec_context_id: -1, + execution_props, + } + } + + pub fn with_exec_id(self, exec_context_id: i64) -> Self { + Self { + exec_context_id, + execution_props: self.execution_props, + } + } + + /// Create a DataFusion physical expression from Spark physical expression + fn create_expr( + &self, + spark_expr: &Expr, + input_schema: SchemaRef, + ) -> Result<Arc<dyn PhysicalExpr>, ExecutionError> { + match spark_expr.expr_struct.as_ref().unwrap() { + ExprStruct::Add(expr) => self.create_binary_expr( + expr.left.as_ref().unwrap(), + expr.right.as_ref().unwrap(), + expr.return_type.as_ref(), + DataFusionOperator::Plus, + input_schema, + ), + ExprStruct::Subtract(expr) => self.create_binary_expr( + expr.left.as_ref().unwrap(), + expr.right.as_ref().unwrap(), + expr.return_type.as_ref(), + DataFusionOperator::Minus, + input_schema, + ), + ExprStruct::Multiply(expr) => self.create_binary_expr( + expr.left.as_ref().unwrap(), + expr.right.as_ref().unwrap(), + expr.return_type.as_ref(), + DataFusionOperator::Multiply, + input_schema, + ), + ExprStruct::Divide(expr) => self.create_binary_expr( + expr.left.as_ref().unwrap(), + expr.right.as_ref().unwrap(), + expr.return_type.as_ref(), + DataFusionOperator::Divide, + input_schema, + ), + ExprStruct::Remainder(expr) => self.create_binary_expr( + expr.left.as_ref().unwrap(), + expr.right.as_ref().unwrap(), + expr.return_type.as_ref(), + DataFusionOperator::Modulo, + input_schema, + ), + ExprStruct::Eq(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::Eq; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::Neq(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::NotEq; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::Gt(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::Gt; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::GtEq(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::GtEq; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::Lt(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::Lt; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::LtEq(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::LtEq; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::Bound(bound) => { + let idx = bound.index as usize; + let column_name = format!("col_{}", idx); + Ok(Arc::new(Column::new(&column_name, idx))) + } + ExprStruct::IsNotNull(is_notnull) => { + let child = self.create_expr(is_notnull.child.as_ref().unwrap(), input_schema)?; + Ok(Arc::new(IsNotNullExpr::new(child))) + } + ExprStruct::IsNull(is_null) => { + let child = self.create_expr(is_null.child.as_ref().unwrap(), input_schema)?; + Ok(Arc::new(IsNullExpr::new(child))) + } + ExprStruct::And(and) => { + let left = self.create_expr(and.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(and.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::And; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::Or(or) => { + let left = self.create_expr(or.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(or.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::Or; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::Literal(literal) => { + let data_type = to_arrow_datatype(literal.datatype.as_ref().unwrap()); + let scalar_value = if literal.is_null { + match data_type { + DataType::Boolean => ScalarValue::Boolean(None), + DataType::Int8 => ScalarValue::Int8(None), + DataType::Int16 => ScalarValue::Int16(None), + DataType::Int32 => ScalarValue::Int32(None), + DataType::Int64 => ScalarValue::Int64(None), + DataType::Float32 => ScalarValue::Float32(None), + DataType::Float64 => ScalarValue::Float64(None), + DataType::Utf8 => ScalarValue::Utf8(None), + DataType::Date32 => ScalarValue::Date32(None), + DataType::Timestamp(TimeUnit::Microsecond, timezone) => { + ScalarValue::TimestampMicrosecond(None, timezone) + } + DataType::Binary => ScalarValue::Binary(None), + DataType::Decimal128(p, s) => ScalarValue::Decimal128(None, p, s), + DataType::Null => ScalarValue::Null, + dt => { + return Err(ExecutionError::GeneralError(format!( + "{:?} is not supported in Comet", + dt + ))) + } + } + } else { + match literal.value.as_ref().unwrap() { + Value::BoolVal(value) => ScalarValue::Boolean(Some(*value)), + Value::ByteVal(value) => ScalarValue::Int8(Some(*value as i8)), + Value::ShortVal(value) => ScalarValue::Int16(Some(*value as i16)), + Value::IntVal(value) => match data_type { + DataType::Int32 => ScalarValue::Int32(Some(*value)), + DataType::Date32 => ScalarValue::Date32(Some(*value)), + dt => { + return Err(ExecutionError::GeneralError(format!( + "Expected either 'Int32' or 'Date32' for IntVal, but found {:?}", + dt + ))) + } + }, + Value::LongVal(value) => match data_type { + DataType::Int64 => ScalarValue::Int64(Some(*value)), + DataType::Timestamp(TimeUnit::Microsecond, None) => { + ScalarValue::TimestampMicrosecond(Some(*value), None) + } + DataType::Timestamp(TimeUnit::Microsecond, Some(tz)) => { + ScalarValue::TimestampMicrosecond(Some(*value), Some(tz)) + } + dt => { + return Err(ExecutionError::GeneralError(format!( + "Expected either 'Int64' or 'Timestamp' for LongVal, but found {:?}", + dt + ))) + } + }, + Value::FloatVal(value) => ScalarValue::Float32(Some(*value)), + Value::DoubleVal(value) => ScalarValue::Float64(Some(*value)), + Value::StringVal(value) => ScalarValue::Utf8(Some(value.clone())), + Value::BytesVal(value) => ScalarValue::Binary(Some(value.clone())), + Value::DecimalVal(value) => { + let big_integer = BigInt::from_signed_bytes_be(value); + let integer = big_integer.to_i128().ok_or_else(|| { + ExecutionError::GeneralError(format!( + "Cannot parse {:?} as i128 for Decimal literal", + big_integer + )) + })?; + + match data_type { + DataType::Decimal128(p, s) => { + ScalarValue::Decimal128(Some(integer), p, s) + } + dt => { + return Err(ExecutionError::GeneralError(format!( + "Decimal literal's data type should be Decimal128 but got {:?}", + dt + ))) + } + } + } + } + }; + Ok(Arc::new(DataFusionLiteral::new(scalar_value))) + } + ExprStruct::Cast(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + let timezone = expr.timezone.clone(); + Ok(Arc::new(Cast::new(child, datatype, timezone))) + } + ExprStruct::Hour(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + let timezone = expr.timezone.clone(); + + Ok(Arc::new(HourExec::new(child, timezone))) + } + ExprStruct::Minute(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + let timezone = expr.timezone.clone(); + + Ok(Arc::new(MinuteExec::new(child, timezone))) + } + ExprStruct::Second(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + let timezone = expr.timezone.clone(); + + Ok(Arc::new(SecondExec::new(child, timezone))) + } + ExprStruct::TruncDate(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema.clone())?; + let format = self.create_expr(expr.format.as_ref().unwrap(), input_schema)?; + + Ok(Arc::new(DateTruncExec::new(child, format))) + } + ExprStruct::TruncTimestamp(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema.clone())?; + let format = self.create_expr(expr.format.as_ref().unwrap(), input_schema)?; + let timezone = expr.timezone.clone(); + + Ok(Arc::new(TimestampTruncExec::new(child, format, timezone))) + } + ExprStruct::Substring(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + // Spark Substring's start is 1-based when start > 0 + let start = expr.start - i32::from(expr.start > 0); + let len = expr.len; + + Ok(Arc::new(SubstringExec::new( + child, + start as i64, + len as u64, + ))) + } + ExprStruct::StringSpace(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + + Ok(Arc::new(StringSpaceExec::new(child))) + } + ExprStruct::Contains(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + + Ok(Arc::new(Contains::new(left, right))) + } + ExprStruct::StartsWith(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + + Ok(Arc::new(StartsWith::new(left, right))) + } + ExprStruct::EndsWith(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + + Ok(Arc::new(EndsWith::new(left, right))) + } + ExprStruct::Like(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + + Ok(Arc::new(Like::new(left, right))) + } + ExprStruct::CheckOverflow(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + let data_type = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + let fail_on_error = expr.fail_on_error; + + Ok(Arc::new(CheckOverflow::new( + child, + data_type, + fail_on_error, + ))) + } + ExprStruct::ScalarFunc(expr) => self.create_scalar_function_expr(expr, input_schema), + ExprStruct::EqNullSafe(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::IsNotDistinctFrom; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::NeqNullSafe(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::IsDistinctFrom; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::BitwiseAnd(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::BitwiseAnd; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::BitwiseNot(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + Ok(Arc::new(BitwiseNotExpr::new(child))) + } + ExprStruct::BitwiseOr(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::BitwiseOr; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::BitwiseXor(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::BitwiseXor; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::BitwiseShiftRight(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::BitwiseShiftRight; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::BitwiseShiftLeft(expr) => { + let left = self.create_expr(expr.left.as_ref().unwrap(), input_schema.clone())?; + let right = self.create_expr(expr.right.as_ref().unwrap(), input_schema)?; + let op = DataFusionOperator::BitwiseShiftLeft; + Ok(Arc::new(BinaryExpr::new(left, op, right))) + } + ExprStruct::Abs(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema.clone())?; + let args = vec![child]; + let expr = create_physical_expr( + &BuiltinScalarFunction::Abs, + &args, + &input_schema, + &self.execution_props, + )?; + Ok(expr) + } + ExprStruct::CaseWhen(case_when) => { + let when_then_pairs = case_when + .when + .iter() + .map(|x| self.create_expr(x, input_schema.clone()).unwrap()) + .zip( + case_when + .then + .iter() + .map(|then| self.create_expr(then, input_schema.clone()).unwrap()), + ) + .collect::<Vec<_>>(); + + let else_phy_expr = match &case_when.else_expr { + None => None, + Some(_) => { + Some(self.create_expr(case_when.else_expr.as_ref().unwrap(), input_schema)?) + } + }; + Ok(Arc::new(CaseExpr::try_new( + None, + when_then_pairs, + else_phy_expr, + )?)) + } + ExprStruct::In(expr) => { + let value = + self.create_expr(expr.in_value.as_ref().unwrap(), input_schema.clone())?; + let list = expr + .lists + .iter() + .map(|x| self.create_expr(x, input_schema.clone()).unwrap()) + .collect::<Vec<_>>(); + Ok(Arc::new(InListExpr::new(value, list, expr.negated, None))) + } + ExprStruct::If(expr) => { + let if_expr = + self.create_expr(expr.if_expr.as_ref().unwrap(), input_schema.clone())?; + let true_expr = + self.create_expr(expr.true_expr.as_ref().unwrap(), input_schema.clone())?; + let false_expr = + self.create_expr(expr.false_expr.as_ref().unwrap(), input_schema)?; + Ok(Arc::new(IfExpr::new(if_expr, true_expr, false_expr))) + } + ExprStruct::Not(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + Ok(Arc::new(NotExpr::new(child))) + } + ExprStruct::Negative(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + Ok(Arc::new(NegativeExpr::new(child))) + } + ExprStruct::NormalizeNanAndZero(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + let data_type = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + Ok(Arc::new(NormalizeNaNAndZero::new(data_type, child))) + } + ExprStruct::Subquery(expr) => { + let id = expr.id; + let data_type = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + Ok(Arc::new(Subquery::new(self.exec_context_id, id, data_type))) + } + expr => Err(ExecutionError::GeneralError(format!( + "Not implemented: {:?}", + expr + ))), + } + } + + /// Create a DataFusion physical sort expression from Spark physical expression + fn create_sort_expr<'a>( + &'a self, + spark_expr: &'a Expr, + input_schema: SchemaRef, + ) -> Result<PhysicalSortExpr, ExecutionError> { + match spark_expr.expr_struct.as_ref().unwrap() { + ExprStruct::SortOrder(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; + let descending = expr.direction == 1; + let nulls_first = expr.null_ordering == 0; + + let options = SortOptions { + descending, + nulls_first, + }; + + Ok(PhysicalSortExpr { + expr: child, + options, + }) + } + expr => Err(ExecutionError::GeneralError(format!( + "{:?} isn't a SortOrder", + expr + ))), + } + } + + fn create_binary_expr( + &self, + left: &Expr, + right: &Expr, + return_type: Option<&spark_expression::DataType>, + op: DataFusionOperator, + input_schema: SchemaRef, + ) -> Result<Arc<dyn PhysicalExpr>, ExecutionError> { + let left = self.create_expr(left, input_schema.clone())?; + let right = self.create_expr(right, input_schema.clone())?; + match ( + op, + left.data_type(&input_schema), + right.data_type(&input_schema), + ) { + ( + DataFusionOperator::Plus + | DataFusionOperator::Minus + | DataFusionOperator::Multiply + | DataFusionOperator::Modulo, + Ok(DataType::Decimal128(p1, s1)), + Ok(DataType::Decimal128(p2, s2)), + ) => { + let data_type = return_type.map(to_arrow_datatype).unwrap(); + // For some Decimal128 operations, we need wider internal digits. + // Cast left and right to Decimal256 and cast the result back to Decimal128 + let left = Arc::new(Cast::new_without_timezone( + left, + DataType::Decimal256(p1, s1), + )); + let right = Arc::new(Cast::new_without_timezone( + right, + DataType::Decimal256(p2, s2), + )); + let child = Arc::new(BinaryExpr::new(left, op, right)); + Ok(Arc::new(Cast::new_without_timezone(child, data_type))) + } + ( + DataFusionOperator::Divide, + Ok(DataType::Decimal128(_p1, _s1)), + Ok(DataType::Decimal128(_p2, _s2)), + ) => { + let data_type = return_type.map(to_arrow_datatype).unwrap(); + let fun_expr = create_comet_physical_fun( + "decimal_div", + &self.execution_props, + data_type.clone(), + )?; + Ok(Arc::new(ScalarFunctionExpr::new( + "decimal_div", + fun_expr, + vec![left, right], + data_type, + None, + ))) + } + _ => Ok(Arc::new(BinaryExpr::new(left, op, right))), + } + } + + /// Create a DataFusion physical plan from Spark physical plan. + /// + /// Note that we need `input_batches` parameter because we need to know the exact schema (not + /// only data type but also dictionary-encoding) at `ScanExec`s. It is because some DataFusion + /// operators, e.g., `ProjectionExec`, gets child operator schema during initialization and + /// uses it later for `RecordBatch`. We may be able to get rid of it once `RecordBatch` + /// relaxes schema check. + /// + /// Note that we return created `Scan`s which will be kept at JNI API. JNI calls will use it to + /// feed in new input batch from Spark JVM side. + pub fn create_plan<'a>( + &'a self, + spark_plan: &'a Operator, + input_batches: &mut Vec<InputBatch>, + ) -> Result<(Vec<ScanExec>, Arc<dyn ExecutionPlan>), ExecutionError> { + let children = &spark_plan.children; + match spark_plan.op_struct.as_ref().unwrap() { + OpStruct::Projection(project) => { + assert!(children.len() == 1); + let (scans, child) = self.create_plan(&children[0], input_batches)?; + let exprs: PhyExprResult = project + .project_list + .iter() + .enumerate() + .map(|(idx, expr)| { + self.create_expr(expr, child.schema()) + .map(|r| (r, format!("col_{}", idx))) + }) + .collect(); + Ok((scans, Arc::new(ProjectionExec::try_new(exprs?, child)?))) + } + OpStruct::Filter(filter) => { + assert!(children.len() == 1); + let (scans, child) = self.create_plan(&children[0], input_batches)?; + let predicate = + self.create_expr(filter.predicate.as_ref().unwrap(), child.schema())?; + + Ok((scans, Arc::new(FilterExec::try_new(predicate, child)?))) + } + OpStruct::HashAgg(agg) => { + assert!(children.len() == 1); + let (scans, child) = self.create_plan(&children[0], input_batches)?; + + let group_exprs: PhyExprResult = agg + .grouping_exprs + .iter() + .enumerate() + .map(|(idx, expr)| { + self.create_expr(expr, child.schema()) + .map(|r| (r, format!("col_{}", idx))) + }) + .collect(); + let group_by = PhysicalGroupBy::new_single(group_exprs?); + let schema = child.schema(); + + let mode = if agg.mode == 0 { + DFAggregateMode::Partial + } else { + DFAggregateMode::Final + }; + + let agg_exprs: PhyAggResult = agg + .agg_exprs + .iter() + .map(|expr| self.create_agg_expr(expr, schema.clone())) + .collect(); + + let num_agg = agg.agg_exprs.len(); + let aggregate = Arc::new( + datafusion::physical_plan::aggregates::AggregateExec::try_new( + mode, + group_by, + agg_exprs?, + vec![None; num_agg], // no filter expressions + vec![None; num_agg], // no order by expressions + child.clone(), + schema.clone(), + )?, + ); + let result_exprs: PhyExprResult = agg + .result_exprs + .iter() + .enumerate() + .map(|(idx, expr)| { + self.create_expr(expr, child.schema()) + .map(|r| (r, format!("col_{}", idx))) + }) + .collect(); + + let exec: Arc<dyn ExecutionPlan> = if agg.result_exprs.is_empty() { + aggregate + } else { + // For final aggregation, DF's hash aggregate exec doesn't support Spark's + // aggregate result expressions like `COUNT(col) + 1`, but instead relying + // on additional `ProjectionExec` to handle the case. Therefore, here we'll + // add a projection node on top of the aggregate node. + // + // Note that `result_exprs` should only be set for final aggregation on the + // Spark side. + Arc::new(ProjectionExec::try_new(result_exprs?, aggregate)?) + }; + + Ok((scans, exec)) + } + OpStruct::Limit(limit) => { + assert!(children.len() == 1); + let num = limit.limit; + let (scans, child) = self.create_plan(&children[0], input_batches)?; + + Ok((scans, Arc::new(LocalLimitExec::new(child, num as usize)))) + } + OpStruct::Sort(sort) => { + assert!(children.len() == 1); + let (scans, child) = self.create_plan(&children[0], input_batches)?; + + let exprs: Result<Vec<PhysicalSortExpr>, ExecutionError> = sort + .sort_orders + .iter() + .map(|expr| self.create_sort_expr(expr, child.schema())) + .collect(); + + let fetch = sort.fetch.map(|num| num as usize); + + let copy_exec = Arc::new(CopyExec::new(child)); + + Ok(( + scans, + Arc::new(SortExec::new(exprs?, copy_exec).with_fetch(fetch)), + )) + } + OpStruct::Scan(scan) => { + let fields = scan.fields.iter().map(to_arrow_datatype).collect_vec(); + if input_batches.is_empty() { + return Err(ExecutionError::GeneralError( + "No input batch for scan".to_string(), + )); + } + // Consumes the first input batch source for the scan + let input_batch = input_batches.remove(0); + + // The `ScanExec` operator will take actual arrays from Spark during execution + let scan = ScanExec::new(input_batch, fields); + Ok((vec![scan.clone()], Arc::new(scan))) + } + OpStruct::Expand(expand) => { + assert!(children.len() == 1); + let (scans, child) = self.create_plan(&children[0], input_batches)?; + + let mut projections = vec![]; + let mut projection = vec![]; + + expand.project_list.iter().try_for_each(|expr| { + let expr = self.create_expr(expr, child.schema())?; + projection.push(expr); + + if projection.len() == expand.num_expr_per_project as usize { + projections.push(projection.clone()); + projection = vec![]; + } + + Ok::<(), ExecutionError>(()) + })?; + + assert!( + !projections.is_empty(), + "Expand should have at least one projection" + ); + + let datatypes = projections[0] + .iter() + .map(|expr| expr.data_type(&child.schema())) + .collect::<Result<Vec<DataType>, _>>()?; + let fields: Vec<Field> = datatypes + .iter() + .enumerate() + .map(|(idx, dt)| Field::new(format!("col_{}", idx), dt.clone(), true)) + .collect(); + let schema = Arc::new(Schema::new(fields)); + + Ok(( + scans, + Arc::new(CometExpandExec::new(projections, child, schema)), + )) + } + } + } + + /// Create a DataFusion physical aggregate expression from Spark physical aggregate expression + fn create_agg_expr( + &self, + spark_expr: &AggExpr, + schema: SchemaRef, + ) -> Result<Arc<dyn AggregateExpr>, ExecutionError> { + match spark_expr.expr_struct.as_ref().unwrap() { + AggExprStruct::Count(expr) => { + let child = self.create_expr(&expr.children[0], schema)?; + Ok(Arc::new(Count::new(child, "count", DataType::Int64))) + } + AggExprStruct::Min(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; + let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + Ok(Arc::new(Min::new(child, "min", datatype))) + } + AggExprStruct::Max(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?; + let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + Ok(Arc::new(Max::new(child, "max", datatype))) + } + AggExprStruct::Sum(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + + match datatype { + DataType::Decimal128(_, _) => { + Ok(Arc::new(SumDecimal::new("sum", child, datatype))) + } + _ => { + // cast to the result data type of SUM if necessary, we should not expect + // a cast failure since it should have already been checked at Spark side + let child = Arc::new(CastExpr::new(child, datatype.clone(), None)); + Ok(Arc::new(Sum::new(child, "sum", datatype))) + } + } + } + AggExprStruct::Avg(expr) => { + let child = self.create_expr(expr.child.as_ref().unwrap(), schema.clone())?; + let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); + let input_datatype = to_arrow_datatype(expr.sum_datatype.as_ref().unwrap()); + match datatype { + DataType::Decimal128(_, _) => Ok(Arc::new(AvgDecimal::new( + child, + "avg", + datatype, + input_datatype, + ))), + _ => { + // cast to the result data type of AVG if the result data type is different + // from the input type, e.g. AVG(Int32). We should not expect a cast + // failure since it should have already been checked at Spark side. + let child = Arc::new(CastExpr::new(child, datatype.clone(), None)); + Ok(Arc::new(Avg::new(child, "avg", datatype))) + } + } + } + } + } + + /// Create a DataFusion physical partitioning from Spark physical partitioning + fn create_partitioning( + &self, + spark_partitioning: &SparkPartitioning, + input_schema: SchemaRef, + ) -> Result<Partitioning, ExecutionError> { + match spark_partitioning.partitioning_struct.as_ref().unwrap() { + PartitioningStruct::HashPartition(hash_partition) => { + let exprs: PartitionPhyExprResult = hash_partition + .hash_expression + .iter() + .map(|x| self.create_expr(x, input_schema.clone())) + .collect(); + Ok(Partitioning::Hash( + exprs?, + hash_partition.num_partitions as usize, + )) + } + PartitioningStruct::SinglePartition(_) => Ok(Partitioning::UnknownPartitioning(1)), + } + } + + fn create_scalar_function_expr( + &self, + expr: &ScalarFunc, + input_schema: SchemaRef, + ) -> Result<Arc<dyn PhysicalExpr>, ExecutionError> { + let args = expr + .args + .iter() + .map(|x| self.create_expr(x, input_schema.clone()).unwrap()) + .collect::<Vec<_>>(); + + let fun_name = &expr.func; + let input_expr_types = args + .iter() + .map(|x| x.data_type(input_schema.as_ref()).unwrap()) + .collect::<Vec<_>>(); + let data_type = match expr.return_type.as_ref().map(to_arrow_datatype) { + Some(t) => t, + None => { + // If no data type is provided from Spark, we'll use DF's return type from the + // scalar function + // Note this assumes the `fun_name` is a defined function in DF. Otherwise, it'll + // throw error. + let fun = &BuiltinScalarFunction::from_str(fun_name)?; + fun.return_type(&input_expr_types)? + } + }; + let fun_expr = + create_comet_physical_fun(fun_name, &self.execution_props, data_type.clone())?; + + let scalar_expr: Arc<dyn PhysicalExpr> = Arc::new(ScalarFunctionExpr::new( + fun_name, + fun_expr, + args.to_vec(), + data_type, + None, + )); + + Ok(scalar_expr) + } +} + +impl From<DataFusionError> for ExecutionError { + fn from(value: DataFusionError) -> Self { + ExecutionError::DataFusionError(value.to_string()) + } +} + +impl From<ExecutionError> for DataFusionError { + fn from(value: ExecutionError) -> Self { + DataFusionError::Execution(value.to_string()) + } +} + +impl From<ExpressionError> for DataFusionError { + fn from(value: ExpressionError) -> Self { + DataFusionError::Execution(value.to_string()) + } +} + +#[cfg(test)] +mod tests { + use std::{sync::Arc, task::Poll}; + + use futures::{poll, StreamExt}; + + use arrow_array::{DictionaryArray, Int32Array, StringArray}; + use arrow_schema::DataType; + use datafusion::{physical_plan::common::collect, prelude::SessionContext}; + use tokio::sync::mpsc; + + use crate::execution::{ + datafusion::planner::PhysicalPlanner, + operators::InputBatch, + spark_expression::{self, literal}, + spark_operator, + }; + + use spark_expression::expr::ExprStruct::*; + use spark_operator::{operator::OpStruct, Operator}; + + #[test] + fn test_unpack_dictionary_primitive() { + let op_scan = Operator { + children: vec![], + op_struct: Some(OpStruct::Scan(spark_operator::Scan { + fields: vec![spark_expression::DataType { + type_id: 3, // Int32 + type_info: None, + }], + })), + }; + + let op = create_filter(op_scan, 3); + let planner = PhysicalPlanner::new(); + let row_count = 100; + + // Create a dictionary array with 100 values, and use it as input to the execution. + let keys = Int32Array::new((0..(row_count as i32)).map(|n| n % 4).collect(), None); + let values = Int32Array::from(vec![0, 1, 2, 3]); + let input_array = DictionaryArray::new(keys, Arc::new(values)); + let input_batch = InputBatch::Batch(vec![Arc::new(input_array)], row_count); + let mut input_batches = vec![input_batch]; + + let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut input_batches).unwrap(); + + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + let mut stream = datafusion_plan.execute(0, task_ctx).unwrap(); + + let runtime = tokio::runtime::Runtime::new().unwrap(); + let (tx, mut rx) = mpsc::channel(1); + + // Separate thread to send the EOF signal once we've processed the only input batch + runtime.spawn(async move { + // Create a dictionary array with 100 values, and use it as input to the execution. + let keys = Int32Array::new((0..(row_count as i32)).map(|n| n % 4).collect(), None); + let values = Int32Array::from(vec![0, 1, 2, 3]); + let input_array = DictionaryArray::new(keys, Arc::new(values)); + let input_batch1 = InputBatch::Batch(vec![Arc::new(input_array)], row_count); + let input_batch2 = InputBatch::EOF; + + let batches = vec![input_batch1, input_batch2]; + + for batch in batches.into_iter() { + tx.send(batch).await.unwrap(); + } + }); + + runtime.block_on(async move { + loop { + let batch = rx.recv().await.unwrap(); + scans[0].set_input_batch(batch); + match poll!(stream.next()) { + Poll::Ready(Some(batch)) => { + assert!(batch.is_ok(), "got error {}", batch.unwrap_err()); + let batch = batch.unwrap(); + assert_eq!(batch.num_rows(), row_count / 4); + // dictionary should be unpacked + assert!(matches!(batch.column(0).data_type(), DataType::Int32)); + } + Poll::Ready(None) => { + break; + } + _ => {} + } + } + }); + } + + const STRING_TYPE_ID: i32 = 7; + + #[test] + fn test_unpack_dictionary_string() { + let op_scan = Operator { + children: vec![], + op_struct: Some(OpStruct::Scan(spark_operator::Scan { + fields: vec![spark_expression::DataType { + type_id: STRING_TYPE_ID, // String + type_info: None, + }], + })), + }; + + let lit = spark_expression::Literal { + value: Some(literal::Value::StringVal("foo".to_string())), + datatype: Some(spark_expression::DataType { + type_id: STRING_TYPE_ID, + type_info: None, + }), + is_null: false, + }; + + let op = create_filter_literal(op_scan, STRING_TYPE_ID, lit); + let planner = PhysicalPlanner::new(); + + let row_count = 100; + + let keys = Int32Array::new((0..(row_count as i32)).map(|n| n % 4).collect(), None); + let values = StringArray::from(vec!["foo", "bar", "hello", "comet"]); + let input_array = DictionaryArray::new(keys, Arc::new(values)); + let input_batch = InputBatch::Batch(vec![Arc::new(input_array)], row_count); + let mut input_batches = vec![input_batch]; + + let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut input_batches).unwrap(); + + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + let mut stream = datafusion_plan.execute(0, task_ctx).unwrap(); + + let runtime = tokio::runtime::Runtime::new().unwrap(); + let (tx, mut rx) = mpsc::channel(1); + + // Separate thread to send the EOF signal once we've processed the only input batch + runtime.spawn(async move { + // Create a dictionary array with 100 values, and use it as input to the execution. + let keys = Int32Array::new((0..(row_count as i32)).map(|n| n % 4).collect(), None); + let values = StringArray::from(vec!["foo", "bar", "hello", "comet"]); + let input_array = DictionaryArray::new(keys, Arc::new(values)); + let input_batch1 = InputBatch::Batch(vec![Arc::new(input_array)], row_count); + + let input_batch2 = InputBatch::EOF; + + let batches = vec![input_batch1, input_batch2]; + + for batch in batches.into_iter() { + tx.send(batch).await.unwrap(); + } + }); + + runtime.block_on(async move { + loop { + let batch = rx.recv().await.unwrap(); + scans[0].set_input_batch(batch); + match poll!(stream.next()) { + Poll::Ready(Some(batch)) => { + assert!(batch.is_ok(), "got error {}", batch.unwrap_err()); + let batch = batch.unwrap(); + assert_eq!(batch.num_rows(), row_count / 4); + // string/binary should still be packed with dictionary + assert!(matches!( + batch.column(0).data_type(), + DataType::Dictionary(_, _) + )); + } + Poll::Ready(None) => { + break; + } + _ => {} + } + } + }); + } + + #[tokio::test()] + #[allow(clippy::field_reassign_with_default)] + async fn to_datafusion_filter() { + let op_scan = spark_operator::Operator { + children: vec![], + op_struct: Some(spark_operator::operator::OpStruct::Scan( + spark_operator::Scan { + fields: vec![spark_expression::DataType { + type_id: 3, + type_info: None, + }], + }, + )), + }; + + let op = create_filter(op_scan, 0); + let planner = PhysicalPlanner::new(); + + let mut input_batches = vec![InputBatch::EOF]; + let (mut scans, datafusion_plan) = planner.create_plan(&op, &mut input_batches).unwrap(); + + let scan = &mut scans[0]; + scan.set_input_batch(InputBatch::EOF); + + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + + let stream = datafusion_plan.execute(0, task_ctx.clone()).unwrap(); + let output = collect(stream).await.unwrap(); + assert!(output.is_empty()); + } + + // Creates a filter operator which takes an `Int32Array` and selects rows that are equal to + // `value`. + fn create_filter(child_op: spark_operator::Operator, value: i32) -> spark_operator::Operator { + let lit = spark_expression::Literal { + value: Some(literal::Value::IntVal(value)), + datatype: Some(spark_expression::DataType { + type_id: 3, + type_info: None, + }), + is_null: false, + }; + + create_filter_literal(child_op, 3, lit) + } + + fn create_filter_literal( + child_op: spark_operator::Operator, + type_id: i32, + lit: spark_expression::Literal, + ) -> spark_operator::Operator { + let mut expr = spark_expression::Expr::default(); + + let mut left = spark_expression::Expr::default(); + left.expr_struct = Some(Bound(spark_expression::BoundReference { + index: 0, + datatype: Some(spark_expression::DataType { + type_id, + type_info: None, + }), + })); + let mut right = spark_expression::Expr::default(); + right.expr_struct = Some(Literal(lit)); + + expr.expr_struct = Some(Eq(Box::new(spark_expression::Equal { + left: Some(Box::new(left)), + right: Some(Box::new(right)), + }))); + + let mut op = spark_operator::Operator::default(); + op.children = vec![child_op]; + op.op_struct = Some(OpStruct::Filter(spark_operator::Filter { + predicate: Some(expr), + })); + op + } +} diff --git a/core/src/execution/datafusion/spark_hash.rs b/core/src/execution/datafusion/spark_hash.rs new file mode 100644 index 0000000000..0413e4559f --- /dev/null +++ b/core/src/execution/datafusion/spark_hash.rs @@ -0,0 +1,473 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! This includes utilities for hashing and murmur3 hashing. + +use arrow::datatypes::{ArrowNativeTypeOp, UInt16Type, UInt32Type, UInt64Type, UInt8Type}; +use std::sync::Arc; + +use datafusion::{ + arrow::{ + array::*, + datatypes::{ + ArrowDictionaryKeyType, ArrowNativeType, DataType, Int16Type, Int32Type, Int64Type, + Int8Type, TimeUnit, + }, + }, + error::{DataFusionError, Result}, +}; + +#[inline] +fn spark_compatible_murmur3_hash<T: AsRef<[u8]>>(data: T, seed: u32) -> u32 { + #[inline] + fn mix_k1(mut k1: i32) -> i32 { + k1 = k1.mul_wrapping(0xcc9e2d51u32 as i32); + k1 = k1.rotate_left(15); + k1 = k1.mul_wrapping(0x1b873593u32 as i32); + k1 + } + + #[inline] + fn mix_h1(mut h1: i32, k1: i32) -> i32 { + h1 ^= k1; + h1 = h1.rotate_left(13); + h1 = h1.mul_wrapping(5).add_wrapping(0xe6546b64u32 as i32); + h1 + } + + #[inline] + fn fmix(mut h1: i32, len: i32) -> i32 { + h1 ^= len; + h1 ^= (h1 as u32 >> 16) as i32; + h1 = h1.mul_wrapping(0x85ebca6bu32 as i32); + h1 ^= (h1 as u32 >> 13) as i32; + h1 = h1.mul_wrapping(0xc2b2ae35u32 as i32); + h1 ^= (h1 as u32 >> 16) as i32; + h1 + } + + #[inline] + unsafe fn hash_bytes_by_int(data: &[u8], seed: u32) -> i32 { + // safety: data length must be aligned to 4 bytes + let mut h1 = seed as i32; + for i in (0..data.len()).step_by(4) { + let ints = data.as_ptr().add(i) as *const i32; + let mut half_word = ints.read_unaligned(); + if cfg!(target_endian = "big") { + half_word = half_word.reverse_bits(); + } + h1 = mix_h1(h1, mix_k1(half_word)); + } + h1 + } + let data = data.as_ref(); + let len = data.len(); + let len_aligned = len - len % 4; + + // safety: + // 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, + ); + + for i in len_aligned..len { + let half_word = *data.get_unchecked(i) as i8 as i32; + h1 = mix_h1(h1, mix_k1(half_word)); + } + fmix(h1, len as i32) as u32 + } +} + +#[test] +fn test_murmur3() { + let _hashes = ["", "a", "ab", "abc", "abcd", "abcde"] + .into_iter() + .map(|s| spark_compatible_murmur3_hash(s.as_bytes(), 42) as i32) + .collect::<Vec<_>>(); + let _expected = vec![ + 142593372, 1485273170, -97053317, 1322437556, -396302900, 814637928, + ]; +} + +macro_rules! hash_array { + ($array_type:ident, $column: ident, $hashes: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + if array.null_count() == 0 { + for (i, hash) in $hashes.iter_mut().enumerate() { + *hash = spark_compatible_murmur3_hash(&array.value(i), *hash); + } + } else { + for (i, hash) in $hashes.iter_mut().enumerate() { + if !array.is_null(i) { + *hash = spark_compatible_murmur3_hash(&array.value(i), *hash); + } + } + } + }; +} + +macro_rules! hash_array_primitive { + ($array_type:ident, $column: ident, $ty: ident, $hashes: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let values = array.values(); + + if array.null_count() == 0 { + for (hash, value) in $hashes.iter_mut().zip(values.iter()) { + *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + } + } else { + for (i, (hash, value)) in $hashes.iter_mut().zip(values.iter()).enumerate() { + if !array.is_null(i) { + *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + } + } + } + }; +} + +macro_rules! hash_array_primitive_float { + ($array_type:ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + let values = array.values(); + + if array.null_count() == 0 { + for (hash, value) in $hashes.iter_mut().zip(values.iter()) { + // Spark uses 0 as hash for -0.0, see `Murmur3Hash` expression. + if *value == 0.0 && value.is_sign_negative() { + *hash = spark_compatible_murmur3_hash((0 as $ty2).to_le_bytes(), *hash); + } else { + *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + } + } + } else { + for (i, (hash, value)) in $hashes.iter_mut().zip(values.iter()).enumerate() { + if !array.is_null(i) { + // Spark uses 0 as hash for -0.0, see `Murmur3Hash` expression. + if *value == 0.0 && value.is_sign_negative() { + *hash = spark_compatible_murmur3_hash((0 as $ty2).to_le_bytes(), *hash); + } else { + *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + } + *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + } + } + } + }; +} + +macro_rules! hash_array_decimal { + ($array_type:ident, $column: ident, $hashes: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + + if array.null_count() == 0 { + for (i, hash) in $hashes.iter_mut().enumerate() { + *hash = spark_compatible_murmur3_hash(array.value(i).to_le_bytes(), *hash); + } + } else { + for (i, hash) in $hashes.iter_mut().enumerate() { + if !array.is_null(i) { + *hash = spark_compatible_murmur3_hash(array.value(i).to_le_bytes(), *hash); + } + } + } + }; +} + +/// Hash the values in a dictionary array +fn create_hashes_dictionary<K: ArrowDictionaryKeyType>( + array: &ArrayRef, + hashes_buffer: &mut [u32], +) -> Result<()> { + let dict_array = array.as_any().downcast_ref::<DictionaryArray<K>>().unwrap(); + + // Hash each dictionary value once, and then use that computed + // hash for each key value to avoid a potentially expensive + // redundant hashing for large dictionary elements (e.g. strings) + let dict_values = Arc::clone(dict_array.values()); + let mut dict_hashes = vec![0; dict_values.len()]; + create_hashes(&[dict_values], &mut dict_hashes)?; + + for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { + if let Some(key) = key { + let idx = key.to_usize().ok_or_else(|| { + DataFusionError::Internal(format!( + "Can not convert key value {:?} to usize in dictionary of type {:?}", + key, + dict_array.data_type() + )) + })?; + *hash = dict_hashes[idx] + } // no update for Null, consistent with other hashes + } + Ok(()) +} + +/// Creates hash values for every row, based on the values in the +/// columns. +/// +/// The number of rows to hash is determined by `hashes_buffer.len()`. +/// `hashes_buffer` should be pre-sized appropriately +pub fn create_hashes<'a>( + arrays: &[ArrayRef], + hashes_buffer: &'a mut [u32], +) -> Result<&'a mut [u32]> { + for col in arrays { + match col.data_type() { + DataType::Boolean => { + let array = col.as_any().downcast_ref::<BooleanArray>().unwrap(); + if array.null_count() == 0 { + for (i, hash) in hashes_buffer.iter_mut().enumerate() { + *hash = spark_compatible_murmur3_hash( + i32::from(array.value(i)).to_le_bytes(), + *hash, + ); + } + } else { + for (i, hash) in hashes_buffer.iter_mut().enumerate() { + if !array.is_null(i) { + *hash = spark_compatible_murmur3_hash( + i32::from(array.value(i)).to_le_bytes(), + *hash, + ); + } + } + } + } + DataType::Int8 => { + hash_array_primitive!(Int8Array, col, i32, hashes_buffer); + } + DataType::Int16 => { + hash_array_primitive!(Int16Array, col, i32, hashes_buffer); + } + DataType::Int32 => { + hash_array_primitive!(Int32Array, col, i32, hashes_buffer); + } + DataType::Int64 => { + hash_array_primitive!(Int64Array, col, i64, hashes_buffer); + } + DataType::Float32 => { + hash_array_primitive_float!(Float32Array, col, f32, i32, hashes_buffer); + } + DataType::Float64 => { + hash_array_primitive_float!(Float64Array, col, f64, i64, hashes_buffer); + } + DataType::Timestamp(TimeUnit::Second, _) => { + hash_array_primitive!(TimestampSecondArray, col, i64, hashes_buffer); + } + DataType::Timestamp(TimeUnit::Millisecond, _) => { + hash_array_primitive!(TimestampMillisecondArray, col, i64, hashes_buffer); + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + hash_array_primitive!(TimestampMicrosecondArray, col, i64, hashes_buffer); + } + DataType::Timestamp(TimeUnit::Nanosecond, _) => { + hash_array_primitive!(TimestampNanosecondArray, col, i64, hashes_buffer); + } + DataType::Date32 => { + hash_array_primitive!(Date32Array, col, i32, hashes_buffer); + } + DataType::Date64 => { + hash_array_primitive!(Date64Array, col, i64, hashes_buffer); + } + DataType::Utf8 => { + hash_array!(StringArray, col, hashes_buffer); + } + DataType::LargeUtf8 => { + hash_array!(LargeStringArray, col, hashes_buffer); + } + DataType::FixedSizeBinary(_) => { + hash_array!(FixedSizeBinaryArray, col, hashes_buffer); + } + DataType::Decimal128(_, _) => { + hash_array_decimal!(Decimal128Array, col, hashes_buffer); + } + DataType::Dictionary(index_type, _) => match **index_type { + DataType::Int8 => { + create_hashes_dictionary::<Int8Type>(col, hashes_buffer)?; + } + DataType::Int16 => { + create_hashes_dictionary::<Int16Type>(col, hashes_buffer)?; + } + DataType::Int32 => { + create_hashes_dictionary::<Int32Type>(col, hashes_buffer)?; + } + DataType::Int64 => { + create_hashes_dictionary::<Int64Type>(col, hashes_buffer)?; + } + DataType::UInt8 => { + create_hashes_dictionary::<UInt8Type>(col, hashes_buffer)?; + } + DataType::UInt16 => { + create_hashes_dictionary::<UInt16Type>(col, hashes_buffer)?; + } + DataType::UInt32 => { + create_hashes_dictionary::<UInt32Type>(col, hashes_buffer)?; + } + DataType::UInt64 => { + create_hashes_dictionary::<UInt64Type>(col, hashes_buffer)?; + } + _ => { + return Err(DataFusionError::Internal(format!( + "Unsupported dictionary type in hasher hashing: {}", + col.data_type(), + ))) + } + }, + _ => { + // This is internal because we should have caught this before. + return Err(DataFusionError::Internal(format!( + "Unsupported data type in hasher: {}", + col.data_type() + ))); + } + } + } + Ok(hashes_buffer) +} + +pub(crate) fn pmod(hash: u32, n: usize) -> usize { + let hash = hash as i32; + let n = n as i32; + let r = hash % n; + let result = if r < 0 { (r + n) % n } else { r }; + result as usize +} + +#[cfg(test)] +mod tests { + use arrow::array::{Float32Array, Float64Array}; + use std::sync::Arc; + + use crate::execution::datafusion::spark_hash::{create_hashes, pmod}; + use datafusion::arrow::array::{ArrayRef, Int32Array, Int64Array, Int8Array, StringArray}; + + #[test] + fn test_i8() { + let i = Arc::new(Int8Array::from(vec![ + Some(1), + Some(0), + Some(-1), + Some(i8::MAX), + Some(i8::MIN), + ])) as ArrayRef; + let mut hashes = vec![42; 5]; + create_hashes(&[i], &mut hashes).unwrap(); + + // generated with Spark Murmur3_x86_32 + let expected = vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365]; + assert_eq!(hashes, expected); + } + + #[test] + fn test_i32() { + let i = Arc::new(Int32Array::from(vec![ + Some(1), + Some(0), + Some(-1), + Some(i32::MAX), + Some(i32::MIN), + ])) as ArrayRef; + let mut hashes = vec![42; 5]; + create_hashes(&[i], &mut hashes).unwrap(); + + // generated with Spark Murmur3_x86_32 + let expected = vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6]; + assert_eq!(hashes, expected); + } + + #[test] + fn test_i64() { + let i = Arc::new(Int64Array::from(vec![ + Some(1), + Some(0), + Some(-1), + Some(i64::MAX), + Some(i64::MIN), + ])) as ArrayRef; + let mut hashes = vec![42; 5]; + create_hashes(&[i], &mut hashes).unwrap(); + + // generated with Spark Murmur3_x86_32 + let expected = vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb]; + assert_eq!(hashes, expected); + } + + #[test] + fn test_f32() { + let i = Arc::new(Float32Array::from(vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ])) as ArrayRef; + let mut hashes = vec![42; 6]; + create_hashes(&[i], &mut hashes).unwrap(); + + // generated with Spark Murmur3_x86_32 + let expected = vec![ + 0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86, + ]; + assert_eq!(hashes, expected); + } + + #[test] + fn test_f64() { + let i = Arc::new(Float64Array::from(vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ])) as ArrayRef; + let mut hashes = vec![42; 6]; + create_hashes(&[i], &mut hashes).unwrap(); + + // generated with Spark Murmur3_x86_32 + let expected = vec![ + 0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9, + ]; + assert_eq!(hashes, expected); + } + + #[test] + fn test_str() { + let i = Arc::new(StringArray::from(vec!["hello", "bar", "", "😁", "天地"])); + let mut hashes = vec![42; 5]; + create_hashes(&[i], &mut hashes).unwrap(); + + // generated with Murmur3Hash(Seq(Literal("")), 42).eval() since Spark is tested against + // this as well + let expected = vec![3286402344, 2486176763, 142593372, 885025535, 2395000894]; + assert_eq!(hashes, expected); + } + + #[test] + fn test_pmod() { + let i: Vec<u32> = vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb]; + let result = i.into_iter().map(|i| pmod(i, 200)).collect::<Vec<usize>>(); + + // expected partition from Spark with n=200 + let expected = vec![69, 5, 193, 171, 115]; + assert_eq!(result, expected); + } +} diff --git a/core/src/execution/jni_api.rs b/core/src/execution/jni_api.rs new file mode 100644 index 0000000000..e940a0853d --- /dev/null +++ b/core/src/execution/jni_api.rs @@ -0,0 +1,507 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Define JNI APIs which can be called from Java/Scala. + +use crate::execution::operators::{InputBatch, ScanExec}; +use arrow::{ + array::{make_array, Array, ArrayData, ArrayRef}, + datatypes::DataType as ArrowDataType, + ffi::{FFI_ArrowArray, FFI_ArrowSchema}, +}; +use arrow_array::RecordBatch; +use datafusion::{ + execution::{ + disk_manager::DiskManagerConfig, + runtime_env::{RuntimeConfig, RuntimeEnv}, + }, + physical_plan::{display::DisplayableExecutionPlan, ExecutionPlan, SendableRecordBatchStream}, + prelude::{SessionConfig, SessionContext}, +}; +use datafusion_common::DataFusionError; +use futures::poll; +use jni::{ + errors::Result as JNIResult, + objects::{JClass, JMap, JObject, JString, ReleaseMode}, + sys::{jbyteArray, jint, jlong, jlongArray}, + JNIEnv, +}; +use std::{collections::HashMap, sync::Arc, task::Poll}; + +use super::{serde, utils::SparkArrowConvert}; + +use crate::{ + errors::{try_unwrap_or_throw, CometError}, + execution::{ + datafusion::planner::PhysicalPlanner, metrics::utils::update_comet_metric, + serde::to_arrow_datatype, spark_operator::Operator, + }, + jvm_bridge::{jni_new_global_ref, JVMClasses}, +}; +use futures::stream::StreamExt; +use jni::{ + objects::{AutoArray, GlobalRef}, + sys::{jbooleanArray, jobjectArray}, +}; +use tokio::runtime::Runtime; + +use log::info; + +/// Comet native execution context. Kept alive across JNI calls. +struct ExecutionContext { + /// The id of the execution context. + pub id: i64, + /// The deserialized Spark plan + pub spark_plan: Operator, + /// The DataFusion root operator converted from the `spark_plan` + pub root_op: Option<Arc<dyn ExecutionPlan>>, + /// The input sources for the DataFusion plan + pub scans: Vec<ScanExec>, + /// The record batch stream to pull results from + pub stream: Option<SendableRecordBatchStream>, + /// The FFI arrays. We need to keep them alive here. + pub ffi_arrays: Vec<(Arc<FFI_ArrowArray>, Arc<FFI_ArrowSchema>)>, + /// Configurations for DF execution + pub conf: HashMap<String, String>, + /// The Tokio runtime used for async. + pub runtime: Runtime, + /// Native metrics + pub metrics: Arc<GlobalRef>, + /// DataFusion SessionContext + pub session_ctx: Arc<SessionContext>, + /// Whether to enable additional debugging checks & messages + pub debug_native: bool, +} + +#[no_mangle] +/// Accept serialized query plan and return the address of the native query plan. +pub extern "system" fn Java_org_apache_comet_Native_createPlan( + env: JNIEnv, + _class: JClass, + id: jlong, + config_object: JObject, + serialized_query: jbyteArray, + metrics_node: JObject, +) -> jlong { + try_unwrap_or_throw(env, || { + // Init JVM classes + JVMClasses::init(&env); + + let bytes = env.convert_byte_array(serialized_query)?; + + // Deserialize query plan + let spark_plan = serde::deserialize_op(bytes.as_slice())?; + + // Sets up context + let mut configs = HashMap::new(); + + let config_map = JMap::from_env(&env, config_object)?; + config_map.iter()?.for_each(|config| { + let key: String = env.get_string(JString::from(config.0)).unwrap().into(); + let value: String = env.get_string(JString::from(config.1)).unwrap().into(); + + configs.insert(key, value); + }); + + // Whether we've enabled additional debugging on the native side + let debug_native = configs + .get("debug_native") + .and_then(|x| x.parse::<bool>().ok()) + .unwrap_or(false); + + // Use multi-threaded tokio runtime to prevent blocking spawned tasks if any + let runtime = tokio::runtime::Builder::new_multi_thread() + .enable_all() + .build()?; + + let metrics = Arc::new(jni_new_global_ref!(env, metrics_node)?); + + // We need to keep the session context alive. Some session state like temporary + // dictionaries are stored in session context. If it is dropped, the temporary + // dictionaries will be dropped as well. + let session = prepare_datafusion_session_context(&configs)?; + + let exec_context = Box::new(ExecutionContext { + id, + spark_plan, + root_op: None, + scans: vec![], + stream: None, + ffi_arrays: vec![], + conf: configs, + runtime, + metrics, + session_ctx: Arc::new(session), + debug_native, + }); + + Ok(Box::into_raw(exec_context) as i64) + }) +} + +/// Parse Comet configs and configure DataFusion session context. +fn prepare_datafusion_session_context( + conf: &HashMap<String, String>, +) -> Result<SessionContext, CometError> { + // Get the batch size from Comet JVM side + let batch_size = conf + .get("batch_size") + .ok_or(CometError::Internal( + "Config 'batch_size' is not specified from Comet JVM side".to_string(), + ))? + .parse::<usize>()?; + + let mut rt_config = RuntimeConfig::new().with_disk_manager(DiskManagerConfig::NewOs); + + // Set up memory limit if specified + if conf.contains_key("memory_limit") { + let memory_limit = conf.get("memory_limit").unwrap().parse::<usize>()?; + + let memory_fraction = conf + .get("memory_fraction") + .ok_or(CometError::Internal( + "Config 'memory_fraction' is not specified from Comet JVM side".to_string(), + ))? + .parse::<f64>()?; + + rt_config = rt_config.with_memory_limit(memory_limit, memory_fraction); + } + + // Get Datafusion configuration from Spark Execution context + // can be configured in Comet Spark JVM using Spark --conf parameters + // e.g: spark-shell --conf spark.datafusion.sql_parser.parse_float_as_decimal=true + let df_config = conf + .iter() + .filter(|(k, _)| k.starts_with("datafusion.")) + .map(|kv| (kv.0.to_owned(), kv.1.to_owned())) + .collect::<Vec<(String, String)>>(); + + let session_config = + SessionConfig::from_string_hash_map(std::collections::HashMap::from_iter(df_config))? + .with_batch_size(batch_size); + + let runtime = RuntimeEnv::new(rt_config).unwrap(); + + Ok(SessionContext::new_with_config_rt( + session_config, + Arc::new(runtime), + )) +} + +/// Prepares arrow arrays for output. +fn prepare_output( + output: Result<RecordBatch, DataFusionError>, + env: JNIEnv, + exec_context: &mut ExecutionContext, +) -> Result<jlongArray, CometError> { + let output_batch = output?; + let results = output_batch.columns(); + let num_rows = output_batch.num_rows(); + + if exec_context.debug_native { + // Validate the output arrays. + for array in results.iter() { + let array_data = array.to_data(); + array_data + .validate_full() + .expect("Invalid output array data"); + } + } + + let return_flag = 1; + + let long_array = env.new_long_array((results.len() * 2) as i32 + 2)?; + env.set_long_array_region(long_array, 0, &[return_flag, num_rows as jlong])?; + + let mut arrays = vec![]; + + let mut i = 0; + while i < results.len() { + let array_ref = results.get(i).ok_or(CometError::IndexOutOfBounds(i))?; + let (array, schema) = array_ref.to_data().to_spark()?; + + unsafe { + let arrow_array = Arc::from_raw(array as *const FFI_ArrowArray); + let arrow_schema = Arc::from_raw(schema as *const FFI_ArrowSchema); + arrays.push((arrow_array, arrow_schema)); + } + + env.set_long_array_region(long_array, (i * 2) as i32 + 2, &[array, schema])?; + i += 1; + } + + // Update metrics + update_metrics(&env, exec_context)?; + + // Record the pointer to allocated Arrow Arrays + exec_context.ffi_arrays = arrays; + + Ok(long_array) +} + +#[no_mangle] +/// Accept serialized query plan and the addresses of Arrow Arrays from Spark, +/// then execute the query. Return addresses of arrow vector. +pub extern "system" fn Java_org_apache_comet_Native_executePlan( + env: JNIEnv, + _class: JClass, + exec_context: jlong, + addresses_array: jobjectArray, + finishes: jbooleanArray, + batch_rows: jint, +) -> jlongArray { + try_unwrap_or_throw(env, || { + let addresses_vec = convert_addresses_arrays(&env, addresses_array)?; + let mut all_inputs: Vec<Vec<ArrayRef>> = Vec::with_capacity(addresses_vec.len()); + + let exec_context = get_execution_context(exec_context); + for addresses in addresses_vec.iter() { + let mut inputs: Vec<ArrayRef> = vec![]; + + let array_num = addresses.size()? as usize; + assert_eq!(array_num % 2, 0, "Arrow Array addresses are invalid!"); + + let num_arrays = array_num / 2; + let array_elements = addresses.as_ptr(); + + let mut i: usize = 0; + while i < num_arrays { + let array_ptr = unsafe { *(array_elements.add(i * 2)) }; + let schema_ptr = unsafe { *(array_elements.add(i * 2 + 1)) }; + let array_data = ArrayData::from_spark((array_ptr, schema_ptr))?; + + if exec_context.debug_native { + // Validate the array data from JVM. + array_data.validate_full().expect("Invalid array data"); + } + + inputs.push(make_array(array_data)); + i += 1; + } + + all_inputs.push(inputs); + } + + // Prepares the input batches. + let eofs = env.get_boolean_array_elements(finishes, ReleaseMode::NoCopyBack)?; + let eof_flags = eofs.as_ptr(); + + // Whether reaching the end of input batches. + let mut finished = true; + let mut input_batches = all_inputs + .into_iter() + .enumerate() + .map(|(idx, inputs)| unsafe { + let eof = eof_flags.add(idx); + + if *eof == 1 { + InputBatch::EOF + } else { + finished = false; + InputBatch::new(inputs, Some(batch_rows as usize)) + } + }) + .collect::<Vec<InputBatch>>(); + + // Retrieve the query + let exec_context_id = exec_context.id; + + // Initialize the execution stream. + // Because we don't know if input arrays are dictionary-encoded when we create + // query plan, we need to defer stream initialization to first time execution. + if exec_context.root_op.is_none() { + let planner = PhysicalPlanner::new().with_exec_id(exec_context_id); + let (scans, root_op) = + planner.create_plan(&exec_context.spark_plan, &mut input_batches)?; + + exec_context.root_op = Some(root_op.clone()); + exec_context.scans = scans; + + if exec_context.debug_native { + let formatted_plan_str = + DisplayableExecutionPlan::new(root_op.as_ref()).indent(true); + info!("Comet native query plan:\n {formatted_plan_str:}"); + } + + let task_ctx = exec_context.session_ctx.task_ctx(); + let stream = exec_context + .root_op + .as_ref() + .unwrap() + .execute(0, task_ctx)?; + exec_context.stream = Some(stream); + } else { + input_batches + .into_iter() + .enumerate() + .for_each(|(idx, input_batch)| { + let scan = &mut exec_context.scans[idx]; + + // Set inputs at `Scan` node. + scan.set_input_batch(input_batch); + }); + } + + loop { + // Polling the stream. + let next_item = exec_context.stream.as_mut().unwrap().next(); + let poll_output = exec_context.runtime.block_on(async { poll!(next_item) }); + + match poll_output { + Poll::Ready(Some(output)) => { + return prepare_output(output, env, exec_context); + } + Poll::Ready(None) => { + // Reaches EOF of output. + + // Update metrics + update_metrics(&env, exec_context)?; + + let long_array = env.new_long_array(1)?; + env.set_long_array_region(long_array, 0, &[-1])?; + + return Ok(long_array); + } + // After reaching the end of any input, a poll pending means there are more than one + // blocking operators, we don't need go back-forth between JVM/Native. Just + // keeping polling. + Poll::Pending if finished => { + // Update metrics + update_metrics(&env, exec_context)?; + + // Output not ready yet + continue; + } + // Not reaching the end of input yet, so a poll pending means there are blocking + // operators. Just returning to keep reading next input. + Poll::Pending => { + // Update metrics + update_metrics(&env, exec_context)?; + return return_pending(env); + } + } + } + }) +} + +fn return_pending(env: JNIEnv) -> Result<jlongArray, CometError> { + let long_array = env.new_long_array(1)?; + env.set_long_array_region(long_array, 0, &[0])?; + + Ok(long_array) +} + +#[no_mangle] +/// Peeks into next output if any. +pub extern "system" fn Java_org_apache_comet_Native_peekNext( + env: JNIEnv, + _class: JClass, + exec_context: jlong, +) -> jlongArray { + try_unwrap_or_throw(env, || { + // Retrieve the query + let exec_context = get_execution_context(exec_context); + + if exec_context.stream.is_none() { + // Plan is not initialized yet. + return return_pending(env); + } + + // Polling the stream. + let next_item = exec_context.stream.as_mut().unwrap().next(); + let poll_output = exec_context.runtime.block_on(async { poll!(next_item) }); + + match poll_output { + Poll::Ready(Some(output)) => prepare_output(output, env, exec_context), + _ => { + // Update metrics + update_metrics(&env, exec_context)?; + return_pending(env) + } + } + }) +} + +#[no_mangle] +/// Drop the native query plan object and context object. +pub extern "system" fn Java_org_apache_comet_Native_releasePlan( + env: JNIEnv, + _class: JClass, + exec_context: jlong, +) { + try_unwrap_or_throw(env, || unsafe { + let execution_context = get_execution_context(exec_context); + let _: Box<ExecutionContext> = Box::from_raw(execution_context); + Ok(()) + }) +} + +/// Updates the metrics of the query plan. +fn update_metrics(env: &JNIEnv, exec_context: &ExecutionContext) -> Result<(), CometError> { + let native_query = exec_context.root_op.as_ref().unwrap(); + let metrics = exec_context.metrics.as_obj(); + update_comet_metric(env, metrics, native_query) +} + +/// Converts a Java array of address arrays to a Rust vector of address arrays. +fn convert_addresses_arrays<'a>( + env: &'a JNIEnv<'a>, + addresses_array: jobjectArray, +) -> JNIResult<Vec<AutoArray<'a, 'a, jlong>>> { + let array_len = env.get_array_length(addresses_array)?; + let mut res: Vec<AutoArray<jlong>> = Vec::new(); + + for i in 0..array_len { + let array: AutoArray<jlong> = env.get_array_elements( + env.get_object_array_element(addresses_array, i)? + .into_inner() as jlongArray, + ReleaseMode::NoCopyBack, + )?; + res.push(array); + } + + Ok(res) +} + +fn convert_datatype_arrays( + env: &'_ JNIEnv<'_>, + serialized_datatypes: jobjectArray, +) -> JNIResult<Vec<ArrowDataType>> { + let array_len = env.get_array_length(serialized_datatypes)?; + let mut res: Vec<ArrowDataType> = Vec::new(); + + for i in 0..array_len { + let array = env + .get_object_array_element(serialized_datatypes, i)? + .into_inner() as jbyteArray; + + let bytes = env.convert_byte_array(array)?; + let data_type = serde::deserialize_data_type(bytes.as_slice()).unwrap(); + let arrow_dt = to_arrow_datatype(&data_type); + res.push(arrow_dt); + } + + Ok(res) +} + +fn get_execution_context<'a>(id: i64) -> &'a mut ExecutionContext { + unsafe { + (id as *mut ExecutionContext) + .as_mut() + .expect("Comet execution context shouldn't be null!") + } +} diff --git a/core/src/execution/kernels/hash.rs b/core/src/execution/kernels/hash.rs new file mode 100644 index 0000000000..de30f74cdf --- /dev/null +++ b/core/src/execution/kernels/hash.rs @@ -0,0 +1,171 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 crate::common::bit; +use ahash::RandomState; +use arrow::datatypes::{i256, ArrowNativeType}; +use arrow_array::{ + downcast_dictionary_array, downcast_primitive_array, Array, ArrayAccessor, ArrayRef, + ArrowPrimitiveType, PrimitiveArray, +}; +use std::fmt::Debug; + +pub fn hash(src: &[ArrayRef], dst: &mut [u64]) { + let state = RandomState::with_seed(42); + src.iter().enumerate().for_each(|(idx, v)| { + downcast_dictionary_array!( + v => { + let keys = v.keys(); + let values = v.values(); + downcast_primitive_array!( + values => hash_dict_typed(&state, idx > 0, keys, values, dst), + dt => panic!("Expected only primitive type but found {}", dt) + ) + }, + dt => { + downcast_primitive_array!( + v => hash_typed(&state, idx > 0, v, dst), + _ => panic!("Expected only primitive type but found {}", dt) + ) + } + ) + }); +} + +fn hash_typed<T>(state: &RandomState, mix: bool, array: T, dst: &mut [u64]) +where + T: ArrayAccessor, + T::Item: Hashable + Debug, +{ + let nullable = array.null_count() > 0; + let num_values = array.len(); + if nullable { + for i in 0..num_values { + if !array.is_null(i) { + unsafe { + let value = array.value_unchecked(i); + hash1(state, mix, i, value, dst); + } + } + } + } else { + for i in 0..num_values { + unsafe { + let value = array.value_unchecked(i); + hash1(state, mix, i, value, dst); + } + } + } +} + +fn hash_dict_typed<K, V>( + state: &RandomState, + mix: bool, + keys: &PrimitiveArray<K>, + values: V, + dst: &mut [u64], +) where + K: ArrowPrimitiveType, + V: ArrayAccessor, + V::Item: Hashable + Debug, +{ + let nullable = keys.null_count() > 0; + let num_keys = keys.len(); + let mut value_hashes = vec![0; values.len()]; + + for (i, value_hash) in value_hashes.iter_mut().enumerate() { + unsafe { + *value_hash = values.value_unchecked(i).create_hash(state); + } + } + + if nullable { + for i in 0..num_keys { + if !keys.is_null(i) { + unsafe { + let idx = keys.value_unchecked(i); + let hash = value_hashes[idx.as_usize()]; + hash1_helper(mix, i, hash, dst); + } + } + } + } else { + for i in 0..num_keys { + unsafe { + let idx = keys.value_unchecked(i); + let hash = value_hashes[idx.as_usize()]; + hash1_helper(mix, i, hash, dst); + } + } + } +} + +#[inline(always)] +fn hash1<T: Hashable>(state: &RandomState, mix: bool, i: usize, value: T, dst: &mut [u64]) { + let hash = value.create_hash(state); + hash1_helper(mix, i, hash, dst); +} + +#[inline(always)] +fn hash1_helper(mix: bool, i: usize, hash: u64, dst: &mut [u64]) { + dst[i] = if mix { + bit::mix_hash(dst[i], hash) + } else { + hash + } +} + +pub(crate) trait Hashable { + fn create_hash(&self, state: &RandomState) -> u64; +} + +macro_rules! impl_hashable { + ($($t:ty),+) => { + $(impl Hashable for $t { + #[inline] + fn create_hash(&self, state: &RandomState) -> u64 { + state.hash_one(self) + } + })+ + }; +} + +impl_hashable!(i8, i16, i32, u8, u16, u32, u64, i128, i256); + +impl Hashable for i64 { + fn create_hash(&self, state: &RandomState) -> u64 { + state.hash_one(self) + } +} + +impl Hashable for half::f16 { + fn create_hash(&self, _: &RandomState) -> u64 { + unimplemented!("hashing on f16 is not supported") + } +} + +impl Hashable for f32 { + fn create_hash(&self, state: &RandomState) -> u64 { + state.hash_one(u32::from_ne_bytes(self.to_ne_bytes())) + } +} + +impl Hashable for f64 { + fn create_hash(&self, state: &RandomState) -> u64 { + state.hash_one(u64::from_ne_bytes(self.to_ne_bytes())) + } +} diff --git a/core/src/execution/kernels/mod.rs b/core/src/execution/kernels/mod.rs new file mode 100644 index 0000000000..76d4e18070 --- /dev/null +++ b/core/src/execution/kernels/mod.rs @@ -0,0 +1,24 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Kernels + +mod hash; +pub use hash::hash; + +pub(crate) mod strings; +pub(crate) mod temporal; diff --git a/core/src/execution/kernels/strings.rs b/core/src/execution/kernels/strings.rs new file mode 100644 index 0000000000..2e5e67b67f --- /dev/null +++ b/core/src/execution/kernels/strings.rs @@ -0,0 +1,243 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! String kernels + +use std::sync::Arc; + +use arrow::{ + array::*, + buffer::{Buffer, MutableBuffer}, + compute::kernels::substring::{substring as arrow_substring, substring_by_char}, + datatypes::{DataType, Int32Type}, +}; + +use crate::errors::ExpressionError; + +/// Returns an ArrayRef with a string consisting of `length` spaces. +/// +/// # Preconditions +/// +/// - elements in `length` must not be negative +pub fn string_space(length: &dyn Array) -> Result<ArrayRef, ExpressionError> { + match length.data_type() { + DataType::Int32 => { + let array = length.as_any().downcast_ref::<Int32Array>().unwrap(); + Ok(generic_string_space::<i32>(array)) + } + DataType::Dictionary(_, _) => { + let dict = as_dictionary_array::<Int32Type>(length); + let values = string_space(dict.values())?; + let result = DictionaryArray::try_new(dict.keys().clone(), values)?; + Ok(Arc::new(result)) + } + dt => panic!( + "Unsupported input type for function 'string_space': {:?}", + dt + ), + } +} + +pub fn substring(array: &dyn Array, start: i64, length: u64) -> Result<ArrayRef, ExpressionError> { + match array.data_type() { + DataType::LargeUtf8 => substring_by_char( + array + .as_any() + .downcast_ref::<LargeStringArray>() + .expect("A large string is expected"), + start, + Some(length), + ) + .map_err(|e| e.into()) + .map(|t| make_array(t.into_data())), + DataType::Utf8 => substring_by_char( + array + .as_any() + .downcast_ref::<StringArray>() + .expect("A string is expected"), + start, + Some(length), + ) + .map_err(|e| e.into()) + .map(|t| make_array(t.into_data())), + DataType::Binary | DataType::LargeBinary => { + arrow_substring(array, start, Some(length)).map_err(|e| e.into()) + } + DataType::Dictionary(_, _) => { + let dict = as_dictionary_array::<Int32Type>(array); + let values = substring(dict.values(), start, length)?; + let result = DictionaryArray::try_new(dict.keys().clone(), values)?; + Ok(Arc::new(result)) + } + dt => panic!("Unsupported input type for function 'substring': {:?}", dt), + } +} + +/// Returns an ArrayRef with a substring starting from `start` and length. +/// +/// # Preconditions +/// +/// - `start` can be negative, in which case the start counts from the end of the string. +/// - `array` must be either [`StringArray`] or [`LargeStringArray`]. +/// +/// Note: this is different from arrow-rs `substring` kernel in that both `start` and `length` are +/// `Int32Array` here. +pub fn substring_with_array( + array: &dyn Array, + start: &Int32Array, + length: &Int32Array, +) -> ArrayRef { + match array.data_type() { + DataType::LargeUtf8 => generic_substring( + array + .as_any() + .downcast_ref::<LargeStringArray>() + .expect("A large string is expected"), + start, + length, + |i| i as i64, + ), + DataType::Utf8 => generic_substring( + array + .as_any() + .downcast_ref::<StringArray>() + .expect("A string is expected"), + start, + length, + |i| i, + ), + _ => panic!("substring does not support type {:?}", array.data_type()), + } +} + +fn generic_string_space<OffsetSize: OffsetSizeTrait>(length: &Int32Array) -> ArrayRef { + let array_len = length.len(); + let mut offsets = MutableBuffer::new((array_len + 1) * std::mem::size_of::<OffsetSize>()); + let mut length_so_far = OffsetSize::zero(); + + // compute null bitmap (copy) + let null_bit_buffer = length.to_data().nulls().map(|b| b.buffer().clone()); + + // Gets slice of length array to access it directly for performance. + let length_data = length.to_data(); + let lengths = length_data.buffers()[0].typed_data::<i32>(); + let total = lengths.iter().map(|l| *l as usize).sum::<usize>(); + let mut values = MutableBuffer::new(total); + + offsets.push(length_so_far); + + let blank = " ".as_bytes()[0]; + values.resize(total, blank); + + (0..array_len).for_each(|i| { + let current_len = lengths[i] as usize; + + length_so_far += OffsetSize::from_usize(current_len).unwrap(); + offsets.push(length_so_far); + }); + + let data = unsafe { + ArrayData::new_unchecked( + GenericStringArray::<OffsetSize>::DATA_TYPE, + array_len, + None, + null_bit_buffer, + 0, + vec![offsets.into(), values.into()], + vec![], + ) + }; + make_array(data) +} + +fn generic_substring<OffsetSize: OffsetSizeTrait, F>( + array: &GenericStringArray<OffsetSize>, + start: &Int32Array, + length: &Int32Array, + f: F, +) -> ArrayRef +where + F: Fn(i32) -> OffsetSize, +{ + assert_eq!(array.len(), start.len()); + assert_eq!(array.len(), length.len()); + + // compute current offsets + let offsets = array.to_data().buffers()[0].clone(); + let offsets: &[OffsetSize] = offsets.typed_data::<OffsetSize>(); + + // compute null bitmap (copy) + let null_bit_buffer = array.to_data().nulls().map(|b| b.buffer().clone()); + + // Gets slices of start and length arrays to access them directly for performance. + let start_data = start.to_data(); + let length_data = length.to_data(); + let starts = start_data.buffers()[0].typed_data::<i32>(); + let lengths = length_data.buffers()[0].typed_data::<i32>(); + + // compute values + let array_data = array.to_data(); + let values = &array_data.buffers()[1]; + let data = values.as_slice(); + + // we have no way to estimate how much this will be. + let mut new_values = MutableBuffer::new(0); + let mut new_offsets: Vec<OffsetSize> = Vec::with_capacity(array.len() + 1); + + let mut length_so_far = OffsetSize::zero(); + new_offsets.push(length_so_far); + (0..array.len()).for_each(|i| { + // the length of this entry + let length_i: OffsetSize = offsets[i + 1] - offsets[i]; + // compute where we should start slicing this entry + let start_pos: OffsetSize = f(starts[i]); + + let start = offsets[i] + + if start_pos >= OffsetSize::zero() { + start_pos + } else { + length_i + start_pos + }; + + let start = start.clamp(offsets[i], offsets[i + 1]); + // compute the length of the slice + let slice_length: OffsetSize = f(lengths[i].max(0)).min(offsets[i + 1] - start); + + length_so_far += slice_length; + + new_offsets.push(length_so_far); + + // we need usize for ranges + let start = start.to_usize().unwrap(); + let slice_length = slice_length.to_usize().unwrap(); + + new_values.extend_from_slice(&data[start..start + slice_length]); + }); + + let data = unsafe { + ArrayData::new_unchecked( + GenericStringArray::<OffsetSize>::DATA_TYPE, + array.len(), + None, + null_bit_buffer, + 0, + vec![Buffer::from_slice_ref(&new_offsets), new_values.into()], + vec![], + ) + }; + make_array(data) +} diff --git a/core/src/execution/kernels/temporal.rs b/core/src/execution/kernels/temporal.rs new file mode 100644 index 0000000000..ec7f2be7e0 --- /dev/null +++ b/core/src/execution/kernels/temporal.rs @@ -0,0 +1,438 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! temporal kernels + +use chrono::{DateTime, Datelike, Duration, NaiveDateTime, Timelike, Utc}; + +use std::sync::Arc; + +use arrow::{array::*, datatypes::DataType}; +use arrow_array::{ + downcast_dictionary_array, downcast_temporal_array, + temporal_conversions::*, + timezone::Tz, + types::{ArrowTemporalType, Date32Type, TimestampMicrosecondType}, + ArrowNumericType, +}; + +use arrow_schema::TimeUnit; + +use crate::errors::ExpressionError; + +// Copied from arrow_arith/temporal.rs +macro_rules! return_compute_error_with { + ($msg:expr, $param:expr) => { + return { + Err(ExpressionError::ArrowError(format!( + "{}: {:?}", + $msg, $param + ))) + } + }; +} + +// The number of days between the beginning of the proleptic gregorian calendar (0001-01-01) +// and the beginning of the Unix Epoch (1970-01-01) +const DAYS_TO_UNIX_EPOCH: i32 = 719_163; +const MICROS_TO_UNIX_EPOCH: i64 = 62_167_132_800 * 1_000_000; + +// Copied from arrow_arith/temporal.rs with modification to the output datatype +// Transforms a array of NaiveDate to an array of Date32 after applying an operation +fn as_datetime_with_op<A: ArrayAccessor<Item = T::Native>, T: ArrowTemporalType, F>( + iter: ArrayIter<A>, + mut builder: PrimitiveBuilder<Date32Type>, + op: F, +) -> Date32Array +where + F: Fn(NaiveDateTime) -> i32, + i64: From<T::Native>, +{ + iter.into_iter().for_each(|value| { + if let Some(value) = value { + match as_datetime::<T>(i64::from(value)) { + Some(dt) => builder.append_value(op(dt)), + None => builder.append_null(), + } + } else { + builder.append_null(); + } + }); + + builder.finish() +} + +// Based on arrow_arith/temporal.rs:extract_component_from_datetime_array +// Transforms an array of DateTime<Tz> to an arrayOf TimeStampMicrosecond after applying an +// operation +fn as_timestamp_tz_with_op<A: ArrayAccessor<Item = T::Native>, T: ArrowTemporalType, F>( + iter: ArrayIter<A>, + mut builder: PrimitiveBuilder<TimestampMicrosecondType>, + tz: &str, + op: F, +) -> Result<TimestampMicrosecondArray, ExpressionError> +where + F: Fn(DateTime<Tz>) -> i64, + i64: From<T::Native>, +{ + let tz: Tz = tz.parse()?; + for value in iter { + match value { + Some(value) => match as_datetime_with_timezone::<T>(value.into(), tz) { + Some(time) => builder.append_value(op(time)), + _ => { + return Err(ExpressionError::ArrowError( + "Unable to read value as datetime".to_string(), + )); + } + }, + None => builder.append_null(), + } + } + Ok(builder.finish()) +} + +#[inline] +fn as_days_from_unix_epoch(dt: Option<NaiveDateTime>) -> i32 { + dt.unwrap().num_days_from_ce() - DAYS_TO_UNIX_EPOCH +} + +// Apply the Tz to the Naive Date Time,,convert to UTC, and return as microseconds in Unix epoch +#[inline] +fn as_micros_from_unix_epoch_utc(dt: Option<DateTime<Tz>>) -> i64 { + dt.unwrap().with_timezone(&Utc).timestamp_micros() +} + +#[inline] +fn trunc_date_to_year<T: Datelike + Timelike>(dt: T) -> Option<T> { + Some(dt) + .and_then(|d| d.with_nanosecond(0)) + .and_then(|d| d.with_second(0)) + .and_then(|d| d.with_minute(0)) + .and_then(|d| d.with_hour(0)) + .and_then(|d| d.with_day0(0)) + .and_then(|d| d.with_month0(0)) +} + +/// returns the month of the beginning of the quarter +#[inline] +fn quarter_month<T: Datelike>(dt: &T) -> u32 { + 1 + 3 * ((dt.month() - 1) / 3) +} + +#[inline] +fn trunc_date_to_quarter<T: Datelike + Timelike>(dt: T) -> Option<T> { + Some(dt) + .and_then(|d| d.with_nanosecond(0)) + .and_then(|d| d.with_second(0)) + .and_then(|d| d.with_minute(0)) + .and_then(|d| d.with_hour(0)) + .and_then(|d| d.with_day0(0)) + .and_then(|d| d.with_month(quarter_month(&d))) +} + +#[inline] +fn trunc_date_to_month<T: Datelike + Timelike>(dt: T) -> Option<T> { + Some(dt) + .and_then(|d| d.with_nanosecond(0)) + .and_then(|d| d.with_second(0)) + .and_then(|d| d.with_minute(0)) + .and_then(|d| d.with_hour(0)) + .and_then(|d| d.with_day0(0)) +} + +#[inline] +fn trunc_date_to_week<T>(dt: T) -> Option<T> +where + T: Datelike + Timelike + std::ops::Sub<Duration, Output = T> + Copy, +{ + Some(dt) + .map(|d| d - Duration::seconds(60 * 60 * 24 * d.weekday() as i64)) + .and_then(|d| d.with_nanosecond(0)) + .and_then(|d| d.with_second(0)) + .and_then(|d| d.with_minute(0)) + .and_then(|d| d.with_hour(0)) +} + +#[inline] +fn trunc_date_to_day<T: Timelike>(dt: T) -> Option<T> { + Some(dt) + .and_then(|d| d.with_nanosecond(0)) + .and_then(|d| d.with_second(0)) + .and_then(|d| d.with_minute(0)) + .and_then(|d| d.with_hour(0)) +} + +#[inline] +fn trunc_date_to_hour<T: Timelike>(dt: T) -> Option<T> { + Some(dt) + .and_then(|d| d.with_nanosecond(0)) + .and_then(|d| d.with_second(0)) + .and_then(|d| d.with_minute(0)) +} + +#[inline] +fn trunc_date_to_minute<T: Timelike>(dt: T) -> Option<T> { + Some(dt) + .and_then(|d| d.with_nanosecond(0)) + .and_then(|d| d.with_second(0)) +} + +#[inline] +fn trunc_date_to_second<T: Timelike>(dt: T) -> Option<T> { + Some(dt).and_then(|d| d.with_nanosecond(0)) +} + +#[inline] +fn trunc_date_to_ms<T: Timelike>(dt: T) -> Option<T> { + Some(dt).and_then(|d| d.with_nanosecond(1_000_000 * (d.nanosecond() / 1_000_000))) +} + +#[inline] +fn trunc_date_to_microsec<T: Timelike>(dt: T) -> Option<T> { + Some(dt).and_then(|d| d.with_nanosecond(1_000 * (d.nanosecond() / 1_000))) +} + +pub fn date_trunc_dyn(array: &dyn Array, format: String) -> Result<ArrayRef, ExpressionError> { + match array.data_type().clone() { + DataType::Dictionary(_, _) => { + downcast_dictionary_array!( + array => { + let truncated_values = date_trunc_dyn(array.values(), format)?; + Ok(Arc::new(array.with_values(truncated_values))) + } + dt => return_compute_error_with!("date_trunc does not support", dt), + ) + } + _ => { + downcast_temporal_array!( + array => { + date_trunc(array, format) + .map(|a| Arc::new(a) as ArrayRef) + } + dt => return_compute_error_with!("date_trunc does not support", dt), + ) + } + } +} + +pub fn date_trunc<T>( + array: &PrimitiveArray<T>, + format: String, +) -> Result<Date32Array, ExpressionError> +where + T: ArrowTemporalType + ArrowNumericType, + i64: From<T::Native>, +{ + let builder = Date32Builder::with_capacity(array.len()); + let iter = ArrayIter::new(array); + match array.data_type() { + DataType::Date32 => match format.to_uppercase().as_str() { + "YEAR" | "YYYY" | "YY" => Ok(as_datetime_with_op::<&PrimitiveArray<T>, T, _>( + iter, + builder, + |dt| as_days_from_unix_epoch(trunc_date_to_year(dt)), + )), + "QUARTER" => Ok(as_datetime_with_op::<&PrimitiveArray<T>, T, _>( + iter, + builder, + |dt| as_days_from_unix_epoch(trunc_date_to_quarter(dt)), + )), + "MONTH" | "MON" | "MM" => Ok(as_datetime_with_op::<&PrimitiveArray<T>, T, _>( + iter, + builder, + |dt| as_days_from_unix_epoch(trunc_date_to_month(dt)), + )), + "WEEK" => Ok(as_datetime_with_op::<&PrimitiveArray<T>, T, _>( + iter, + builder, + |dt| as_days_from_unix_epoch(trunc_date_to_week(dt)), + )), + _ => Err(ExpressionError::ArrowError(format!( + "Unsupported format: {:?} for function 'date_trunc'", + format + ))), + }, + dt => return_compute_error_with!( + "Unsupported input type '{:?}' for function 'date_trunc'", + dt + ), + } +} + +pub fn timestamp_trunc_dyn(array: &dyn Array, format: String) -> Result<ArrayRef, ExpressionError> { + match array.data_type().clone() { + DataType::Dictionary(_, _) => { + downcast_dictionary_array!( + array => { + let truncated_values = timestamp_trunc_dyn(array.values(), format)?; + Ok(Arc::new(array.with_values(truncated_values))) + } + dt => return_compute_error_with!("timestamp_trunc does not support", dt), + ) + } + _ => { + downcast_temporal_array!( + array => { + timestamp_trunc(array, format) + .map(|a| Arc::new(a) as ArrayRef) + } + dt => return_compute_error_with!("timestamp_trunc does not support", dt), + ) + } + } +} + +pub fn timestamp_trunc<T>( + array: &PrimitiveArray<T>, + format: String, +) -> Result<TimestampMicrosecondArray, ExpressionError> +where + T: ArrowTemporalType + ArrowNumericType, + i64: From<T::Native>, +{ + let builder = TimestampMicrosecondBuilder::with_capacity(array.len()); + let iter = ArrayIter::new(array); + match array.data_type() { + DataType::Timestamp(TimeUnit::Microsecond, Some(tz)) => { + match format.to_uppercase().as_str() { + "YEAR" | "YYYY" | "YY" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_year(dt)) + }) + } + "QUARTER" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_quarter(dt)) + }) + } + "MONTH" | "MON" | "MM" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_month(dt)) + }) + } + "WEEK" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_week(dt)) + }) + } + "DAY" | "DD" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_day(dt)) + }) + } + "HOUR" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_hour(dt)) + }) + } + "MINUTE" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_minute(dt)) + }) + } + "SECOND" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_second(dt)) + }) + } + "MILLISECOND" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_ms(dt)) + }) + } + "MICROSECOND" => { + as_timestamp_tz_with_op::<&PrimitiveArray<T>, T, _>(iter, builder, tz, |dt| { + as_micros_from_unix_epoch_utc(trunc_date_to_microsec(dt)) + }) + } + _ => Err(ExpressionError::ArrowError(format!( + "Unsupported format: {:?} for function 'timestamp_trunc'", + format + ))), + } + } + dt => return_compute_error_with!( + "Unsupported input type '{:?}' for function 'timestamp_trunc'", + dt + ), + } +} + +#[cfg(test)] +mod tests { + use crate::execution::kernels::temporal::{date_trunc, timestamp_trunc}; + use arrow_array::{Date32Array, TimestampMicrosecondArray}; + + #[test] + fn test_date_trunc() { + let size = 1000; + let mut vec: Vec<i32> = Vec::with_capacity(size); + for i in 0..size { + vec.push(i as i32); + } + let array = Date32Array::from(vec); + for fmt in [ + "YEAR", "YYYY", "YY", "QUARTER", "MONTH", "MON", "MM", "WEEK", + ] { + match date_trunc(&array, fmt.to_string()) { + Ok(a) => { + for i in 0..size { + assert!(array.values().get(i) >= a.values().get(i)) + } + } + _ => assert!(false), + } + } + } + + #[test] + fn test_timestamp_trunc() { + let size = 1000; + let mut vec: Vec<i64> = Vec::with_capacity(size); + for i in 0..size { + vec.push(i as i64); + } + let array = TimestampMicrosecondArray::from(vec).with_timezone_utc(); + for fmt in [ + "YEAR", + "YYYY", + "YY", + "QUARTER", + "MONTH", + "MON", + "MM", + "WEEK", + "DAY", + "DD", + "HOUR", + "MINUTE", + "SECOND", + "MILLISECOND", + "MICROSECOND", + ] { + match timestamp_trunc(&array, fmt.to_string()) { + Ok(a) => { + for i in 0..size { + assert!(array.values().get(i) >= a.values().get(i)) + } + } + _ => assert!(false), + } + } + } +} diff --git a/core/src/execution/metrics/mod.rs b/core/src/execution/metrics/mod.rs new file mode 100644 index 0000000000..a777b6863c --- /dev/null +++ b/core/src/execution/metrics/mod.rs @@ -0,0 +1,18 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 utils; diff --git a/core/src/execution/metrics/utils.rs b/core/src/execution/metrics/utils.rs new file mode 100644 index 0000000000..eb36a55623 --- /dev/null +++ b/core/src/execution/metrics/utils.rs @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 crate::{ + errors::CometError, + jvm_bridge::{jni_call, jni_new_string}, +}; +use datafusion::physical_plan::ExecutionPlan; +use jni::{objects::JObject, JNIEnv}; +use std::sync::Arc; + +/// Updates the metrics of a CometMetricNode. This function is called recursively to +/// update the metrics of all the children nodes. The metrics are pulled from the +/// DataFusion execution plan and pushed to the Java side through JNI. +pub fn update_comet_metric( + env: &JNIEnv, + metric_node: JObject, + execution_plan: &Arc<dyn ExecutionPlan>, +) -> Result<(), CometError> { + update_metrics( + env, + metric_node, + &execution_plan + .metrics() + .unwrap_or_default() + .iter() + .map(|m| m.value()) + .map(|m| (m.name(), m.as_usize() as i64)) + .collect::<Vec<_>>(), + )?; + + for (i, child_plan) in execution_plan.children().iter().enumerate() { + let child_metric_node: JObject = jni_call!(env, + comet_metric_node(metric_node).get_child_node(i as i32) -> JObject + )?; + if child_metric_node.is_null() { + continue; + } + update_comet_metric(env, child_metric_node, child_plan)?; + } + Ok(()) +} + +#[inline] +fn update_metrics( + env: &JNIEnv, + metric_node: JObject, + metric_values: &[(&str, i64)], +) -> Result<(), CometError> { + for &(name, value) in metric_values { + let jname = jni_new_string!(env, &name)?; + jni_call!(env, comet_metric_node(metric_node).add(jname, value) -> ())?; + } + Ok(()) +} diff --git a/core/src/execution/mod.rs b/core/src/execution/mod.rs new file mode 100644 index 0000000000..b0c60cc523 --- /dev/null +++ b/core/src/execution/mod.rs @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! PoC of vectorization execution through JNI to Rust. +pub mod datafusion; +pub mod jni_api; + +pub mod kernels; // for benchmarking + +mod metrics; +pub mod operators; +pub mod serde; +mod timezone; +pub(crate) mod utils; + +// Include generated modules from .proto files. +#[allow(missing_docs)] +pub mod spark_expression { + include!(concat!("generated", "/spark.spark_expression.rs")); +} + +// Include generated modules from .proto files. +#[allow(missing_docs)] +pub mod spark_partitioning { + include!(concat!("generated", "/spark.spark_partitioning.rs")); +} + +// Include generated modules from .proto files. +#[allow(missing_docs)] +pub mod spark_operator { + include!(concat!("generated", "/spark.spark_operator.rs")); +} + +#[cfg(test)] +mod tests { + #[test] + fn it_works() { + let result = 2 + 2; + assert_eq!(result, 4); + } +} diff --git a/core/src/execution/operators/copy.rs b/core/src/execution/operators/copy.rs new file mode 100644 index 0000000000..394c1ca80d --- /dev/null +++ b/core/src/execution/operators/copy.rs @@ -0,0 +1,163 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; + +use futures::{Stream, StreamExt}; + +use arrow_array::{ArrayRef, RecordBatch}; +use arrow_schema::{DataType, Field, FieldRef, Schema, SchemaRef}; + +use datafusion::{execution::TaskContext, physical_expr::*, physical_plan::*}; +use datafusion_common::{DataFusionError, Result as DataFusionResult}; + +use super::copy_or_cast_array; + +/// An utility execution node which makes deep copies of input batches. +/// +/// In certain scenarios like sort, DF execution nodes only make shallow copy of input batches. +/// This could cause issues for Comet, since we re-use column vectors across different batches. +/// For those scenarios, this can be used as an adapter node. +#[derive(Debug)] +pub struct CopyExec { + input: Arc<dyn ExecutionPlan>, + schema: SchemaRef, +} + +impl CopyExec { + pub fn new(input: Arc<dyn ExecutionPlan>) -> Self { + let fields: Vec<Field> = input + .schema() + .fields + .iter() + .map(|f: &FieldRef| match f.data_type() { + DataType::Dictionary(_, value_type) => { + Field::new(f.name(), value_type.as_ref().clone(), f.is_nullable()) + } + _ => f.as_ref().clone(), + }) + .collect(); + + let schema = Arc::new(Schema::new(fields)); + + Self { input, schema } + } +} + +impl DisplayAs for CopyExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "CopyExec") + } + } + } +} + +impl ExecutionPlan for CopyExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_partitioning(&self) -> Partitioning { + self.input.output_partitioning() + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + self.input.output_ordering() + } + + fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> { + self.input.children() + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn ExecutionPlan>>, + ) -> DataFusionResult<Arc<dyn ExecutionPlan>> { + let input = self.input.clone(); + let new_input = input.with_new_children(children)?; + Ok(Arc::new(CopyExec { + input: new_input, + schema: self.schema.clone(), + })) + } + + fn execute( + &self, + partition: usize, + context: Arc<TaskContext>, + ) -> DataFusionResult<SendableRecordBatchStream> { + let child_stream = self.input.execute(partition, context)?; + Ok(Box::pin(CopyStream::new(self.schema(), child_stream))) + } + + fn statistics(&self) -> DataFusionResult<Statistics> { + self.input.statistics() + } +} + +struct CopyStream { + schema: SchemaRef, + child_stream: SendableRecordBatchStream, +} + +impl CopyStream { + fn new(schema: SchemaRef, child_stream: SendableRecordBatchStream) -> Self { + Self { + schema, + child_stream, + } + } + + // TODO: replace copy_or_cast_array with copy_array if upstream sort kernel fixes + // dictionary array sorting issue. + fn copy(&self, batch: RecordBatch) -> DataFusionResult<RecordBatch> { + let vectors = batch + .columns() + .iter() + .map(|v| copy_or_cast_array(v)) + .collect::<Result<Vec<ArrayRef>, _>>()?; + RecordBatch::try_new(self.schema.clone(), vectors).map_err(DataFusionError::ArrowError) + } +} + +impl Stream for CopyStream { + type Item = DataFusionResult<RecordBatch>; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> { + self.child_stream.poll_next_unpin(cx).map(|x| match x { + Some(Ok(batch)) => Some(self.copy(batch)), + other => other, + }) + } +} + +impl RecordBatchStream for CopyStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} diff --git a/core/src/execution/operators/mod.rs b/core/src/execution/operators/mod.rs new file mode 100644 index 0000000000..5d05fdb8db --- /dev/null +++ b/core/src/execution/operators/mod.rs @@ -0,0 +1,104 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Operators + +use arrow::{ + array::{make_array, Array, ArrayRef, MutableArrayData}, + datatypes::DataType, + downcast_dictionary_array, +}; + +use arrow::compute::{cast_with_options, CastOptions}; +use arrow_schema::ArrowError; +use std::{fmt::Debug, sync::Arc}; + +mod scan; +pub use scan::*; + +mod copy; +pub use copy::*; + +/// Error returned during executing operators. +#[derive(thiserror::Error, Debug)] +pub enum ExecutionError { + /// Simple error + #[allow(dead_code)] + #[error("General execution error with reason {0}.")] + GeneralError(String), + + /// Error when deserializing an operator. + #[error("Fail to deserialize to native operator with reason {0}.")] + DeserializeError(String), + + /// Error when processing Arrow array. + #[error("Fail to process Arrow array with reason {0}.")] + ArrowError(String), + + /// DataFusion error + #[error("Error from DataFusion {0}.")] + DataFusionError(String), +} + +/// Copy an Arrow Array +pub fn copy_array(array: &dyn Array) -> ArrayRef { + let capacity = array.len(); + let data = array.to_data(); + + let mut mutable = MutableArrayData::new(vec![&data], false, capacity); + + mutable.extend(0, 0, capacity); + + if matches!(array.data_type(), DataType::Dictionary(_, _)) { + let copied_dict = make_array(mutable.freeze()); + let ref_copied_dict = &copied_dict; + + downcast_dictionary_array!( + ref_copied_dict => { + // Copying dictionary value array + let values = ref_copied_dict.values(); + let data = values.to_data(); + + let mut mutable = MutableArrayData::new(vec![&data], false, values.len()); + mutable.extend(0, 0, values.len()); + + let copied_dict = ref_copied_dict.with_values(make_array(mutable.freeze())); + Arc::new(copied_dict) + } + t => unreachable!("Should not reach here: {}", t) + ) + } else { + make_array(mutable.freeze()) + } +} + +/// Copy an Arrow Array or cast to primitive type if it is a dictionary array. +/// This is used for `CopyExec` to copy/cast the input array. If the input array +/// is a dictionary array, we will cast the dictionary array to primitive type +/// (i.e., unpack the dictionary array) and copy the primitive array. If the input +/// array is a primitive array, we simply copy the array. +pub fn copy_or_cast_array(array: &dyn Array) -> Result<ArrayRef, ArrowError> { + match array.data_type() { + DataType::Dictionary(_, value_type) => { + let options = CastOptions::default(); + let casted = cast_with_options(array, value_type.as_ref(), &options); + + casted.and_then(|a| copy_or_cast_array(a.as_ref())) + } + _ => Ok(copy_array(array)), + } +} diff --git a/core/src/execution/operators/scan.rs b/core/src/execution/operators/scan.rs new file mode 100644 index 0000000000..09afc5f030 --- /dev/null +++ b/core/src/execution/operators/scan.rs @@ -0,0 +1,271 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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, + pin::Pin, + sync::{Arc, Mutex}, + task::{Context, Poll}, +}; + +use futures::Stream; +use itertools::Itertools; + +use arrow::compute::{cast_with_options, CastOptions}; +use arrow_array::{ArrayRef, RecordBatch, RecordBatchOptions}; +use arrow_schema::{DataType, Field, Schema, SchemaRef}; + +use datafusion::{ + execution::TaskContext, + physical_expr::*, + physical_plan::{ExecutionPlan, *}, +}; +use datafusion_common::{DataFusionError, Result as DataFusionResult}; + +#[derive(Debug, Clone)] +pub struct ScanExec { + pub batch: Arc<Mutex<Option<InputBatch>>>, + pub data_types: Vec<DataType>, +} + +impl ScanExec { + pub fn new(batch: InputBatch, data_types: Vec<DataType>) -> Self { + Self { + batch: Arc::new(Mutex::new(Some(batch))), + data_types, + } + } + + /// Feeds input batch into this `Scan`. + pub fn set_input_batch(&mut self, input: InputBatch) { + *self.batch.try_lock().unwrap() = Some(input); + } + + /// Checks if the input data type `dt` is a dictionary type with primitive value type. + /// If so, unpacks it and returns the primitive value type. + /// + /// Otherwise, this returns the original data type. + /// + /// This is necessary since DataFusion doesn't handle dictionary array with values + /// being primitive type. + /// + /// TODO: revisit this once DF has imprved its dictionary type support. Ideally we shouldn't + /// do this in Comet but rather let DF to handle it for us. + fn unpack_dictionary_type(dt: &DataType) -> DataType { + if let DataType::Dictionary(_, vt) = dt { + if !matches!(vt.as_ref(), DataType::Utf8 | DataType::Binary) { + return vt.as_ref().clone(); + } + } + + dt.clone() + } +} + +impl ExecutionPlan for ScanExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + // `unwrap` is safe because `schema` is only called during converting + // Spark plan to DataFusion plan. At the moment, `batch` is not EOF. + let binding = self.batch.try_lock().unwrap(); + let input_batch = binding.as_ref().unwrap(); + + let fields = match input_batch { + // Note that if `columns` is empty, we'll get an empty schema + InputBatch::Batch(columns, _) => { + columns + .iter() + .enumerate() + .map(|(idx, c)| { + let datatype = Self::unpack_dictionary_type(c.data_type()); + // We don't use the field name. Put a placeholder. + if matches!(datatype, DataType::Dictionary(_, _)) { + Field::new_dict( + format!("col_{}", idx), + datatype, + true, + idx as i64, + false, + ) + } else { + Field::new(format!("col_{}", idx), datatype, true) + } + }) + .collect::<Vec<Field>>() + } + _ => self + .data_types + .iter() + .enumerate() + .map(|(idx, dt)| Field::new(format!("col_{}", idx), dt.clone(), true)) + .collect(), + }; + + Arc::new(Schema::new(fields)) + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(1) + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + None + } + + fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> { + vec![] + } + + fn with_new_children( + self: Arc<Self>, + _: Vec<Arc<dyn ExecutionPlan>>, + ) -> datafusion::common::Result<Arc<dyn ExecutionPlan>> { + Ok(self) + } + + fn execute( + &self, + _: usize, + _: Arc<TaskContext>, + ) -> datafusion::common::Result<SendableRecordBatchStream> { + Ok(Box::pin(ScanStream::new(self.clone(), self.schema()))) + } +} + +impl DisplayAs for ScanExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "ScanExec")?; + let fields: Vec<String> = self + .data_types + .iter() + .enumerate() + .map(|(idx, dt)| format!("col_{idx:}: {dt:}")) + .collect(); + write!(f, ": schema=[{}]", fields.join(", "))?; + } + } + Ok(()) + } +} + +/// A async-stream feeds input batch from `Scan` into DataFusion physical plan. +struct ScanStream { + /// The `Scan` node producing input batches + scan: ScanExec, + /// Schema representing the data + schema: SchemaRef, +} + +impl ScanStream { + pub fn new(scan: ScanExec, schema: SchemaRef) -> Self { + Self { scan, schema } + } + + fn build_record_batch( + &self, + columns: &[ArrayRef], + num_rows: usize, + ) -> DataFusionResult<RecordBatch, DataFusionError> { + let schema_fields = self.schema.fields(); + assert_eq!(columns.len(), schema_fields.len()); + + // Cast if necessary + let cast_options = CastOptions::default(); + let new_columns: Vec<ArrayRef> = columns + .iter() + .zip(schema_fields.iter()) + .map(|(column, f)| { + if column.data_type() != f.data_type() { + cast_with_options(column, f.data_type(), &cast_options).unwrap() + } else { + column.clone() + } + }) + .collect(); + + let options = RecordBatchOptions::new().with_row_count(Some(num_rows)); + RecordBatch::try_new_with_options(self.schema.clone(), new_columns, &options) + .map_err(DataFusionError::ArrowError) + } +} + +impl Stream for ScanStream { + type Item = DataFusionResult<RecordBatch>; + + fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll<Option<Self::Item>> { + let mut scan_batch = self.scan.batch.try_lock().unwrap(); + let input_batch = &*scan_batch; + let result = match input_batch { + // Input batch is not ready. + None => Poll::Pending, + Some(batch) => match batch { + InputBatch::EOF => Poll::Ready(None), + InputBatch::Batch(columns, num_rows) => { + Poll::Ready(Some(self.build_record_batch(columns, *num_rows))) + } + }, + }; + + // Reset the current input batch so it won't be processed again + *scan_batch = None; + result + } +} + +impl RecordBatchStream for ScanStream { + /// Get the schema + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} + +#[derive(Clone, Debug)] +pub enum InputBatch { + /// The end of input batches. + EOF, + + /// A normal batch with columns and number of rows. + /// It is possible to have zero-column batch with non-zero number of rows, + /// i.e. reading empty schema from scan. + Batch(Vec<ArrayRef>, usize), +} + +impl InputBatch { + /// Constructs a `InputBatch` from columns and optional number of rows. + /// If `num_rows` is none, this function will calculate it from given + /// columns. + pub fn new(columns: Vec<ArrayRef>, num_rows: Option<usize>) -> Self { + let num_rows = num_rows.unwrap_or_else(|| { + let lengths = columns.iter().map(|a| a.len()).unique().collect::<Vec<_>>(); + assert!(lengths.len() <= 1, "Columns have different lengths."); + + if lengths.is_empty() { + // All are scalar values + 1 + } else { + lengths[0] + } + }); + + InputBatch::Batch(columns, num_rows) + } +} diff --git a/core/src/execution/proto/expr.proto b/core/src/execution/proto/expr.proto new file mode 100644 index 0000000000..7311c12453 --- /dev/null +++ b/core/src/execution/proto/expr.proto @@ -0,0 +1,454 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + + + +syntax = "proto3"; + +package spark.spark_expression; + +option java_package = "org.apache.comet.serde"; + +// The basic message representing a Spark expression. +message Expr { + oneof expr_struct { + Literal literal = 2; + BoundReference bound = 3; + Add add = 4; + Subtract subtract = 5; + Multiply multiply = 6; + Divide divide = 7; + Cast cast = 8; + Equal eq = 9; + NotEqual neq = 10; + GreaterThan gt = 11; + GreaterThanEqual gt_eq = 12; + LessThan lt = 13; + LessThanEqual lt_eq = 14; + IsNull is_null = 15; + IsNotNull is_not_null = 16; + And and = 17; + Or or = 18; + SortOrder sort_order = 19; + Substring substring = 20; + StringSpace string_space = 21; + Hour hour = 22; + Minute minute = 23; + Second second = 24; + CheckOverflow check_overflow = 25; + Like like = 26; + StartsWith startsWith = 27; + EndsWith endsWith = 28; + Contains contains = 29; + // RLike rlike = 30; + ScalarFunc scalarFunc = 31; + EqualNullSafe eqNullSafe = 32; + NotEqualNullSafe neqNullSafe = 33; + BitwiseAnd bitwiseAnd = 34; + BitwiseOr bitwiseOr = 35; + BitwiseXor bitwiseXor = 36; + Remainder remainder = 37; + CaseWhen caseWhen = 38; + In in = 39; + Not not = 40; + Negative negative = 41; + BitwiseShiftRight bitwiseShiftRight = 42; + BitwiseShiftLeft bitwiseShiftLeft = 43; + IfExpr if = 44; + NormalizeNaNAndZero normalize_nan_and_zero = 45; + TruncDate truncDate = 46; + TruncTimestamp truncTimestamp = 47; + BitwiseNot bitwiseNot = 48; + Abs abs = 49; + Subquery subquery = 50; + } +} + +message AggExpr { + oneof expr_struct { + Count count = 2; + Sum sum = 3; + Min min = 4; + Max max = 5; + Avg avg = 6; + } +} + +message Count { + repeated Expr children = 1; +} + +message Sum { + Expr child = 1; + DataType datatype = 2; + bool fail_on_error = 3; +} + +message Min { + Expr child = 1; + DataType datatype = 2; +} + +message Max { + Expr child = 1; + DataType datatype = 2; +} + +message Avg { + Expr child = 1; + DataType datatype = 2; + DataType sum_datatype = 3; + bool fail_on_error = 4; // currently unused (useful for deciding Ansi vs Legacy mode) +} + +message Literal { + oneof value { + bool bool_val = 1; + // Protobuf doesn't provide int8 and int16, we put them into int32 and convert + // to int8 and int16 when deserializing. + int32 byte_val = 2; + int32 short_val = 3; + int32 int_val = 4; + int64 long_val = 5; + float float_val = 6; + double double_val = 7; + string string_val = 8; + bytes bytes_val = 9; + bytes decimal_val = 10; + } + + DataType datatype = 11; + bool is_null = 12; +} + +message Add { + Expr left = 1; + Expr right = 2; + bool fail_on_error = 3; + DataType return_type = 4; +} + +message Subtract { + Expr left = 1; + Expr right = 2; + bool fail_on_error = 3; + DataType return_type = 4; +} + +message Multiply { + Expr left = 1; + Expr right = 2; + bool fail_on_error = 3; + DataType return_type = 4; +} + +message Divide { + Expr left = 1; + Expr right = 2; + bool fail_on_error = 3; + DataType return_type = 4; +} + +message Remainder { + Expr left = 1; + Expr right = 2; + bool fail_on_error = 3; + DataType return_type = 4; +} + +message Cast { + Expr child = 1; + DataType datatype = 2; + string timezone = 3; +} + +message Equal { + Expr left = 1; + Expr right = 2; +} + +message NotEqual { + Expr left = 1; + Expr right = 2; +} + +message EqualNullSafe { + Expr left = 1; + Expr right = 2; +} + +message NotEqualNullSafe { + Expr left = 1; + Expr right = 2; +} + +message GreaterThan { + Expr left = 1; + Expr right = 2; +} + +message GreaterThanEqual { + Expr left = 1; + Expr right = 2; +} + +message LessThan { + Expr left = 1; + Expr right = 2; +} + +message LessThanEqual { + Expr left = 1; + Expr right = 2; +} + +message And { + Expr left = 1; + Expr right = 2; +} + +message Or { + Expr left = 1; + Expr right = 2; +} + +message IsNull { + Expr child = 1; +} + +message IsNotNull { + Expr child = 1; +} + +// Bound to a particular vector array in input batch. +message BoundReference { + int32 index = 1; + DataType datatype = 2; +} + +message SortOrder { + Expr child = 1; + SortDirection direction = 2; + NullOrdering null_ordering = 3; +} + +message Substring { + Expr child = 1; + int32 start = 2; + int32 len = 3; +} + +message StringSpace { + Expr child = 1; +} + +message Hour { + Expr child = 1; + string timezone = 2; +} + +message Minute { + Expr child = 1; + string timezone = 2; +} + +message Second { + Expr child = 1; + string timezone = 2; +} + +message CheckOverflow { + Expr child = 1; + DataType datatype = 2; + bool fail_on_error = 3; +} + +message Like { + Expr left = 1; + Expr right = 2; +} + +// message RLike { +// Expr left = 1; +// Expr right = 2; +// } + +message StartsWith { + Expr left = 1; + Expr right = 2; +} + +message EndsWith { + Expr left = 1; + Expr right = 2; +} + +message Contains { + Expr left = 1; + Expr right = 2; +} + +message ScalarFunc { + string func = 1; + repeated Expr args = 2; + DataType return_type = 3; +} + +message BitwiseAnd { + Expr left = 1; + Expr right = 2; +} + +message BitwiseOr { + Expr left = 1; + Expr right = 2; +} + +message BitwiseXor { + Expr left = 1; + Expr right = 2; +} + +message BitwiseShiftRight { + Expr left = 1; + Expr right = 2; +} + +message BitwiseShiftLeft { + Expr left = 1; + Expr right = 2; +} + +message CaseWhen { + // The expr field is added to be consistent with CaseExpr definition in DataFusion. + // This field is not really used. When constructing a CaseExpr, this expr field + // is always set to None. The reason that we always set this expr field to None + // is because Spark parser converts the expr to a EqualTo conditions. After the + // conversion, we don't see this expr any more so it's always None. + Expr expr = 1; + repeated Expr when = 2; + repeated Expr then = 3; + Expr else_expr = 4; +} + +message In { + Expr in_value = 1; + repeated Expr lists = 2; + bool negated = 3; +} + +message NormalizeNaNAndZero { + Expr child = 1; + DataType datatype = 2; +} + +message Not { + Expr child = 1; +} + +message Negative { + Expr child = 1; +} + +message IfExpr { + Expr if_expr = 1; + Expr true_expr = 2; + Expr false_expr = 3; +} + +message TruncDate { + Expr child = 1; + Expr format = 2; +} + +message TruncTimestamp { + Expr format = 1; + Expr child = 2; + string timezone = 3; +} + +message BitwiseNot { + Expr child = 1; +} + +message Abs { + Expr child = 1; +} + +message Subquery { + int64 id = 1; + DataType datatype = 2; +} + +enum SortDirection { + Ascending = 0; + Descending = 1; +} + +enum NullOrdering { + NullsFirst = 0; + NullsLast = 1; +} + +message DataType { + enum DataTypeId { + BOOL = 0; + INT8 = 1; + INT16 = 2; + INT32 = 3; + INT64 = 4; + FLOAT = 5; + DOUBLE = 6; + STRING = 7; + BYTES = 8; + TIMESTAMP = 9; + DECIMAL = 10; + TIMESTAMP_NTZ = 11; + DATE = 12; + NULL = 13; + LIST = 14; + STRUCT = 15; + } + DataTypeId type_id = 1; + + message DataTypeInfo { + oneof datatype_struct { + DecimalInfo decimal = 2; + ListInfo list = 3; + StructInfo struct = 4; + } + } + + message DecimalInfo { + int32 precision = 1; + int32 scale = 2; + } + + message ListInfo { + DataType element_type = 1; + bool contains_null = 2; + } + + message StructInfo { + repeated string field_names = 1; + repeated DataType field_datatypes = 2; + repeated bool field_nullable = 3; + } + + DataTypeInfo type_info = 2; +} + diff --git a/core/src/execution/proto/operator.proto b/core/src/execution/proto/operator.proto new file mode 100644 index 0000000000..f4f7853966 --- /dev/null +++ b/core/src/execution/proto/operator.proto @@ -0,0 +1,82 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + + + +syntax = "proto3"; + +package spark.spark_operator; + +import "expr.proto"; +import "partitioning.proto"; + +option java_package = "org.apache.comet.serde"; + +// The basic message representing a Spark operator. +message Operator { + // The child operators of this + repeated Operator children = 1; + + oneof op_struct { + Scan scan = 100; + Projection projection = 101; + Filter filter = 102; + Sort sort = 103; + HashAggregate hash_agg = 104; + Limit limit = 105; + Expand expand = 106; + } +} + +message Scan { + repeated spark.spark_expression.DataType fields = 1; +} + +message Projection { + repeated spark.spark_expression.Expr project_list = 1; +} + +message Filter { + spark.spark_expression.Expr predicate = 1; +} + +message Sort { + repeated spark.spark_expression.Expr sort_orders = 1; + optional int32 fetch = 3; +} + +message HashAggregate { + repeated spark.spark_expression.Expr grouping_exprs = 1; + repeated spark.spark_expression.AggExpr agg_exprs = 2; + repeated spark.spark_expression.Expr result_exprs = 3; + AggregateMode mode = 5; +} + +message Limit { + int32 limit = 1; + int32 offset = 2; +} + +enum AggregateMode { + Partial = 0; + Final = 1; +} + +message Expand { + repeated spark.spark_expression.Expr project_list = 1; + int32 num_expr_per_project = 3; +} diff --git a/core/src/execution/proto/partitioning.proto b/core/src/execution/proto/partitioning.proto new file mode 100644 index 0000000000..21bd505609 --- /dev/null +++ b/core/src/execution/proto/partitioning.proto @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + + + +syntax = "proto3"; + +package spark.spark_partitioning; + +import "expr.proto"; + +option java_package = "org.apache.comet.serde"; + +// The basic message representing a Spark partitioning. +message Partitioning { + oneof partitioning_struct { + HashRepartition hash_partition = 2; + SinglePartition single_partition = 3; + } +} + +message HashRepartition { + repeated spark.spark_expression.Expr hash_expression = 1; + int32 num_partitions = 2; +} + +message SinglePartition { +} diff --git a/core/src/execution/serde.rs b/core/src/execution/serde.rs new file mode 100644 index 0000000000..2f38b4527f --- /dev/null +++ b/core/src/execution/serde.rs @@ -0,0 +1,147 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Ser/De for expression/operators. + +use super::{ + operators::ExecutionError, spark_expression, spark_expression::DataType, spark_operator, +}; +use crate::{ + errors::ExpressionError, + execution::spark_expression::data_type::{ + data_type_info::DatatypeStruct, + DataTypeId, + DataTypeId::{Bool, Bytes, Decimal, Double, Float, Int16, Int32, Int64, Int8, String}, + }, +}; +use arrow::datatypes::{DataType as ArrowDataType, TimeUnit}; +use arrow_schema::Field; +use prost::Message; +use std::{io::Cursor, sync::Arc}; + +impl From<prost::DecodeError> for ExpressionError { + fn from(error: prost::DecodeError) -> ExpressionError { + ExpressionError::Deserialize(error.to_string()) + } +} + +impl From<prost::DecodeError> for ExecutionError { + fn from(error: prost::DecodeError) -> ExecutionError { + ExecutionError::DeserializeError(error.to_string()) + } +} + +/// Deserialize bytes to protobuf type of expression +pub fn deserialize_expr(buf: &[u8]) -> Result<spark_expression::Expr, ExpressionError> { + match spark_expression::Expr::decode(&mut Cursor::new(buf)) { + Ok(e) => Ok(e), + Err(err) => Err(ExpressionError::from(err)), + } +} + +/// Deserialize bytes to protobuf type of operator +pub fn deserialize_op(buf: &[u8]) -> Result<spark_operator::Operator, ExecutionError> { + match spark_operator::Operator::decode(&mut Cursor::new(buf)) { + Ok(e) => Ok(e), + Err(err) => Err(ExecutionError::from(err)), + } +} + +/// Deserialize bytes to protobuf type of data type +pub fn deserialize_data_type(buf: &[u8]) -> Result<spark_expression::DataType, ExecutionError> { + match spark_expression::DataType::decode(&mut Cursor::new(buf)) { + Ok(e) => Ok(e), + Err(err) => Err(ExecutionError::from(err)), + } +} + +/// Converts Protobuf data type to Arrow data type. +pub fn to_arrow_datatype(dt_value: &DataType) -> ArrowDataType { + match DataTypeId::try_from(dt_value.type_id).unwrap() { + Bool => ArrowDataType::Boolean, + Int8 => ArrowDataType::Int8, + Int16 => ArrowDataType::Int16, + Int32 => ArrowDataType::Int32, + Int64 => ArrowDataType::Int64, + Float => ArrowDataType::Float32, + Double => ArrowDataType::Float64, + String => ArrowDataType::Utf8, + Bytes => ArrowDataType::Binary, + Decimal => match dt_value + .type_info + .as_ref() + .unwrap() + .datatype_struct + .as_ref() + .unwrap() + { + DatatypeStruct::Decimal(info) => { + ArrowDataType::Decimal128(info.precision as u8, info.scale as i8) + } + _ => unreachable!(), + }, + DataTypeId::Timestamp => { + ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".to_string().into())) + } + DataTypeId::TimestampNtz => ArrowDataType::Timestamp(TimeUnit::Microsecond, None), + DataTypeId::Date => ArrowDataType::Date32, + DataTypeId::Null => ArrowDataType::Null, + DataTypeId::List => match dt_value + .type_info + .as_ref() + .unwrap() + .datatype_struct + .as_ref() + .unwrap() + { + DatatypeStruct::List(info) => { + let field = Field::new( + "item", + to_arrow_datatype(info.element_type.as_ref().unwrap()), + info.contains_null, + ); + ArrowDataType::List(Arc::new(field)) + } + _ => unreachable!(), + }, + DataTypeId::Struct => match dt_value + .type_info + .as_ref() + .unwrap() + .datatype_struct + .as_ref() + .unwrap() + { + DatatypeStruct::Struct(info) => { + let fields = info + .field_names + .iter() + .enumerate() + .map(|(idx, name)| { + Field::new( + name, + to_arrow_datatype(&info.field_datatypes[idx]), + info.field_nullable[idx], + ) + }) + .collect(); + ArrowDataType::Struct(fields) + } + _ => unreachable!(), + }, + } +} diff --git a/core/src/execution/timezone.rs b/core/src/execution/timezone.rs new file mode 100644 index 0000000000..7aad386aa9 --- /dev/null +++ b/core/src/execution/timezone.rs @@ -0,0 +1,143 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +/// Utils for timezone. This is basically from arrow-array::timezone (private). +use arrow_schema::ArrowError; +use chrono::{ + format::{parse, Parsed, StrftimeItems}, + offset::TimeZone, + FixedOffset, LocalResult, NaiveDate, NaiveDateTime, Offset, +}; +use std::str::FromStr; + +/// Parses a fixed offset of the form "+09:00" +fn parse_fixed_offset(tz: &str) -> Result<FixedOffset, ArrowError> { + let mut parsed = Parsed::new(); + + if let Ok(fixed_offset) = + parse(&mut parsed, tz, StrftimeItems::new("%:z")).and_then(|_| parsed.to_fixed_offset()) + { + return Ok(fixed_offset); + } + + if let Ok(fixed_offset) = + parse(&mut parsed, tz, StrftimeItems::new("%#z")).and_then(|_| parsed.to_fixed_offset()) + { + return Ok(fixed_offset); + } + + Err(ArrowError::ParseError(format!( + "Invalid timezone \"{}\": Expected format [+-]XX:XX, [+-]XX, or [+-]XXXX", + tz + ))) +} + +/// An [`Offset`] for [`Tz`] +#[derive(Debug, Copy, Clone)] +pub struct TzOffset { + tz: Tz, + offset: FixedOffset, +} + +impl std::fmt::Display for TzOffset { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.offset.fmt(f) + } +} + +impl Offset for TzOffset { + fn fix(&self) -> FixedOffset { + self.offset + } +} + +/// An Arrow [`TimeZone`] +#[derive(Debug, Copy, Clone)] +pub struct Tz(TzInner); + +#[derive(Debug, Copy, Clone)] +enum TzInner { + Timezone(chrono_tz::Tz), + Offset(FixedOffset), +} + +impl FromStr for Tz { + type Err = ArrowError; + + fn from_str(tz: &str) -> Result<Self, Self::Err> { + if tz.starts_with('+') || tz.starts_with('-') { + Ok(Self(TzInner::Offset(parse_fixed_offset(tz)?))) + } else { + Ok(Self(TzInner::Timezone(tz.parse().map_err(|e| { + ArrowError::ParseError(format!("Invalid timezone \"{}\": {}", tz, e)) + })?))) + } + } +} + +macro_rules! tz { + ($s:ident, $tz:ident, $b:block) => { + match $s.0 { + TzInner::Timezone($tz) => $b, + TzInner::Offset($tz) => $b, + } + }; +} + +impl TimeZone for Tz { + type Offset = TzOffset; + + fn from_offset(offset: &Self::Offset) -> Self { + offset.tz + } + + fn offset_from_local_date(&self, local: &NaiveDate) -> LocalResult<Self::Offset> { + tz!(self, tz, { + tz.offset_from_local_date(local).map(|x| TzOffset { + tz: *self, + offset: x.fix(), + }) + }) + } + + fn offset_from_local_datetime(&self, local: &NaiveDateTime) -> LocalResult<Self::Offset> { + tz!(self, tz, { + tz.offset_from_local_datetime(local).map(|x| TzOffset { + tz: *self, + offset: x.fix(), + }) + }) + } + + fn offset_from_utc_date(&self, utc: &NaiveDate) -> Self::Offset { + tz!(self, tz, { + TzOffset { + tz: *self, + offset: tz.offset_from_utc_date(utc).fix(), + } + }) + } + + fn offset_from_utc_datetime(&self, utc: &NaiveDateTime) -> Self::Offset { + tz!(self, tz, { + TzOffset { + tz: *self, + offset: tz.offset_from_utc_datetime(utc).fix(), + } + }) + } +} diff --git a/core/src/execution/utils.rs b/core/src/execution/utils.rs new file mode 100644 index 0000000000..cb21391a23 --- /dev/null +++ b/core/src/execution/utils.rs @@ -0,0 +1,120 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::ArrayData, + error::ArrowError, + ffi::{from_ffi, FFI_ArrowArray, FFI_ArrowSchema}, +}; + +/// Utils for array vector, etc. +use crate::errors::ExpressionError; +use crate::execution::operators::ExecutionError; + +impl From<ArrowError> for ExecutionError { + fn from(error: ArrowError) -> ExecutionError { + ExecutionError::ArrowError(error.to_string()) + } +} + +impl From<ArrowError> for ExpressionError { + fn from(error: ArrowError) -> ExpressionError { + ExpressionError::ArrowError(error.to_string()) + } +} + +impl From<ExpressionError> for ArrowError { + fn from(error: ExpressionError) -> ArrowError { + ArrowError::ComputeError(error.to_string()) + } +} + +pub trait SparkArrowConvert { + /// Build Arrow Arrays from C data interface passed from Spark. + /// It accepts a tuple (ArrowArray address, ArrowSchema address). + fn from_spark(addresses: (i64, i64)) -> Result<Self, ExecutionError> + where + Self: Sized; + + /// Convert Arrow Arrays to C data interface. + /// It returns a tuple (ArrowArray address, ArrowSchema address). + fn to_spark(&self) -> Result<(i64, i64), ExecutionError>; +} + +impl SparkArrowConvert for ArrayData { + fn from_spark(addresses: (i64, i64)) -> Result<Self, ExecutionError> { + let (array_ptr, schema_ptr) = addresses; + + let array_ptr = array_ptr as *mut FFI_ArrowArray; + let schema_ptr = schema_ptr as *mut FFI_ArrowSchema; + + if array_ptr.is_null() || schema_ptr.is_null() { + return Err(ExecutionError::ArrowError( + "At least one of passed pointers is null".to_string(), + )); + }; + + // `ArrowArray` will convert raw pointers back to `Arc`. No worries + // about memory leak. + let mut ffi_array = unsafe { + let array_data = std::ptr::replace(array_ptr, FFI_ArrowArray::empty()); + let schema_data = std::ptr::replace(schema_ptr, FFI_ArrowSchema::empty()); + + from_ffi(array_data, &schema_data)? + }; + + // Align imported buffers from Java. + ffi_array.align_buffers(); + + Ok(ffi_array) + } + + /// Converts this ArrowData to pointers of Arrow C data interface. + /// Returned pointers are Arc-ed and should be freed manually. + #[allow(clippy::arc_with_non_send_sync)] + fn to_spark(&self) -> Result<(i64, i64), ExecutionError> { + let arrow_array = Arc::new(FFI_ArrowArray::new(self)); + let arrow_schema = Arc::new(FFI_ArrowSchema::try_from(self.data_type())?); + + let (array, schema) = (Arc::into_raw(arrow_array), Arc::into_raw(arrow_schema)); + + Ok((array as i64, schema as i64)) + } +} + +/// Converts a slice of bytes to i128. The bytes are serialized in big-endian order by +/// `BigInteger.toByteArray()` in Java. +pub fn bytes_to_i128(slice: &[u8]) -> i128 { + let mut bytes = [0; 16]; + let mut i = 0; + while i != 16 && i != slice.len() { + bytes[i] = slice[slice.len() - 1 - i]; + i += 1; + } + + // if the decimal is negative, we need to flip all the bits + if (slice[0] as i8) < 0 { + while i < 16 { + bytes[i] = !bytes[i]; + i += 1; + } + } + + i128::from_le_bytes(bytes) +} diff --git a/core/src/jvm_bridge/comet_exec.rs b/core/src/jvm_bridge/comet_exec.rs new file mode 100644 index 0000000000..e28fc080f6 --- /dev/null +++ b/core/src/jvm_bridge/comet_exec.rs @@ -0,0 +1,101 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 jni::{ + errors::Result as JniResult, + objects::{JClass, JStaticMethodID}, + signature::{JavaType, Primitive}, + JNIEnv, +}; + +use super::get_global_jclass; + +/// A struct that holds all the JNI methods and fields for JVM CometExec object. +pub struct CometExec<'a> { + pub class: JClass<'a>, + pub method_get_bool: JStaticMethodID<'a>, + pub method_get_bool_ret: JavaType, + pub method_get_byte: JStaticMethodID<'a>, + pub method_get_byte_ret: JavaType, + pub method_get_short: JStaticMethodID<'a>, + pub method_get_short_ret: JavaType, + pub method_get_int: JStaticMethodID<'a>, + pub method_get_int_ret: JavaType, + pub method_get_long: JStaticMethodID<'a>, + pub method_get_long_ret: JavaType, + pub method_get_float: JStaticMethodID<'a>, + pub method_get_float_ret: JavaType, + pub method_get_double: JStaticMethodID<'a>, + pub method_get_double_ret: JavaType, + pub method_get_decimal: JStaticMethodID<'a>, + pub method_get_decimal_ret: JavaType, + pub method_get_string: JStaticMethodID<'a>, + pub method_get_string_ret: JavaType, + pub method_get_binary: JStaticMethodID<'a>, + pub method_get_binary_ret: JavaType, + pub method_is_null: JStaticMethodID<'a>, + pub method_is_null_ret: JavaType, +} + +impl<'a> CometExec<'a> { + pub const JVM_CLASS: &'static str = "org/apache/spark/sql/comet/CometScalarSubquery"; + + pub fn new(env: &JNIEnv<'a>) -> JniResult<CometExec<'a>> { + // Get the global class reference + let class = get_global_jclass(env, Self::JVM_CLASS)?; + + Ok(CometExec { + class, + method_get_bool: env + .get_static_method_id(class, "getBoolean", "(JJ)Z") + .unwrap(), + method_get_bool_ret: JavaType::Primitive(Primitive::Boolean), + method_get_byte: env.get_static_method_id(class, "getByte", "(JJ)B").unwrap(), + method_get_byte_ret: JavaType::Primitive(Primitive::Byte), + method_get_short: env + .get_static_method_id(class, "getShort", "(JJ)S") + .unwrap(), + method_get_short_ret: JavaType::Primitive(Primitive::Short), + method_get_int: env.get_static_method_id(class, "getInt", "(JJ)I").unwrap(), + method_get_int_ret: JavaType::Primitive(Primitive::Int), + method_get_long: env.get_static_method_id(class, "getLong", "(JJ)J").unwrap(), + method_get_long_ret: JavaType::Primitive(Primitive::Long), + method_get_float: env + .get_static_method_id(class, "getFloat", "(JJ)F") + .unwrap(), + method_get_float_ret: JavaType::Primitive(Primitive::Float), + method_get_double: env + .get_static_method_id(class, "getDouble", "(JJ)D") + .unwrap(), + method_get_double_ret: JavaType::Primitive(Primitive::Double), + method_get_decimal: env + .get_static_method_id(class, "getDecimal", "(JJ)[B") + .unwrap(), + method_get_decimal_ret: JavaType::Array(Box::new(JavaType::Primitive(Primitive::Byte))), + method_get_string: env + .get_static_method_id(class, "getString", "(JJ)Ljava/lang/String;") + .unwrap(), + method_get_string_ret: JavaType::Object("java/lang/String".to_owned()), + method_get_binary: env + .get_static_method_id(class, "getBinary", "(JJ)[B") + .unwrap(), + method_get_binary_ret: JavaType::Array(Box::new(JavaType::Primitive(Primitive::Byte))), + method_is_null: env.get_static_method_id(class, "isNull", "(JJ)Z").unwrap(), + method_is_null_ret: JavaType::Primitive(Primitive::Boolean), + }) + } +} diff --git a/core/src/jvm_bridge/comet_metric_node.rs b/core/src/jvm_bridge/comet_metric_node.rs new file mode 100644 index 0000000000..1d4928a09f --- /dev/null +++ b/core/src/jvm_bridge/comet_metric_node.rs @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 jni::{ + errors::Result as JniResult, + objects::{JClass, JMethodID}, + signature::{JavaType, Primitive}, + JNIEnv, +}; + +use super::get_global_jclass; + +/// A struct that holds all the JNI methods and fields for JVM CometMetricNode class. +pub struct CometMetricNode<'a> { + pub class: JClass<'a>, + pub method_get_child_node: JMethodID<'a>, + pub method_get_child_node_ret: JavaType, + pub method_add: JMethodID<'a>, + pub method_add_ret: JavaType, +} + +impl<'a> CometMetricNode<'a> { + pub const JVM_CLASS: &'static str = "org/apache/spark/sql/comet/CometMetricNode"; + + pub fn new(env: &JNIEnv<'a>) -> JniResult<CometMetricNode<'a>> { + // Get the global class reference + let class = get_global_jclass(env, Self::JVM_CLASS)?; + + Ok(CometMetricNode { + class, + method_get_child_node: env + .get_method_id( + class, + "getChildNode", + format!("(I)L{:};", Self::JVM_CLASS).as_str(), + ) + .unwrap(), + method_get_child_node_ret: JavaType::Object(Self::JVM_CLASS.to_owned()), + method_add: env + .get_method_id(class, "add", "(Ljava/lang/String;J)V") + .unwrap(), + method_add_ret: JavaType::Primitive(Primitive::Void), + }) + } +} diff --git a/core/src/jvm_bridge/mod.rs b/core/src/jvm_bridge/mod.rs new file mode 100644 index 0000000000..6f162a0ea1 --- /dev/null +++ b/core/src/jvm_bridge/mod.rs @@ -0,0 +1,213 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! JNI JVM related functions + +use jni::{ + errors::{Error, Result as JniResult}, + objects::{JClass, JObject, JString, JValue}, + AttachGuard, JNIEnv, +}; +use once_cell::sync::OnceCell; + +/// Macro for converting JNI Error to Comet Error. +#[macro_export] +macro_rules! jni_map_error { + ($env:expr, $result:expr) => {{ + match $result { + Ok(result) => datafusion::error::Result::Ok(result), + Err(jni_error) => Err($crate::errors::CometError::JNI { source: jni_error }), + } + }}; +} + +/// Macro for converting Rust types to JNI types. +macro_rules! jvalues { + ($($args:expr,)* $(,)?) => {{ + &[$(jni::objects::JValue::from($args)),*] as &[jni::objects::JValue] + }} +} + +/// Macro for create a new JNI string. +macro_rules! jni_new_string { + ($env:expr, $value:expr) => {{ + $crate::jvm_bridge::jni_map_error!($env, $env.new_string($value)) + }}; +} + +/// Macro for calling a JNI method. +/// The syntax is: +/// jni_call!(env, comet_metric_node(metric_node).add(jname, value) -> ())?; +/// comet_metric_node is the class name stored in [[JVMClasses]]. +/// metric_node is the Java object on which the method is called. +/// add is the method name. +/// jname and value are the arguments. +macro_rules! jni_call { + ($env:expr, $clsname:ident($obj:expr).$method:ident($($args:expr),* $(,)?) -> $ret:ty) => {{ + $crate::jvm_bridge::jni_map_error!( + $env, + $env.call_method_unchecked( + $obj, + paste::paste! {$crate::jvm_bridge::JVMClasses::get().[<$clsname>].[<method_ $method>]}, + paste::paste! {$crate::jvm_bridge::JVMClasses::get().[<$clsname>].[<method_ $method _ret>]}.clone(), + $crate::jvm_bridge::jvalues!($($args,)*) + ) + ).and_then(|result| $crate::jvm_bridge::jni_map_error!($env, <$ret>::try_from(result))) + }} +} + +macro_rules! jni_static_call { + ($env:expr, $clsname:ident.$method:ident($($args:expr),* $(,)?) -> $ret:ty) => {{ + $crate::jvm_bridge::jni_map_error!( + $env, + $env.call_static_method_unchecked( + paste::paste! {$crate::jvm_bridge::JVMClasses::get().[<$clsname>].[<class>]}, + paste::paste! {$crate::jvm_bridge::JVMClasses::get().[<$clsname>].[<method_ $method>]}, + paste::paste! {$crate::jvm_bridge::JVMClasses::get().[<$clsname>].[<method_ $method _ret>]}.clone(), + $crate::jvm_bridge::jvalues!($($args,)*) + ) + ).and_then(|result| $crate::jvm_bridge::jni_map_error!($env, <$ret>::try_from(result))) + }} +} + +/// Wrapper for JString. Because we cannot implement `TryFrom` trait for `JString` as they +/// are defined in different crates. +pub struct StringWrapper<'a> { + value: JString<'a>, +} + +impl<'a> StringWrapper<'a> { + pub fn new(value: JString<'a>) -> StringWrapper<'a> { + Self { value } + } + + pub fn get(&self) -> &JString { + &self.value + } +} + +pub struct BinaryWrapper<'a> { + value: JObject<'a>, +} + +impl<'a> BinaryWrapper<'a> { + pub fn new(value: JObject<'a>) -> BinaryWrapper<'a> { + Self { value } + } + + pub fn get(&self) -> &JObject { + &self.value + } +} + +impl<'a> TryFrom<JValue<'a>> for StringWrapper<'a> { + type Error = Error; + + fn try_from(value: JValue<'a>) -> Result<StringWrapper<'a>, Error> { + match value { + JValue::Object(b) => Ok(StringWrapper::new(JString::from(b))), + _ => Err(Error::WrongJValueType("object", value.type_name())), + } + } +} + +impl<'a> TryFrom<JValue<'a>> for BinaryWrapper<'a> { + type Error = Error; + + fn try_from(value: JValue<'a>) -> Result<BinaryWrapper<'a>, Error> { + match value { + JValue::Object(b) => Ok(BinaryWrapper::new(b)), + _ => Err(Error::WrongJValueType("object", value.type_name())), + } + } +} + +/// Macro for creating a new global reference. +macro_rules! jni_new_global_ref { + ($env:expr, $obj:expr) => {{ + $crate::jni_map_error!($env, $env.new_global_ref($obj)) + }}; +} + +pub(crate) use jni_call; +pub(crate) use jni_map_error; +pub(crate) use jni_new_global_ref; +pub(crate) use jni_new_string; +pub(crate) use jni_static_call; +pub(crate) use jvalues; + +/// Gets a global reference to a Java class. +pub fn get_global_jclass(env: &JNIEnv<'_>, cls: &str) -> JniResult<JClass<'static>> { + let local_jclass = env.find_class(cls)?; + let global = env.new_global_ref::<JObject>(local_jclass.into())?; + + // A hack to make the `JObject` static. This is safe because the global reference is never + // gc-ed by the JVM before dropping the global reference. + let global_obj = unsafe { std::mem::transmute::<_, JObject<'static>>(global.as_obj()) }; + // Prevent the global reference from being dropped. + let _ = std::mem::ManuallyDrop::new(global); + + Ok(JClass::from(global_obj)) +} + +mod comet_exec; +pub use comet_exec::*; +mod comet_metric_node; +use crate::JAVA_VM; +pub use comet_metric_node::*; + +/// The JVM classes that are used in the JNI calls. +pub struct JVMClasses<'a> { + /// The CometMetricNode class. Used for updating the metrics. + pub comet_metric_node: CometMetricNode<'a>, + /// The static CometExec class. Used for getting the subquery result. + pub comet_exec: CometExec<'a>, +} + +unsafe impl<'a> Send for JVMClasses<'a> {} +unsafe impl<'a> Sync for JVMClasses<'a> {} + +/// Keeps global references to JVM classes. Used for JNI calls to JVM. +static JVM_CLASSES: OnceCell<JVMClasses> = OnceCell::new(); + +impl JVMClasses<'_> { + /// Creates a new JVMClasses struct. + pub fn init(env: &JNIEnv) { + JVM_CLASSES.get_or_init(|| { + // A hack to make the `JNIEnv` static. It is not safe but we don't really use the + // `JNIEnv` except for creating the global references of the classes. + let env = unsafe { std::mem::transmute::<_, &'static JNIEnv>(env) }; + + JVMClasses { + comet_metric_node: CometMetricNode::new(env).unwrap(), + comet_exec: CometExec::new(env).unwrap(), + } + }); + } + + pub fn get() -> &'static JVMClasses<'static> { + unsafe { JVM_CLASSES.get_unchecked() } + } + + /// Gets the JNIEnv for the current thread. + pub fn get_env() -> AttachGuard<'static> { + unsafe { + let java_vm = JAVA_VM.get_unchecked(); + java_vm.attach_current_thread().unwrap() + } + } +} diff --git a/core/src/lib.rs b/core/src/lib.rs new file mode 100644 index 0000000000..c85263f4f6 --- /dev/null +++ b/core/src/lib.rs @@ -0,0 +1,137 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#![allow(incomplete_features)] +#![allow(non_camel_case_types)] +#![allow(dead_code)] +#![allow(clippy::upper_case_acronyms)] +#![allow(clippy::derive_partial_eq_without_eq)] // For prost generated struct +#![cfg_attr(feature = "nightly", feature(core_intrinsics))] +#![feature(int_roundings)] +#![feature(specialization)] + +// Branch prediction hint. This is currently only available on nightly. +#[cfg(feature = "nightly")] +use core::intrinsics::{likely, unlikely}; + +use jni::{ + objects::{JClass, JString}, + JNIEnv, JavaVM, +}; +use log::{info, LevelFilter}; +use log4rs::{ + append::console::ConsoleAppender, + config::{load_config_file, Appender, Deserializers, Root}, + encode::pattern::PatternEncoder, + Config, +}; +#[cfg(feature = "mimalloc")] +use mimalloc::MiMalloc; +use once_cell::sync::OnceCell; + +pub use data_type::*; + +use crate::errors::{try_unwrap_or_throw, CometError, CometResult}; + +#[macro_use] +mod errors; +#[macro_use] +pub mod common; +mod data_type; +pub mod execution; +mod jvm_bridge; +pub mod parquet; + +#[cfg(feature = "mimalloc")] +#[global_allocator] +static GLOBAL: MiMalloc = MiMalloc; + +static JAVA_VM: OnceCell<JavaVM> = OnceCell::new(); + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_NativeBase_init( + env: JNIEnv, + _: JClass, + log_conf_path: JString, +) { + // Initialize the error handling to capture panic backtraces + errors::init(); + + try_unwrap_or_throw(env, || { + let path: String = env.get_string(log_conf_path)?.into(); + + // empty path means there is no custom log4rs config file provided, so fallback to use + // the default configuration + let log_config = if path.is_empty() { + default_logger_config() + } else { + load_config_file(path, Deserializers::default()) + .map_err(|err| CometError::Config(err.to_string())) + }; + + let _ = log4rs::init_config(log_config?).map_err(|err| CometError::Config(err.to_string())); + + // Initialize the global Java VM + let java_vm = env.get_java_vm()?; + JAVA_VM.get_or_init(|| java_vm); + + info!("Comet native library initialized"); + Ok(()) + }) +} + +const LOG_PATTERN: &str = "{d(%y/%m/%d %H:%M:%S)} {l} {f}: {m}{n}"; + +// Creates a default log4rs config, which logs to console with `INFO` level. +fn default_logger_config() -> CometResult<Config> { + let console_append = ConsoleAppender::builder() + .encoder(Box::new(PatternEncoder::new(LOG_PATTERN))) + .build(); + let appender = Appender::builder().build("console", Box::new(console_append)); + let root = Root::builder().appender("console").build(LevelFilter::Info); + Config::builder() + .appender(appender) + .build(root) + .map_err(|err| CometError::Config(err.to_string())) +} + +// These are borrowed from hashbrown crate: +// https://github.com/rust-lang/hashbrown/blob/master/src/raw/mod.rs + +// On stable we can use #[cold] to get a equivalent effect: this attributes +// suggests that the function is unlikely to be called +#[cfg(not(feature = "nightly"))] +#[inline] +#[cold] +fn cold() {} + +#[cfg(not(feature = "nightly"))] +#[inline] +fn likely(b: bool) -> bool { + if !b { + cold(); + } + b +} +#[cfg(not(feature = "nightly"))] +#[inline] +fn unlikely(b: bool) -> bool { + if b { + cold(); + } + b +} diff --git a/core/src/parquet/compression.rs b/core/src/parquet/compression.rs new file mode 100644 index 0000000000..37b857f4a2 --- /dev/null +++ b/core/src/parquet/compression.rs @@ -0,0 +1,319 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Contains codec interface and supported codec implementations. +//! +//! See [`Compression`](crate::basic::Compression) enum for all available compression +//! algorithms. +//! +//! # Example +//! +//! ```no_run +//! use comet::parquet::{basic::Compression, compression::create_codec}; +//! +//! let mut codec = match create_codec(Compression::SNAPPY) { +//! Ok(Some(codec)) => codec, +//! _ => panic!(), +//! }; +//! +//! let data = vec![b'p', b'a', b'r', b'q', b'u', b'e', b't']; +//! let mut compressed = vec![]; +//! codec.compress(&data[..], &mut compressed).unwrap(); +//! +//! let mut output = vec![]; +//! codec.decompress(&compressed[..], &mut output).unwrap(); +//! +//! assert_eq!(output, data); +//! ``` + +use super::basic::Compression as CodecType; +use crate::errors::{ParquetError, ParquetResult as Result}; + +use brotli::Decompressor; +use flate2::{read, write, Compression}; +use snap::raw::{decompress_len, max_compress_len, Decoder, Encoder}; +use std::io::{copy, Read, Write}; + +/// Parquet compression codec interface. +#[allow(clippy::ptr_arg)] +pub trait Codec { + /// Compresses data stored in slice `input_buf` and writes the compressed result + /// to `output_buf`. + /// Note that you'll need to call `clear()` before reusing the same `output_buf` + /// across different `compress` calls. + fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<()>; + + /// Decompresses data stored in slice `input_buf` and writes output to `output_buf`. + /// Returns the total number of bytes written. + fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<usize>; +} + +/// Given the compression type `codec`, returns a codec used to compress and decompress +/// bytes for the compression type. +/// This returns `None` if the codec type is `UNCOMPRESSED`. +pub fn create_codec(codec: CodecType) -> Result<Option<Box<dyn Codec>>> { + match codec { + CodecType::BROTLI => Ok(Some(Box::new(BrotliCodec::new()))), + CodecType::GZIP => Ok(Some(Box::new(GZipCodec::new()))), + CodecType::SNAPPY => Ok(Some(Box::new(SnappyCodec::new()))), + CodecType::LZ4 => Ok(Some(Box::new(LZ4Codec::new()))), + CodecType::ZSTD => Ok(Some(Box::new(ZSTDCodec::new()))), + CodecType::UNCOMPRESSED => Ok(None), + _ => Err(nyi_err!("The codec type {} is not supported yet", codec)), + } +} + +/// Codec for Snappy compression format. +pub struct SnappyCodec { + decoder: Decoder, + encoder: Encoder, +} + +impl SnappyCodec { + /// Creates new Snappy compression codec. + pub(crate) fn new() -> Self { + Self { + decoder: Decoder::new(), + encoder: Encoder::new(), + } + } +} + +impl Codec for SnappyCodec { + fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<usize> { + let len = decompress_len(input_buf)?; + output_buf.resize(len, 0); + self.decoder + .decompress(input_buf, output_buf) + .map_err(|e| e.into()) + } + + fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<()> { + let output_buf_len = output_buf.len(); + let required_len = max_compress_len(input_buf.len()); + output_buf.resize(output_buf_len + required_len, 0); + let n = self + .encoder + .compress(input_buf, &mut output_buf[output_buf_len..])?; + output_buf.truncate(output_buf_len + n); + Ok(()) + } +} + +/// Codec for GZIP compression algorithm. +pub struct GZipCodec {} + +impl GZipCodec { + /// Creates new GZIP compression codec. + pub(crate) fn new() -> Self { + Self {} + } +} + +impl Codec for GZipCodec { + fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<usize> { + let mut decoder = read::GzDecoder::new(input_buf); + decoder.read_to_end(output_buf).map_err(|e| e.into()) + } + + fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<()> { + let mut encoder = write::GzEncoder::new(output_buf, Compression::default()); + encoder.write_all(input_buf)?; + encoder.try_finish().map_err(|e| e.into()) + } +} + +const BROTLI_DEFAULT_BUFFER_SIZE: usize = 4096; +const BROTLI_DEFAULT_COMPRESSION_QUALITY: u32 = 1; // supported levels 0-9 +const BROTLI_DEFAULT_LG_WINDOW_SIZE: u32 = 22; // recommended between 20-22 + +/// Codec for Brotli compression algorithm. +pub struct BrotliCodec {} + +impl BrotliCodec { + /// Creates new Brotli compression codec. + pub(crate) fn new() -> Self { + Self {} + } +} + +impl Codec for BrotliCodec { + fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<usize> { + Decompressor::new(input_buf, BROTLI_DEFAULT_BUFFER_SIZE) + .read_to_end(output_buf) + .map_err(|e| e.into()) + } + + fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<()> { + let mut encoder = brotli::CompressorWriter::new( + output_buf, + BROTLI_DEFAULT_BUFFER_SIZE, + BROTLI_DEFAULT_COMPRESSION_QUALITY, + BROTLI_DEFAULT_LG_WINDOW_SIZE, + ); + encoder.write_all(input_buf)?; + encoder.flush().map_err(|e| e.into()) + } +} + +const LZ4_BUFFER_SIZE: usize = 4096; + +/// Codec for LZ4 compression algorithm. +pub struct LZ4Codec {} + +impl LZ4Codec { + /// Creates new LZ4 compression codec. + pub(crate) fn new() -> Self { + Self {} + } +} + +impl Codec for LZ4Codec { + fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<usize> { + let mut decoder = lz4::Decoder::new(input_buf)?; + let mut buffer: [u8; LZ4_BUFFER_SIZE] = [0; LZ4_BUFFER_SIZE]; + let mut total_len = 0; + loop { + let len = decoder.read(&mut buffer)?; + if len == 0 { + break; + } + total_len += len; + output_buf.write_all(&buffer[0..len])?; + } + Ok(total_len) + } + + fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<()> { + let mut encoder = lz4::EncoderBuilder::new().build(output_buf)?; + let mut from = 0; + loop { + let to = std::cmp::min(from + LZ4_BUFFER_SIZE, input_buf.len()); + encoder.write_all(&input_buf[from..to])?; + from += LZ4_BUFFER_SIZE; + if from >= input_buf.len() { + break; + } + } + encoder.finish().1.map_err(|e| e.into()) + } +} + +/// Codec for Zstandard compression algorithm. +pub struct ZSTDCodec {} + +impl ZSTDCodec { + /// Creates new Zstandard compression codec. + pub(crate) fn new() -> Self { + Self {} + } +} + +/// Compression level (1-21) for ZSTD. Choose 1 here for better compression speed. +const ZSTD_COMPRESSION_LEVEL: i32 = 1; + +impl Codec for ZSTDCodec { + fn decompress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<usize> { + let mut decoder = zstd::Decoder::new(input_buf)?; + match copy(&mut decoder, output_buf) { + Ok(n) => Ok(n as usize), + Err(e) => Err(e.into()), + } + } + + fn compress(&mut self, input_buf: &[u8], output_buf: &mut Vec<u8>) -> Result<()> { + let mut encoder = zstd::Encoder::new(output_buf, ZSTD_COMPRESSION_LEVEL)?; + encoder.write_all(input_buf)?; + match encoder.finish() { + Ok(_) => Ok(()), + Err(e) => Err(e.into()), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + use crate::parquet::util::test_common::*; + + fn test_roundtrip(c: CodecType, data: &[u8]) { + let mut c1 = create_codec(c).unwrap().unwrap(); + let mut c2 = create_codec(c).unwrap().unwrap(); + + // Compress with c1 + let mut compressed = Vec::new(); + let mut decompressed = Vec::new(); + c1.compress(data, &mut compressed) + .expect("Error when compressing"); + + // Decompress with c2 + let mut decompressed_size = c2 + .decompress(compressed.as_slice(), &mut decompressed) + .expect("Error when decompressing"); + assert_eq!(data.len(), decompressed_size); + decompressed.truncate(decompressed_size); + assert_eq!(data, decompressed.as_slice()); + + compressed.clear(); + + // Compress with c2 + c2.compress(data, &mut compressed) + .expect("Error when compressing"); + + // Decompress with c1 + decompressed_size = c1 + .decompress(compressed.as_slice(), &mut decompressed) + .expect("Error when decompressing"); + assert_eq!(data.len(), decompressed_size); + decompressed.truncate(decompressed_size); + assert_eq!(data, decompressed.as_slice()); + } + + fn test_codec(c: CodecType) { + let sizes = vec![100, 10000, 100000]; + for size in sizes { + let data = random_bytes(size); + test_roundtrip(c, &data); + } + } + + #[test] + fn test_codec_snappy() { + test_codec(CodecType::SNAPPY); + } + + #[test] + fn test_codec_gzip() { + test_codec(CodecType::GZIP); + } + + #[test] + fn test_codec_brotli() { + test_codec(CodecType::BROTLI); + } + + #[test] + fn test_codec_lz4() { + test_codec(CodecType::LZ4); + } + + #[test] + fn test_codec_zstd() { + test_codec(CodecType::ZSTD); + } +} diff --git a/core/src/parquet/data_type.rs b/core/src/parquet/data_type.rs new file mode 100644 index 0000000000..0fc960e9b2 --- /dev/null +++ b/core/src/parquet/data_type.rs @@ -0,0 +1,111 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use super::read::{PlainDecoding, PlainDictDecoding}; + +pub trait DataType: PlainDecoding + PlainDictDecoding + 'static {} + +macro_rules! make_type { + ($name:ident) => { + pub struct $name {} + impl DataType for $name {} + }; +} + +make_type!(BoolType); +make_type!(Int8Type); +make_type!(UInt8Type); +make_type!(Int16Type); +make_type!(UInt16Type); +make_type!(Int32Type); +make_type!(Int32To64Type); +make_type!(UInt32Type); +make_type!(Int64Type); +make_type!(UInt64Type); +make_type!(FloatType); +make_type!(DoubleType); +make_type!(FloatToDoubleType); +make_type!(ByteArrayType); +make_type!(StringType); +make_type!(Int32DecimalType); +make_type!(Int64DecimalType); +make_type!(FLBADecimalType); +make_type!(FLBADecimal32Type); +make_type!(FLBADecimal64Type); +make_type!(FLBAType); +make_type!(Int32DateType); +make_type!(Int64TimestampMillisType); +make_type!(Int64TimestampMicrosType); +make_type!(Int96TimestampMicrosType); + +pub trait AsBytes { + /// Returns the slice of bytes for an instance of this data type. + fn as_bytes(&self) -> &[u8]; +} + +impl AsBytes for Vec<u8> { + fn as_bytes(&self) -> &[u8] { + self.as_slice() + } +} + +impl<'a> AsBytes for &'a str { + fn as_bytes(&self) -> &[u8] { + (self as &str).as_bytes() + } +} + +impl AsBytes for [u8] { + fn as_bytes(&self) -> &[u8] { + self + } +} + +impl AsBytes for str { + fn as_bytes(&self) -> &[u8] { + (self as &str).as_bytes() + } +} + +macro_rules! make_as_bytes { + ($source_ty:ident) => { + impl AsBytes for $source_ty { + #[allow(clippy::size_of_in_element_count)] + fn as_bytes(&self) -> &[u8] { + unsafe { + ::std::slice::from_raw_parts( + self as *const $source_ty as *const u8, + ::std::mem::size_of::<$source_ty>(), + ) + } + } + } + }; +} + +make_as_bytes!(bool); +make_as_bytes!(i8); +make_as_bytes!(u8); +make_as_bytes!(i16); +make_as_bytes!(u16); +make_as_bytes!(i32); +make_as_bytes!(u32); +make_as_bytes!(i64); +make_as_bytes!(u64); +make_as_bytes!(f32); +make_as_bytes!(f64); +make_as_bytes!(i128); diff --git a/core/src/parquet/mod.rs b/core/src/parquet/mod.rs new file mode 100644 index 0000000000..b1a7b939ce --- /dev/null +++ b/core/src/parquet/mod.rs @@ -0,0 +1,562 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 data_type; +pub mod mutable_vector; +pub use mutable_vector::*; + +#[macro_use] +pub mod util; +pub mod read; + +use std::{boxed::Box, ptr::NonNull, sync::Arc}; + +use crate::errors::{try_unwrap_or_throw, CometError}; + +use arrow::ffi::{FFI_ArrowArray, FFI_ArrowSchema}; + +/// JNI exposed methods +use jni::JNIEnv; +use jni::{ + objects::{GlobalRef, JByteBuffer, JClass}, + sys::{ + jboolean, jbooleanArray, jbyte, jbyteArray, jdouble, jfloat, jint, jintArray, jlong, + jlongArray, jobject, jobjectArray, jshort, + }, +}; + +use crate::execution::utils::SparkArrowConvert; +use arrow::buffer::{Buffer, MutableBuffer}; +use jni::objects::ReleaseMode; +use read::ColumnReader; +use util::jni::{convert_column_descriptor, convert_encoding}; + +use self::util::jni::TypePromotionInfo; + +const STR_CLASS_NAME: &str = "java/lang/String"; + +/// Parquet read context maintained across multiple JNI calls. +struct Context { + pub column_reader: ColumnReader, + pub arrays: Option<(Arc<FFI_ArrowArray>, Arc<FFI_ArrowSchema>)>, + last_data_page: Option<GlobalRef>, +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_initColumnReader( + env: JNIEnv, + _jclass: JClass, + primitive_type: jint, + logical_type: jint, + read_primitive_type: jint, + jni_path: jobjectArray, + max_dl: jint, + max_rl: jint, + bit_width: jint, + is_signed: jboolean, + type_length: jint, + precision: jint, + read_precision: jint, + scale: jint, + time_unit: jint, + is_adjusted_utc: jboolean, + batch_size: jint, + use_decimal_128: jboolean, + use_legacy_date_timestamp: jboolean, +) -> jlong { + try_unwrap_or_throw(env, || { + let desc = convert_column_descriptor( + &env, + primitive_type, + logical_type, + max_dl, + max_rl, + bit_width, + is_signed, + type_length, + precision, + scale, + time_unit, + is_adjusted_utc, + jni_path, + )?; + let promotion_info = TypePromotionInfo::new_from_jni(read_primitive_type, read_precision); + let ctx = Context { + column_reader: ColumnReader::get( + desc, + promotion_info, + batch_size as usize, + use_decimal_128 != 0, + use_legacy_date_timestamp != 0, + ), + arrays: None, + last_data_page: None, + }; + let res = Box::new(ctx); + Ok(Box::into_raw(res) as i64) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setDictionaryPage( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + page_value_count: jint, + page_data: jbyteArray, + encoding: jint, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + + // convert value encoding ordinal to the native encoding definition + let encoding = convert_encoding(encoding); + + // copy the input on-heap buffer to native + let page_len = env.get_array_length(page_data)?; + let mut buffer = MutableBuffer::from_len_zeroed(page_len as usize); + env.get_byte_array_region(page_data, 0, from_u8_slice(buffer.as_slice_mut()))?; + + reader.set_dictionary_page(page_value_count as usize, buffer.into(), encoding); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setPageV1( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + page_value_count: jint, + page_data: jbyteArray, + value_encoding: jint, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + + // convert value encoding ordinal to the native encoding definition + let encoding = convert_encoding(value_encoding); + + // copy the input on-heap buffer to native + let page_len = env.get_array_length(page_data)?; + let mut buffer = MutableBuffer::from_len_zeroed(page_len as usize); + env.get_byte_array_region(page_data, 0, from_u8_slice(buffer.as_slice_mut()))?; + + reader.set_page_v1(page_value_count as usize, buffer.into(), encoding); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setPageBufferV1( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + page_value_count: jint, + buffer: jobject, + value_encoding: jint, +) { + try_unwrap_or_throw(env, || { + let ctx = get_context(handle)?; + let reader = &mut ctx.column_reader; + + // convert value encoding ordinal to the native encoding definition + let encoding = convert_encoding(value_encoding); + + // Get slices from Java DirectByteBuffer + let jbuffer = JByteBuffer::from(buffer); + + // Convert the page to global reference so it won't get GC'd by Java. Also free the last + // page if there is any. + ctx.last_data_page = Some(env.new_global_ref(jbuffer)?); + + let buf_slice = env.get_direct_buffer_address(jbuffer)?; + + unsafe { + let page_ptr = NonNull::new_unchecked(buf_slice.as_ptr() as *mut u8); + let buffer = Buffer::from_custom_allocation( + page_ptr, + buf_slice.len(), + Arc::new(FFI_ArrowArray::empty()), + ); + reader.set_page_v1(page_value_count as usize, buffer, encoding); + } + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setPageV2( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + page_value_count: jint, + def_level_data: jbyteArray, + rep_level_data: jbyteArray, + value_data: jbyteArray, + value_encoding: jint, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + + // convert value encoding ordinal to the native encoding definition + let encoding = convert_encoding(value_encoding); + + // copy the input on-heap buffer to native + let dl_len = env.get_array_length(def_level_data)?; + let mut dl_buffer = MutableBuffer::from_len_zeroed(dl_len as usize); + env.get_byte_array_region(def_level_data, 0, from_u8_slice(dl_buffer.as_slice_mut()))?; + + let rl_len = env.get_array_length(rep_level_data)?; + let mut rl_buffer = MutableBuffer::from_len_zeroed(rl_len as usize); + env.get_byte_array_region(rep_level_data, 0, from_u8_slice(rl_buffer.as_slice_mut()))?; + + let v_len = env.get_array_length(value_data)?; + let mut v_buffer = MutableBuffer::from_len_zeroed(v_len as usize); + env.get_byte_array_region(value_data, 0, from_u8_slice(v_buffer.as_slice_mut()))?; + + reader.set_page_v2( + page_value_count as usize, + dl_buffer.into(), + rl_buffer.into(), + v_buffer.into(), + encoding, + ); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setNull( + env: JNIEnv, + _jclass: JClass, + handle: jlong, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_null(); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setBoolean( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jboolean, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_boolean(value != 0); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setByte( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jbyte, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_fixed::<i8>(value); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setShort( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jshort, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_fixed::<i16>(value); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setInt( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jint, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_fixed::<i32>(value); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setLong( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jlong, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_fixed::<i64>(value); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setFloat( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jfloat, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_fixed::<f32>(value); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setDouble( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jdouble, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_fixed::<f64>(value); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setBinary( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jbyteArray, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + + let len = env.get_array_length(value)?; + let mut buffer = MutableBuffer::from_len_zeroed(len as usize); + env.get_byte_array_region(value, 0, from_u8_slice(buffer.as_slice_mut()))?; + reader.set_binary(buffer); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setDecimal( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jbyteArray, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + + let len = env.get_array_length(value)?; + let mut buffer = MutableBuffer::from_len_zeroed(len as usize); + env.get_byte_array_region(value, 0, from_u8_slice(buffer.as_slice_mut()))?; + reader.set_decimal_flba(buffer); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setPosition( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + value: jlong, + size: jint, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.set_position(value, size as usize); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setIndices( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + offset: jlong, + batch_size: jint, + indices: jlongArray, +) -> jlong { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + let indices = env.get_long_array_elements(indices, ReleaseMode::NoCopyBack)?; + let len = indices.size()? as usize; + // paris alternately contains start index and length of continuous indices + let pairs = unsafe { core::slice::from_raw_parts_mut(indices.as_ptr(), len) }; + let mut skipped = 0; + let mut filled = 0; + for i in (0..len).step_by(2) { + let index = pairs[i]; + let count = pairs[i + 1]; + let skip = std::cmp::min(count, offset - skipped); + skipped += skip; + if count == skip { + continue; + } else if batch_size as i64 == filled { + break; + } + let count = std::cmp::min(count - skip, batch_size as i64 - filled); + filled += count; + reader.set_position(index + skip, count as usize); + } + Ok(filled) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_setIsDeleted( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + is_deleted: jbooleanArray, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + + let len = env.get_array_length(is_deleted)?; + let mut buffer = MutableBuffer::from_len_zeroed(len as usize); + env.get_boolean_array_region(is_deleted, 0, buffer.as_slice_mut())?; + reader.set_is_deleted(buffer); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_resetBatch( + env: JNIEnv, + _jclass: JClass, + handle: jlong, +) { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + reader.reset_batch(); + Ok(()) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_readBatch( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + batch_size: jint, + null_pad_size: jint, +) -> jintArray { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + let (num_values, num_nulls) = + reader.read_batch(batch_size as usize, null_pad_size as usize); + let res = env.new_int_array(2)?; + let buf: [i32; 2] = [num_values as i32, num_nulls as i32]; + env.set_int_array_region(res, 0, &buf)?; + Ok(res) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_skipBatch( + env: JNIEnv, + _jclass: JClass, + handle: jlong, + batch_size: jint, + discard: jboolean, +) -> jint { + try_unwrap_or_throw(env, || { + let reader = get_reader(handle)?; + Ok(reader.skip_batch(batch_size as usize, discard == 0) as jint) + }) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_currentBatch( + env: JNIEnv, + _jclass: JClass, + handle: jlong, +) -> jlongArray { + try_unwrap_or_throw(env, || { + let ctx = get_context(handle)?; + let reader = &mut ctx.column_reader; + let data = reader.current_batch(); + let (array, schema) = data.to_spark()?; + + unsafe { + let arrow_array = Arc::from_raw(array as *const FFI_ArrowArray); + let arrow_schema = Arc::from_raw(schema as *const FFI_ArrowSchema); + ctx.arrays = Some((arrow_array, arrow_schema)); + + let res = env.new_long_array(2)?; + let buf: [i64; 2] = [array, schema]; + env.set_long_array_region(res, 0, &buf) + .expect("set long array region failed"); + Ok(res) + } + }) +} + +#[inline] +fn get_context<'a>(handle: jlong) -> Result<&'a mut Context, CometError> { + unsafe { + (handle as *mut Context) + .as_mut() + .ok_or_else(|| CometError::NullPointer("null context handle".to_string())) + } +} + +#[inline] +fn get_reader<'a>(handle: jlong) -> Result<&'a mut ColumnReader, CometError> { + Ok(&mut get_context(handle)?.column_reader) +} + +#[no_mangle] +pub extern "system" fn Java_org_apache_comet_parquet_Native_closeColumnReader( + env: JNIEnv, + _jclass: JClass, + handle: jlong, +) { + try_unwrap_or_throw(env, || { + unsafe { + let ctx = handle as *mut Context; + let _ = Box::from_raw(ctx); + }; + Ok(()) + }) +} + +fn from_u8_slice(src: &mut [u8]) -> &mut [i8] { + let raw_ptr = src.as_mut_ptr() as *mut i8; + unsafe { std::slice::from_raw_parts_mut(raw_ptr, src.len()) } +} diff --git a/core/src/parquet/mutable_vector.rs b/core/src/parquet/mutable_vector.rs new file mode 100644 index 0000000000..f1428fd396 --- /dev/null +++ b/core/src/parquet/mutable_vector.rs @@ -0,0 +1,251 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::ArrayData, datatypes::DataType as ArrowDataType}; + +use crate::common::{bit, CometBuffer}; + +const DEFAULT_ARRAY_LEN: usize = 4; + +/// A mutable vector that can be re-used across batches, for Parquet read. +/// +/// Note this class is similar to [`MutableVector`](crate::common::MutableVector). However, the +/// latter has functionalities such as `ValueGetter`, `ValueSetter`. In addition, it represents +/// String and Binary data using [`StringView`](crate::data_type::StringView), while this struct +/// uses Arrow format to represent them. +/// +/// TODO: unify the two structs in future +#[derive(Debug)] +pub struct ParquetMutableVector { + /// The Arrow type for the elements of this vector. + pub(crate) arrow_type: ArrowDataType, + + /// The number of total elements in this vector. + pub(crate) num_values: usize, + + /// The number of null elements in this vector, must <= `num_values`. + pub(crate) num_nulls: usize, + + /// The capacity of the vector + pub(crate) capacity: usize, + + /// How many bits are required to store a single value + pub(crate) bit_width: usize, + + /// The validity buffer of this Arrow vector. A bit set at position `i` indicates the `i`th + /// element is not null. Otherwise, an unset bit at position `i` indicates the `i`th element is + /// null. + pub(crate) validity_buffer: CometBuffer, + + /// The value buffer of this Arrow vector. This could store either offsets if the vector + /// is of list or struct type, or actual values themselves otherwise. + pub(crate) value_buffer: CometBuffer, + + /// Child vectors for non-primitive types (e.g., list, struct). + pub(crate) children: Vec<ParquetMutableVector>, + + /// Dictionary (i.e., values) associated with this vector. Only set if using dictionary + /// encoding. + pub(crate) dictionary: Option<Box<ParquetMutableVector>>, +} + +impl ParquetMutableVector { + pub fn new(capacity: usize, arrow_type: &ArrowDataType) -> Self { + let bit_width = Self::bit_width(arrow_type); + Self::new_with_bit_width(capacity, arrow_type.clone(), bit_width) + } + + pub fn new_with_bit_width( + capacity: usize, + arrow_type: ArrowDataType, + bit_width: usize, + ) -> Self { + let validity_len = bit::ceil(capacity, 8); + let validity_buffer = CometBuffer::new(validity_len); + + let mut value_capacity = capacity; + if Self::is_binary_type(&arrow_type) { + // Arrow offset array needs to have one extra slot + value_capacity += 1; + } + // Make sure the capacity is positive + let len = bit::ceil(value_capacity * bit_width, 8); + let mut value_buffer = CometBuffer::new(len); + + let mut children = Vec::new(); + + match arrow_type { + ArrowDataType::Binary | ArrowDataType::Utf8 => { + children.push(ParquetMutableVector::new_with_bit_width( + capacity, + ArrowDataType::Int8, + DEFAULT_ARRAY_LEN * 8, + )); + } + _ => {} + } + + if Self::is_binary_type(&arrow_type) { + // Setup the first offset which is always 0. + let zero: u32 = 0; + bit::memcpy_value(&zero, 4, &mut value_buffer); + } + + Self { + arrow_type, + num_values: 0, + num_nulls: 0, + capacity, + bit_width, + validity_buffer, + value_buffer, + children, + dictionary: None, + } + } + + /// Whether the given value at `idx` of this vector is null. + #[inline] + pub fn is_null(&self, idx: usize) -> bool { + unsafe { !bit::get_bit_raw(self.validity_buffer.as_ptr(), idx) } + } + + /// Resets this vector to the initial state. + #[inline] + pub fn reset(&mut self) { + self.num_values = 0; + self.num_nulls = 0; + self.validity_buffer.reset(); + if Self::is_binary_type(&self.arrow_type) { + // Reset the first offset to 0 + let zero: u32 = 0; + bit::memcpy_value(&zero, 4, &mut self.value_buffer); + // Also reset the child value vector + let child = &mut self.children[0]; + child.reset(); + } else if Self::should_reset_value_buffer(&self.arrow_type) { + self.value_buffer.reset(); + } + } + + /// Appends a new null value to the end of this vector. + #[inline] + pub fn put_null(&mut self) { + self.put_nulls(1) + } + + /// Appends `n` null values to the end of this vector. + #[inline] + pub fn put_nulls(&mut self, n: usize) { + // We need to update offset buffer for binary. + if Self::is_binary_type(&self.arrow_type) { + let mut offset = self.num_values * 4; + let prev_offset_value = bit::read_num_bytes_u32(4, &self.value_buffer[offset..]); + offset += 4; + (0..n).for_each(|_| { + bit::memcpy_value(&prev_offset_value, 4, &mut self.value_buffer[offset..]); + offset += 4; + }); + } + + self.num_nulls += n; + self.num_values += n; + } + + /// Returns the number of total values (including both null and non-null) of this vector. + #[inline] + pub fn num_values(&self) -> usize { + self.num_values + } + + /// Returns the number of null values of this vector. + #[inline] + pub fn num_nulls(&self) -> usize { + self.num_nulls + } + + /// Sets the dictionary of this to be `dict`. + pub fn set_dictionary(&mut self, dict: ParquetMutableVector) { + self.dictionary = Some(Box::new(dict)) + } + + /// Clones this into an Arrow [`ArrayData`](arrow::array::ArrayData). Note that the caller of + /// this method MUST make sure the returned `ArrayData` won't live longer than this vector + /// itself. Otherwise, dangling pointer may happen. + /// + /// # Safety + /// + /// This method is highly unsafe since it calls `CometBuffer::to_arrow` which leaks raw + /// pointer to the memory region that are tracked by `CometBuffer`. Please see comments on + /// `to_arrow` buffer to understand the motivation. + pub fn get_array_data(&mut self) -> ArrayData { + unsafe { + let data_type = if let Some(d) = &self.dictionary { + ArrowDataType::Dictionary( + Box::new(ArrowDataType::Int32), + Box::new(d.arrow_type.clone()), + ) + } else { + self.arrow_type.clone() + }; + let mut builder = ArrayData::builder(data_type) + .len(self.num_values) + .add_buffer(self.value_buffer.to_arrow()) + .null_bit_buffer(Some(self.validity_buffer.to_arrow())) + .null_count(self.num_nulls); + + if Self::is_binary_type(&self.arrow_type) && self.dictionary.is_none() { + let child = &mut self.children[0]; + builder = builder.add_buffer(child.value_buffer.to_arrow()); + } + + if let Some(d) = &mut self.dictionary { + builder = builder.add_child_data(d.get_array_data()); + } + + builder.build_unchecked() + } + } + + /// Returns the number of bits it takes to store one element of `arrow_type` in the value buffer + /// of this vector. + pub fn bit_width(arrow_type: &ArrowDataType) -> usize { + match arrow_type { + ArrowDataType::Boolean => 1, + ArrowDataType::Int8 => 8, + ArrowDataType::Int16 => 16, + ArrowDataType::Int32 | ArrowDataType::Float32 | ArrowDataType::Date32 => 32, + ArrowDataType::Int64 | ArrowDataType::Float64 | ArrowDataType::Timestamp(_, _) => 64, + ArrowDataType::FixedSizeBinary(type_length) => *type_length as usize * 8, + ArrowDataType::Decimal128(..) => 128, // Arrow stores decimal with 16 bytes + ArrowDataType::Binary | ArrowDataType::Utf8 => 32, // Only count offset size + dt => panic!("Unsupported Arrow data type: {:?}", dt), + } + } + + #[inline] + fn is_binary_type(dt: &ArrowDataType) -> bool { + matches!(dt, ArrowDataType::Binary | ArrowDataType::Utf8) + } + + #[inline] + fn should_reset_value_buffer(dt: &ArrowDataType) -> bool { + // - Boolean type expects have a zeroed value buffer + // - Decimal may pad buffer with 0xff so we need to clear them before a new batch + matches!(dt, ArrowDataType::Boolean | ArrowDataType::Decimal128(_, _)) + } +} diff --git a/core/src/parquet/read/column.rs b/core/src/parquet/read/column.rs new file mode 100644 index 0000000000..6fc73f93fc --- /dev/null +++ b/core/src/parquet/read/column.rs @@ -0,0 +1,828 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{marker::PhantomData, sync::Arc}; + +use arrow::{ + array::ArrayData, + buffer::{Buffer, MutableBuffer}, + datatypes::{ArrowNativeType, DataType as ArrowDataType, TimeUnit}, +}; + +use parquet::{ + basic::{Encoding, LogicalType, TimeUnit as ParquetTimeUnit, Type as PhysicalType}, + schema::types::{ColumnDescPtr, ColumnDescriptor}, +}; + +use crate::parquet::{ + data_type::*, read::DECIMAL_BYTE_WIDTH, util::jni::TypePromotionInfo, ParquetMutableVector, +}; + +use super::{ + levels::LevelDecoder, + values::{get_decoder, Decoder}, + ReadOptions, +}; + +use crate::common::{bit, bit::log2}; + +/// Maximum number of decimal digits an i32 can represent +const DECIMAL_MAX_INT_DIGITS: i32 = 9; + +/// Maximum number of decimal digits an i64 can represent +const DECIMAL_MAX_LONG_DIGITS: i32 = 18; + +pub enum ColumnReader { + BoolColumnReader(TypedColumnReader<BoolType>), + Int8ColumnReader(TypedColumnReader<Int8Type>), + UInt8ColumnReader(TypedColumnReader<UInt8Type>), + Int16ColumnReader(TypedColumnReader<Int16Type>), + UInt16ColumnReader(TypedColumnReader<UInt16Type>), + Int32ColumnReader(TypedColumnReader<Int32Type>), + Int32To64ColumnReader(TypedColumnReader<Int32To64Type>), + UInt32ColumnReader(TypedColumnReader<UInt32Type>), + Int32DecimalColumnReader(TypedColumnReader<Int32DecimalType>), + Int32DateColumnReader(TypedColumnReader<Int32DateType>), + Int64ColumnReader(TypedColumnReader<Int64Type>), + UInt64DecimalColumnReader(TypedColumnReader<UInt64Type>), + Int64DecimalColumnReader(TypedColumnReader<Int64DecimalType>), + Int64TimestampMillisColumnReader(TypedColumnReader<Int64TimestampMillisType>), + Int64TimestampMicrosColumnReader(TypedColumnReader<Int64TimestampMicrosType>), + Int64TimestampNanosColumnReader(TypedColumnReader<Int64Type>), + Int96ColumnReader(TypedColumnReader<Int96TimestampMicrosType>), + FloatColumnReader(TypedColumnReader<FloatType>), + FloatToDoubleColumnReader(TypedColumnReader<FloatToDoubleType>), + DoubleColumnReader(TypedColumnReader<DoubleType>), + ByteArrayColumnReader(TypedColumnReader<ByteArrayType>), + StringColumnReader(TypedColumnReader<StringType>), + FLBADecimalColumnReader(TypedColumnReader<FLBADecimalType>), + FLBADecimal32ColumnReader(TypedColumnReader<FLBADecimal32Type>), + FLBADecimal64ColumnReader(TypedColumnReader<FLBADecimal64Type>), + FLBAColumnReader(TypedColumnReader<FLBAType>), +} + +impl ColumnReader { + /// Creates a new column reader according to the input `desc`. + /// + /// - `desc`: The actual descriptor for the underlying Parquet files + /// - `promotion_info`: Extra information about type promotion. This is passed in to support + /// schema evolution, e.g., int -> long, where Parquet type is int but Spark type is long. + /// - `use_decimal_128`: Whether to read small precision decimals as `i128` instead of as `i32` + /// or `i64` as Spark does + /// - `use_legacy_date_timestamp_or_ntz`: Whether to read dates/timestamps that were written + /// using the legacy Julian/Gregorian hybrid calendar as it is. If false, exceptions will be + /// thrown. If the spark type is TimestampNTZ, this should be true. + pub fn get( + desc: ColumnDescriptor, + promotion_info: TypePromotionInfo, + capacity: usize, + use_decimal_128: bool, + use_legacy_date_timestamp_or_ntz: bool, + ) -> Self { + let read_options = ReadOptions { + use_legacy_date_timestamp_or_ntz, + }; + macro_rules! typed_reader { + ($reader_ty:ident, $arrow_ty:ident) => { + Self::$reader_ty(TypedColumnReader::new( + desc, + capacity, + ArrowDataType::$arrow_ty, + read_options, + )) + }; + ($reader_ty:ident, $arrow_ty:expr) => { + Self::$reader_ty(TypedColumnReader::new( + desc, + capacity, + $arrow_ty, + read_options, + )) + }; + } + + match desc.physical_type() { + PhysicalType::BOOLEAN => typed_reader!(BoolColumnReader, Boolean), + PhysicalType::INT32 => { + if let Some(ref logical_type) = desc.logical_type() { + match logical_type { + lt @ LogicalType::Integer { + bit_width, + is_signed, + } => match (bit_width, is_signed) { + (8, true) => typed_reader!(Int8ColumnReader, Int8), + (8, false) => typed_reader!(UInt8ColumnReader, Int16), + (16, true) => typed_reader!(Int16ColumnReader, Int16), + (16, false) => typed_reader!(UInt16ColumnReader, Int32), + (32, true) => typed_reader!(Int32ColumnReader, Int32), + (32, false) => typed_reader!(UInt32ColumnReader, Int64), + _ => unimplemented!("Unsupported INT32 annotation: {:?}", lt), + }, + LogicalType::Decimal { scale, precision } => { + if use_decimal_128 { + typed_reader!( + Int32DecimalColumnReader, + ArrowDataType::Decimal128(*precision as u8, *scale as i8) + ) + } else { + typed_reader!(Int32ColumnReader, Int32) + } + } + LogicalType::Date => typed_reader!(Int32DateColumnReader, Date32), + lt => unimplemented!("Unsupported logical type for INT32: {:?}", lt), + } + } else { + // We support type promotion from int to long + match promotion_info.physical_type { + PhysicalType::INT32 => typed_reader!(Int32ColumnReader, Int32), + PhysicalType::INT64 => typed_reader!(Int32To64ColumnReader, Int64), + t => unimplemented!("Unsupported read physical type for INT32: {}", t), + } + } + } + PhysicalType::INT64 => { + if let Some(ref logical_type) = desc.logical_type() { + match logical_type { + lt @ LogicalType::Integer { + bit_width, + is_signed, + } => match (bit_width, is_signed) { + (64, true) => typed_reader!(Int64ColumnReader, Int64), + (64, false) => typed_reader!( + UInt64DecimalColumnReader, + ArrowDataType::Decimal128(20u8, 0i8) + ), + _ => panic!("Unsupported INT64 annotation: {:?}", lt), + }, + LogicalType::Decimal { scale, precision } => { + if use_decimal_128 { + typed_reader!( + Int64DecimalColumnReader, + ArrowDataType::Decimal128(*precision as u8, *scale as i8) + ) + } else { + typed_reader!(Int64ColumnReader, Int64) + } + } + LogicalType::Timestamp { + is_adjusted_to_u_t_c, + unit, + } => { + // To be consistent with Spark, we always store as micro-second and + // convert milli-second to it. + let time_unit = TimeUnit::Microsecond; + let time_zone = if *is_adjusted_to_u_t_c { + Some("UTC".to_string().into()) + } else { + None + }; + match unit { + ParquetTimeUnit::MILLIS(_) => { + typed_reader!( + Int64TimestampMillisColumnReader, + ArrowDataType::Timestamp(time_unit, time_zone) + ) + } + ParquetTimeUnit::MICROS(_) => { + typed_reader!( + Int64TimestampMicrosColumnReader, + ArrowDataType::Timestamp(time_unit, time_zone) + ) + } + ParquetTimeUnit::NANOS(_) => { + typed_reader!( + Int64TimestampNanosColumnReader, + ArrowDataType::Int64 + ) + } + } + } + lt => panic!("Unsupported logical type for INT64: {:?}", lt), + } + } else { + // By default it is INT(64, true) + typed_reader!(Int64ColumnReader, Int64) + } + } + PhysicalType::INT96 => { + typed_reader!( + Int96ColumnReader, + ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".to_string().into())) + ) + } + PhysicalType::FLOAT => match promotion_info.physical_type { + // We support type promotion from float to double + PhysicalType::FLOAT => typed_reader!(FloatColumnReader, Float32), + PhysicalType::DOUBLE => typed_reader!(FloatToDoubleColumnReader, Float64), + t => panic!("Unsupported read physical type: {} for FLOAT", t), + }, + + PhysicalType::DOUBLE => typed_reader!(DoubleColumnReader, Float64), + PhysicalType::BYTE_ARRAY => { + if let Some(logical_type) = desc.logical_type() { + match logical_type { + LogicalType::String => typed_reader!(StringColumnReader, Utf8), + // https://github.com/apache/parquet-format/blob/master/LogicalTypes.md + // "enum type should interpret ENUM annotated field as a UTF-8" + LogicalType::Enum => typed_reader!(StringColumnReader, Utf8), + lt => panic!("Unsupported logical type for BYTE_ARRAY: {:?}", lt), + } + } else { + typed_reader!(ByteArrayColumnReader, Binary) + } + } + PhysicalType::FIXED_LEN_BYTE_ARRAY => { + if let Some(logical_type) = desc.logical_type() { + match logical_type { + LogicalType::Decimal { precision, scale } => { + if !use_decimal_128 && precision <= DECIMAL_MAX_INT_DIGITS { + typed_reader!(FLBADecimal32ColumnReader, Int32) + } else if !use_decimal_128 && precision <= DECIMAL_MAX_LONG_DIGITS { + typed_reader!(FLBADecimal64ColumnReader, Int64) + } else { + typed_reader!( + FLBADecimalColumnReader, + ArrowDataType::Decimal128(precision as u8, scale as i8) + ) + } + } + LogicalType::Uuid => { + let type_length = desc.type_length(); + typed_reader!( + FLBAColumnReader, + ArrowDataType::FixedSizeBinary(type_length) + ) + } + t => panic!("Unsupported logical type for FIXED_LEN_BYTE_ARRAY: {:?}", t), + } + } else { + let type_length = desc.type_length(); + typed_reader!( + FLBAColumnReader, + ArrowDataType::FixedSizeBinary(type_length) + ) + } + } + } + } +} + +macro_rules! make_func { + ($self:ident, $func:ident $(,$args:ident)*) => ({ + match *$self { + Self::BoolColumnReader(ref typed) => typed.$func($($args),*), + Self::Int8ColumnReader(ref typed) => typed.$func($($args),*), + Self::UInt8ColumnReader(ref typed) => typed.$func($($args),*), + Self::Int16ColumnReader(ref typed) => typed.$func($($args),*), + Self::UInt16ColumnReader(ref typed) => typed.$func($($args),*), + Self::Int32ColumnReader(ref typed) => typed.$func($($args),*), + Self::Int32To64ColumnReader(ref typed) => typed.$func($($args), *), + Self::UInt32ColumnReader(ref typed) => typed.$func($($args),*), + Self::Int32DateColumnReader(ref typed) => typed.$func($($args),*), + Self::Int32DecimalColumnReader(ref typed) => typed.$func($($args),*), + Self::Int64ColumnReader(ref typed) => typed.$func($($args),*), + Self::UInt64DecimalColumnReader(ref typed) => typed.$func($($args),*), + Self::Int64DecimalColumnReader(ref typed) => typed.$func($($args),*), + Self::Int64TimestampMillisColumnReader(ref typed) => typed.$func($($args),*), + Self::Int64TimestampMicrosColumnReader(ref typed) => typed.$func($($args),*), + Self::Int64TimestampNanosColumnReader(ref typed) => typed.$func($($args),*), + Self::FloatColumnReader(ref typed) => typed.$func($($args),*), + Self::DoubleColumnReader(ref typed) => typed.$func($($args),*), + Self::FloatToDoubleColumnReader(ref typed) => typed.$func($($args),*), + Self::ByteArrayColumnReader(ref typed) => typed.$func($($args),*), + Self::StringColumnReader(ref typed) => typed.$func($($args),*), + Self::FLBADecimalColumnReader(ref typed) => typed.$func($($args),*), + Self::FLBADecimal32ColumnReader(ref typed) => typed.$func($($args),*), + Self::FLBADecimal64ColumnReader(ref typed) => typed.$func($($args),*), + Self::FLBAColumnReader(ref typed) => typed.$func($($args),*), + Self::Int96ColumnReader(ref typed) => typed.$func($($args),*), + } + }); +} + +macro_rules! make_func_mut { + ($self:ident, $func:ident $(,$args:ident)*) => ({ + match *$self { + Self::BoolColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int8ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::UInt8ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int16ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::UInt16ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int32ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int32To64ColumnReader(ref mut typed) => typed.$func($($args), *), + Self::UInt32ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int32DateColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int32DecimalColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int64ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::UInt64DecimalColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int64DecimalColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int64TimestampMillisColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int64TimestampMicrosColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int64TimestampNanosColumnReader(ref mut typed) => typed.$func($($args),*), + Self::FloatColumnReader(ref mut typed) => typed.$func($($args),*), + Self::DoubleColumnReader(ref mut typed) => typed.$func($($args),*), + Self::FloatToDoubleColumnReader(ref mut typed) => typed.$func($($args),*), + Self::ByteArrayColumnReader(ref mut typed) => typed.$func($($args),*), + Self::StringColumnReader(ref mut typed) => typed.$func($($args),*), + Self::FLBADecimalColumnReader(ref mut typed) => typed.$func($($args),*), + Self::FLBADecimal32ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::FLBADecimal64ColumnReader(ref mut typed) => typed.$func($($args),*), + Self::FLBAColumnReader(ref mut typed) => typed.$func($($args),*), + Self::Int96ColumnReader(ref mut typed) => typed.$func($($args),*), + } + }); +} + +impl ColumnReader { + #[inline] + pub fn get_descriptor(&self) -> &ColumnDescriptor { + make_func!(self, get_descriptor) + } + + #[inline] + pub fn set_dictionary_page( + &mut self, + page_value_count: usize, + page_data: Buffer, + encoding: Encoding, + ) { + make_func_mut!( + self, + set_dictionary_page, + page_value_count, + page_data, + encoding + ) + } + + #[inline] + pub fn set_page_v1(&mut self, page_value_count: usize, page_data: Buffer, encoding: Encoding) { + make_func_mut!(self, set_page_v1, page_value_count, page_data, encoding) + } + + #[inline] + pub fn set_page_v2( + &mut self, + page_value_count: usize, + def_level_data: Buffer, + rep_level_data: Buffer, + value_data: Buffer, + encoding: Encoding, + ) { + make_func_mut!( + self, + set_page_v2, + page_value_count, + def_level_data, + rep_level_data, + value_data, + encoding + ) + } + + #[inline] + pub fn set_null(&mut self) { + make_func_mut!(self, set_null) + } + + #[inline] + pub fn set_boolean(&mut self, value: bool) { + make_func_mut!(self, set_boolean, value) + } + + #[inline] + pub fn set_fixed<U: ArrowNativeType + AsBytes>(&mut self, value: U) { + make_func_mut!(self, set_fixed, value) + } + + #[inline] + pub fn set_binary(&mut self, value: MutableBuffer) { + make_func_mut!(self, set_binary, value) + } + + #[inline] + pub fn set_decimal_flba(&mut self, value: MutableBuffer) { + make_func_mut!(self, set_decimal_flba, value) + } + + #[inline] + pub fn set_position(&mut self, value: i64, size: usize) { + make_func_mut!(self, set_position, value, size) + } + + #[inline] + pub fn set_is_deleted(&mut self, value: MutableBuffer) { + make_func_mut!(self, set_is_deleted, value) + } + + #[inline] + pub fn reset_batch(&mut self) { + make_func_mut!(self, reset_batch) + } + + #[inline] + pub fn current_batch(&mut self) -> ArrayData { + make_func_mut!(self, current_batch) + } + + #[inline] + pub fn read_batch(&mut self, total: usize, null_pad_size: usize) -> (usize, usize) { + make_func_mut!(self, read_batch, total, null_pad_size) + } + + #[inline] + pub fn skip_batch(&mut self, total: usize, put_nulls: bool) -> usize { + make_func_mut!(self, skip_batch, total, put_nulls) + } +} + +/// A batched reader for a primitive Parquet column. +pub struct TypedColumnReader<T: DataType> { + desc: ColumnDescPtr, + arrow_type: ArrowDataType, + rep_level_decoder: Option<LevelDecoder>, + def_level_decoder: Option<LevelDecoder>, + value_decoder: Option<Box<dyn Decoder>>, + + /// The remaining number of values to read in the current page + num_values_in_page: usize, + /// The value vector for this column reader; reused across batches. + vector: ParquetMutableVector, + /// The batch size for this column vector. + capacity: usize, + /// Number of bits used to represent one value in Parquet. + bit_width: usize, + /// Whether this is a constant column reader (always return constant vector). + is_const: bool, + + // Options for reading Parquet + read_options: ReadOptions, + + /// Marker to allow `T` in the generic parameter of the struct. + _phantom: PhantomData<T>, +} + +impl<T: DataType> TypedColumnReader<T> { + pub fn new( + desc: ColumnDescriptor, + capacity: usize, + arrow_type: ArrowDataType, + read_options: ReadOptions, + ) -> Self { + let vector = ParquetMutableVector::new(capacity, &arrow_type); + let bit_width = ParquetMutableVector::bit_width(&arrow_type); + Self { + desc: Arc::new(desc), + arrow_type, + rep_level_decoder: None, + def_level_decoder: None, + value_decoder: None, + num_values_in_page: 0, + vector, + capacity, + bit_width, + is_const: false, + read_options, + _phantom: PhantomData, + } + } + + #[inline] + pub fn get_descriptor(&self) -> &ColumnDescriptor { + &self.desc + } + + /// Reset the current batch. This will clear all the content of the current columnar vector as + /// well as reset all of its internal states. + #[inline] + pub fn reset_batch(&mut self) { + self.vector.reset() + } + + /// Returns the current batch that's been constructed. + /// + /// Note: the caller must make sure the returned Arrow vector is fully consumed before calling + /// `read_batch` again. + #[inline] + pub fn current_batch(&mut self) -> ArrayData { + self.vector.get_array_data() + } + + /// Reads a batch of at most `total` values from the current page this reader has. Returns a + /// tuple where the first element is the actual number of values read (including both nulls and + /// non-nulls), and the second element is the actual number of nulls read. + /// + /// Pad nulls for the amount of `null_pad_size` before reading. + /// + /// If the return number of values is < `total`, it means the current page is drained and the + /// caller should call `set_page_v1` or `set_page_v2` before calling next `read_batch`. + pub fn read_batch(&mut self, total: usize, null_pad_size: usize) -> (usize, usize) { + debug_assert!( + self.value_decoder.is_some() && self.def_level_decoder.is_some(), + "set_page_v1/v2 should have been called" + ); + + let n = ::std::cmp::min(self.num_values_in_page, total); + self.num_values_in_page -= n; + let value_decoder = self.value_decoder.as_mut().unwrap(); + let dl_decoder = self.def_level_decoder.as_mut().unwrap(); + + let previous_num_nulls = self.vector.num_nulls; + self.vector.put_nulls(null_pad_size); + dl_decoder.read_batch(n, &mut self.vector, value_decoder.as_mut()); + + (n, self.vector.num_nulls - previous_num_nulls) + } + + /// Skips a batch of at most `total` values from the current page this reader has, and returns + /// the actual number of values skipped. + /// + /// If the return value is < `total`, it means the current page is drained and the caller should + /// call `set_page_v1` or `set_page_v2` before calling next `skip_batch`. + pub fn skip_batch(&mut self, total: usize, put_nulls: bool) -> usize { + debug_assert!( + self.value_decoder.is_some() && self.def_level_decoder.is_some(), + "set_page_v1/v2 should have been called" + ); + + let n = ::std::cmp::min(self.num_values_in_page, total); + self.num_values_in_page -= n; + let value_decoder = self.value_decoder.as_mut().unwrap(); + let dl_decoder = self.def_level_decoder.as_mut().unwrap(); + + dl_decoder.skip_batch(n, &mut self.vector, value_decoder.as_mut(), put_nulls); + + n + } + + /// Sets the dictionary page for this column reader and eagerly reads it. + /// + /// # Panics + /// + /// - If being called more than once during the lifetime of this column reader. A Parquet column + /// chunk should only contain a single dictionary page. + /// - If the input `encoding` is neither `PLAIN`, `PLAIN_DICTIONARY` nor `RLE_DICTIONARY`. + pub fn set_dictionary_page( + &mut self, + page_value_count: usize, + page_data: Buffer, + mut encoding: Encoding, + ) { + // In Parquet format v1, both dictionary page and data page use the same encoding + // `PLAIN_DICTIONARY`, while in v2, dictioanry page uses `PLAIN` and data page uses + // `RLE_DICTIONARY`. + // + // Here, we convert `PLAIN` from v2 dictionary page to `PLAIN_DICTIONARY`, so that v1 and v2 + // shares the same encoding. Later on, `get_decoder` will use the `PlainDecoder` for this + // case. + if encoding == Encoding::PLAIN { + encoding = Encoding::PLAIN_DICTIONARY; + } + + if encoding != Encoding::PLAIN_DICTIONARY { + panic!("Invalid encoding type for Parquet dictionary: {}", encoding); + } + + if self.vector.dictionary.is_some() { + panic!("Parquet column cannot have more than one dictionary"); + } + + // Create a new vector for dictionary values + let mut value_vector = ParquetMutableVector::new(page_value_count, &self.arrow_type); + + let mut dictionary = self.get_decoder(page_data, page_value_count, encoding); + dictionary.read_batch(&mut value_vector, page_value_count); + value_vector.num_values = page_value_count; + + // Re-create the parent vector since it is initialized with the dictionary value type, not + // the key type (which is always integer). + self.vector = ParquetMutableVector::new(self.capacity, &ArrowDataType::Int32); + self.vector.set_dictionary(value_vector); + } + + /// Resets the Parquet data page for this column reader. + pub fn set_page_v1( + &mut self, + page_value_count: usize, + page_data: Buffer, + mut encoding: Encoding, + ) { + // In v1, when data is encoded with dictionary, data page uses `PLAIN_DICTIONARY`, while v2 + // uses `RLE_DICTIONARY`. To consolidate the two, here we convert `PLAIN_DICTIONARY` to + // `RLE_DICTIONARY` following v2. Later on, `get_decoder` will use `DictDecoder` for this + // case. + if encoding == Encoding::PLAIN_DICTIONARY { + encoding = Encoding::RLE_DICTIONARY; + } + + self.num_values_in_page = page_value_count; + self.check_dictionary(&encoding); + + let mut page_buffer = page_data; + + let bit_width = log2(self.desc.max_rep_level() as u64 + 1) as u8; + let mut rl_decoder = LevelDecoder::new(self.desc.clone(), bit_width, true); + let offset = rl_decoder.set_data(page_value_count, &page_buffer); + self.rep_level_decoder = Some(rl_decoder); + page_buffer = page_buffer.slice(offset); + + let bit_width = log2(self.desc.max_def_level() as u64 + 1) as u8; + let mut dl_decoder = LevelDecoder::new(self.desc.clone(), bit_width, true); + let offset = dl_decoder.set_data(page_value_count, &page_buffer); + self.def_level_decoder = Some(dl_decoder); + page_buffer = page_buffer.slice(offset); + + let value_decoder = self.get_decoder(page_buffer, page_value_count, encoding); + self.value_decoder = Some(value_decoder); + } + + /// Resets the Parquet data page for this column reader. + pub fn set_page_v2( + &mut self, + page_value_count: usize, + def_level_data: Buffer, + rep_level_data: Buffer, + value_data: Buffer, + encoding: Encoding, + ) { + self.num_values_in_page = page_value_count; + self.check_dictionary(&encoding); + + let bit_width = log2(self.desc.max_rep_level() as u64 + 1) as u8; + let mut rl_decoder = LevelDecoder::new(self.desc.clone(), bit_width, false); + rl_decoder.set_data(page_value_count, &rep_level_data); + self.rep_level_decoder = Some(rl_decoder); + + let bit_width = log2(self.desc.max_def_level() as u64 + 1) as u8; + let mut dl_decoder = LevelDecoder::new(self.desc.clone(), bit_width, false); + dl_decoder.set_data(page_value_count, &def_level_data); + self.def_level_decoder = Some(dl_decoder); + + let value_decoder = self.get_decoder(value_data, page_value_count, encoding); + self.value_decoder = Some(value_decoder); + } + + /// Sets all values in the vector of this column reader to be null. + pub fn set_null(&mut self) { + self.check_const("set_null"); + self.vector.put_nulls(self.capacity); + } + + /// Sets all values in the vector of this column reader to be `value`. + pub fn set_boolean(&mut self, value: bool) { + self.check_const("set_boolean"); + if value { + let dst = self.vector.value_buffer.as_slice_mut(); + bit::set_bits(dst, 0, self.capacity); + } + self.vector.num_values += self.capacity; + } + + /// Sets all values in the vector of this column reader to be `value`. + pub fn set_fixed<U: ArrowNativeType + AsBytes>(&mut self, value: U) { + self.check_const("set_fixed"); + let type_size = std::mem::size_of::<U>(); + + let mut offset = 0; + for _ in 0..self.capacity { + bit::memcpy_value(&value, type_size, &mut self.vector.value_buffer[offset..]); + offset += type_size; + } + self.vector.num_values += self.capacity; + } + + /// Sets all values in the vector of this column reader to be binary represented by `buffer`. + pub fn set_binary(&mut self, buffer: MutableBuffer) { + self.check_const("set_binary"); + + // TODO: consider using dictionary here + + let len = buffer.len(); + let total_len = len * self.capacity; + let offset_buf = self.vector.value_buffer.as_slice_mut(); + let child_vector = &mut self.vector.children[0]; + let value_buf = &mut child_vector.value_buffer; + + value_buf.resize(total_len); + + let mut value_buf_offset = 0; + let mut offset_buf_offset = 4; + for _ in 0..self.capacity { + bit::memcpy(&buffer, &mut value_buf.as_slice_mut()[value_buf_offset..]); + value_buf_offset += len; + + bit::memcpy_value( + &(value_buf_offset as i32), + 4, + &mut offset_buf[offset_buf_offset..], + ); + offset_buf_offset += 4; + } + self.vector.num_values += self.capacity; + } + + /// Sets all values in the vector of this column reader to be decimal represented by `buffer`. + pub fn set_decimal_flba(&mut self, buffer: MutableBuffer) { + self.check_const("set_decimal_flba"); + + // TODO: consider using dictionary here + + let len = buffer.len(); + let mut bytes: [u8; DECIMAL_BYTE_WIDTH] = [0; DECIMAL_BYTE_WIDTH]; + + for i in 0..len { + bytes[len - i - 1] = buffer[i]; + } + if bytes[len - 1] & 0x80 == 0x80 { + bytes[len..DECIMAL_BYTE_WIDTH].fill(0xff); + } + + let mut offset = 0; + for _ in 0..self.capacity { + bit::memcpy(&bytes, &mut self.vector.value_buffer[offset..]); + offset += DECIMAL_BYTE_WIDTH; + } + self.vector.num_values += self.capacity; + } + + /// Sets position values of this column reader to the vector starting from `value`. + pub fn set_position(&mut self, value: i64, size: usize) { + let i64_size = std::mem::size_of::<i64>(); + + let mut offset = self.vector.num_values * i64_size; + for i in value..(value + size as i64) { + // TODO: is it better to convert self.value_buffer to &mut [i64] and for-loop update? + bit::memcpy_value(&i, i64_size, &mut self.vector.value_buffer[offset..]); + offset += i64_size; + } + self.vector.num_values += size; + } + + /// Sets the values in the vector of this column reader to be a boolean array represented + /// by `buffer`. + pub fn set_is_deleted(&mut self, buffer: MutableBuffer) { + let len = buffer.len(); + let dst = self.vector.value_buffer.as_slice_mut(); + for i in 0..len { + if buffer[i] == 1 { + bit::set_bit(dst, i); + } else if buffer[i] == 0 { + bit::unset_bit(dst, i); + } + } + self.vector.num_values += len; + } + + /// Check a few pre-conditions for setting constants, as well as setting + /// that `is_const` to true for the particular column reader. + fn check_const(&mut self, method_name: &str) { + assert!( + self.value_decoder.is_none(), + "{} cannot be called after set_page_v1/set_page_v2!", + method_name + ); + assert!(!self.is_const, "can only set constant once!"); + self.is_const = true; + } + + fn check_dictionary(&mut self, encoding: &Encoding) { + // The column has a dictionary while the new page is of PLAIN encoding. In this case, we + // should eagerly decode all the dictionary indices and convert the underlying vector to a + // plain encoded vector. + if self.vector.dictionary.is_some() && *encoding == Encoding::PLAIN { + let new_vector = ParquetMutableVector::new(self.capacity, &self.arrow_type); + let old_vector = std::mem::replace(&mut self.vector, new_vector); + T::decode_dict(old_vector, &mut self.vector, self.bit_width); + debug_assert!(self.vector.dictionary.is_none()); + } + } + + fn get_decoder( + &self, + value_data: Buffer, + page_value_count: usize, + encoding: Encoding, + ) -> Box<dyn Decoder> { + get_decoder::<T>( + value_data, + page_value_count, + encoding, + self.desc.clone(), + self.read_options, + ) + } +} diff --git a/core/src/parquet/read/levels.rs b/core/src/parquet/read/levels.rs new file mode 100644 index 0000000000..d43c286842 --- /dev/null +++ b/core/src/parquet/read/levels.rs @@ -0,0 +1,230 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::mem; + +use arrow::buffer::Buffer; +use parquet::schema::types::ColumnDescPtr; + +use super::values::Decoder; +use crate::{ + common::bit::{self, read_num_bytes_u32, BitReader}, + parquet::ParquetMutableVector, + unlikely, +}; + +const INITIAL_BUF_LEN: usize = 16; + +enum Mode { + RLE, + BitPacked, +} + +/// A decoder for Parquet definition & repetition levels. +pub struct LevelDecoder { + /// The descriptor of the column that this level decoder is associated to. + desc: ColumnDescPtr, + /// Number of bits used to represent the levels. + bit_width: u8, + /// Mode for the current run. + mode: Mode, + /// Total number of values (including both null and non-null) to be decoded. + num_values: usize, + /// The current value in a RLE run. Unused if BitPacked. + current_value: i32, + /// The number of total values in the current RLE run. Unused if BitPacked. + current_count: usize, + /// The current buffer used in a BitPacked run. Unused if RLE. + /// This will be resized if the total number of values in the BitPacked run is larger than its + /// capacity. + current_buffer: Vec<i32>, // TODO: double check this + /// The index into `current_buffer` in a BitPacked run. Unused if RLE. + current_buffer_idx: usize, + /// A bit reader wrapping the input buffer for levels. + bit_reader: Option<BitReader>, + /// Whether we need to read the length of data. This is typically true for Parquet page V1, but + /// not for V2, since it uses separate buffer for definition & repetition levels. + need_length: bool, +} + +impl LevelDecoder { + pub fn new(desc: ColumnDescPtr, bit_width: u8, need_length: bool) -> Self { + Self { + desc, + bit_width, + mode: Mode::RLE, + num_values: 0, + current_value: 0, + current_count: 0, + current_buffer: vec![0; INITIAL_BUF_LEN], + current_buffer_idx: 0, + bit_reader: None, + need_length, + } + } + + /// Sets data for this level decoder, and returns total number of bytes consumed. This is used + /// for reading DataPage v1 levels. + pub fn set_data(&mut self, page_value_count: usize, page_data: &Buffer) -> usize { + self.num_values = page_value_count; + if self.bit_width == 0 { + // Special case where the page doesn't have encoded rl/dl data. Here we'll treat it as + // an RLE run of `page_value_count` number of 0s. + self.mode = Mode::RLE; + self.current_count = page_value_count; + 0 + } else if self.need_length { + let u32_size = mem::size_of::<u32>(); + let data_size = read_num_bytes_u32(u32_size, page_data.as_slice()) as usize; + self.bit_reader = Some(BitReader::new(page_data.slice(u32_size), data_size)); + u32_size + data_size + } else { + // No need to read length, just read the whole buffer + self.bit_reader = Some(BitReader::new_all(page_data.to_owned())); + 0 + } + } + + /// Reads a batch of `total` values into `vector`. The value decoding is done by + /// `value_decoder`. + pub fn read_batch( + &mut self, + total: usize, + vector: &mut ParquetMutableVector, + value_decoder: &mut dyn Decoder, + ) { + let mut left = total; + while left > 0 { + if unlikely(self.current_count == 0) { + self.read_next_group(); + } + + debug_assert!(self.current_count > 0); + + let n = ::std::cmp::min(left, self.current_count); + let max_def_level = self.desc.max_def_level(); + + match self.mode { + Mode::RLE => { + if self.current_value as i16 == max_def_level { + value_decoder.read_batch(vector, n); + bit::set_bits(vector.validity_buffer.as_slice_mut(), vector.num_values, n); + vector.num_values += n; + } else { + vector.put_nulls(n); + } + } + Mode::BitPacked => { + for i in 0..n { + if self.current_buffer[self.current_buffer_idx + i] == max_def_level as i32 + { + value_decoder.read(vector); + bit::set_bit(vector.validity_buffer.as_slice_mut(), vector.num_values); + vector.num_values += 1; + } else { + vector.put_null(); + } + } + self.current_buffer_idx += n; + } + } + + left -= n; + self.current_count -= n; + } + } + + /// Skips a batch of `total` values. The value decoding is done by `value_decoder`. + pub fn skip_batch( + &mut self, + total: usize, + vector: &mut ParquetMutableVector, + value_decoder: &mut dyn Decoder, + put_nulls: bool, + ) { + let mut skip = total; + while skip > 0 { + if unlikely(self.current_count == 0) { + self.read_next_group(); + } + + debug_assert!(self.current_count > 0); + + let n = ::std::cmp::min(skip, self.current_count); + let max_def_level = self.desc.max_def_level(); + + match self.mode { + Mode::RLE => { + if self.current_value as i16 == max_def_level { + value_decoder.skip_batch(n); + } + } + Mode::BitPacked => { + let mut num_skips = 0; + for i in 0..n { + if self.current_buffer[self.current_buffer_idx + i] == max_def_level as i32 + { + num_skips += 1; + } + } + value_decoder.skip_batch(num_skips); + self.current_buffer_idx += n; + } + } + if put_nulls { + vector.put_nulls(n); + } + + skip -= n; + self.current_count -= n; + } + } + + /// Loads the next group from this RLE/BitPacked hybrid reader. + fn read_next_group(&mut self) { + let bit_reader = self.bit_reader.as_mut().expect("bit_reader should be set"); + if let Some(indicator_value) = bit_reader.get_vlq_int() { + self.mode = if indicator_value & 1 == 1 { + Mode::BitPacked + } else { + Mode::RLE + }; + + match self.mode { + Mode::BitPacked => { + self.current_count = ((indicator_value >> 1) * 8) as usize; + if self.current_buffer.len() < self.current_count { + self.current_buffer.resize(self.current_count, 0); + } + self.current_buffer_idx = 0; + bit_reader.get_batch( + &mut self.current_buffer[..self.current_count], + self.bit_width as usize, + ); + } + Mode::RLE => { + // RLE + self.current_count = (indicator_value >> 1) as usize; + let value_width = bit::ceil(self.bit_width as usize, 8); + self.current_value = bit_reader + .get_aligned::<i32>(value_width) + .expect("current value should be set"); + } + } + } + } +} diff --git a/core/src/parquet/read/mod.rs b/core/src/parquet/read/mod.rs new file mode 100644 index 0000000000..4d057a06c9 --- /dev/null +++ b/core/src/parquet/read/mod.rs @@ -0,0 +1,114 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 column; +pub mod levels; +pub mod values; + +pub use column::ColumnReader; +use parquet::schema::types::ColumnDescPtr; + +use super::ParquetMutableVector; +use crate::common::bit::{self, BitReader}; +use arrow::buffer::Buffer; +use bytes::Buf; + +/// Number of bytes to store a decimal value in Arrow value vector +pub(crate) const DECIMAL_BYTE_WIDTH: usize = 16; + +#[derive(Clone, Copy)] +pub struct ReadOptions { + // Whether to read legacy dates/timestamps as it is. If false, throw exceptions. + pub(crate) use_legacy_date_timestamp_or_ntz: bool, +} + +/// Internal states for PLAIN decoder. Used in combination of `PlainDecoding`. +pub struct PlainDecoderInner { + /// The input buffer containing values to be decoded + data: Buffer, + + /// The current offset in `data`, in bytes. + offset: usize, + + /// The number of total values in `data` + value_count: usize, + + /// Reads `data` bit by bit, used if `T` is [`BoolType`]. + bit_reader: BitReader, + + /// Options for reading Parquet + read_options: ReadOptions, + + /// The Parquet column descriptor + desc: ColumnDescPtr, +} + +/// A trait for [`super::DataType`] to implement how PLAIN encoded data is to be decoded into Arrow +/// format given an input and output buffer. +/// +/// The actual implementations of this trait is in `read/values.rs`. +pub trait PlainDecoding { + /// Decodes `num` of items from `src`, and store the result into `dst`, in Arrow format. + /// + /// Note: this assumes the `src` has data for at least `num` elements, and won't do any + /// bound checking. The condition MUST be guaranteed from the caller side. + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize); + + /// Skip `num` of items from `src` + /// + /// Note: this assumes the `src` has data for at least `num` elements, and won't do any + /// bound checking. The condition MUST be guaranteed from the caller side. + fn skip(src: &mut PlainDecoderInner, num: usize); +} + +pub trait PlainDictDecoding { + /// Eagerly decode vector `src` which must have dictionary associated. The decoded values are + /// appended into `dst`. + fn decode_dict(src: ParquetMutableVector, dst: &mut ParquetMutableVector, bit_width: usize) { + assert!(dst.dictionary.is_none()); + assert!(src.dictionary.is_some()); + + let mut value_buf = src.value_buffer.as_slice(); + let validity_buf = src.validity_buffer.as_slice(); + let dictionary = src.dictionary.as_ref().unwrap(); + + for i in 0..src.num_values { + if bit::get_bit(validity_buf, i) { + // non-null value: lookup the value position and copy its value into `dst` + let val_idx = value_buf.get_u32_le(); + Self::decode_dict_one(i, val_idx as usize, dictionary, dst, bit_width); + dst.num_values += 1; + } else { + value_buf.advance(4); + dst.put_null(); + } + } + + dst.validity_buffer = src.validity_buffer; + } + + /// Decode a single value from `src`, whose position in the dictionary indices (i.e., keys) + /// is `idx` and the positions in the dictionary values is `val_idx`. The decoded value is + /// appended to `dst`. + fn decode_dict_one( + idx: usize, + val_idx: usize, + src: &ParquetMutableVector, + dst: &mut ParquetMutableVector, + bit_width: usize, + ); +} diff --git a/core/src/parquet/read/values.rs b/core/src/parquet/read/values.rs new file mode 100644 index 0000000000..9d9bbb3c9e --- /dev/null +++ b/core/src/parquet/read/values.rs @@ -0,0 +1,1008 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{marker::PhantomData, mem, ptr::copy_nonoverlapping}; + +use arrow::buffer::Buffer; +use bytes::Buf; +use log::debug; +use parquet::{basic::Encoding, schema::types::ColumnDescPtr}; + +use super::{PlainDecoderInner, PlainDecoding, PlainDictDecoding, ReadOptions}; +use crate::{ + common::bit::{self, BitReader}, + parquet::{data_type::*, read::DECIMAL_BYTE_WIDTH, ParquetMutableVector}, + unlikely, +}; + +pub fn get_decoder<T: DataType>( + value_data: Buffer, + num_values: usize, + encoding: Encoding, + desc: ColumnDescPtr, + read_options: ReadOptions, +) -> Box<dyn Decoder> { + let decoder: Box<dyn Decoder> = match encoding { + Encoding::PLAIN | Encoding::PLAIN_DICTIONARY => Box::new(PlainDecoder::<T>::new( + value_data, + num_values, + desc, + read_options, + )), + // This is for dictionary indices + Encoding::RLE_DICTIONARY => Box::new(DictDecoder::new(value_data, num_values)), + _ => panic!("Unsupported encoding: {}", encoding), + }; + decoder +} + +/// A Parquet decoder for values within a Parquet data page. +pub trait Decoder { + /// Consumes a single value from the decoder and stores it into `dst`. + /// + /// # Preconditions + /// + /// * `dst` have enough length to hold at least one value. + /// * `data` of this decoder should have enough bytes left to be decoded. + fn read(&mut self, dst: &mut ParquetMutableVector); + + /// Consumes a batch of `num` values from the data and stores them to `dst`. + /// + /// # Preconditions + /// + /// * `dst` should have length >= `num * T::type_size()` . + /// * `data` of this decoder should have >= `num * T::type_size()` bytes left to be decoded. + fn read_batch(&mut self, dst: &mut ParquetMutableVector, num: usize); + + /// Skips a batch of `num` values from the data. + /// + /// # Preconditions + /// + /// * `data` of this decoder should have >= `num * T::type_size()` bytes left to be decoded. + fn skip_batch(&mut self, num: usize); + + /// Returns the encoding for this decoder. + fn encoding(&self) -> Encoding; +} + +/// The switch off date between Julian and Gregorian calendar. See +/// https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html +const JULIAN_GREGORIAN_SWITCH_OFF_DAY: i32 = -141427; + +/// The switch off timestamp (in micros) between Julian and Gregorian calendar. See +/// https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html +const JULIAN_GREGORIAN_SWITCH_OFF_TS: i64 = -2208988800000000; + +/// See http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian +/// Also see Spark's `DateTimeUtils.JULIAN_DAY_OF_EPOCH` +const JULIAN_DAY_OF_EPOCH: i32 = 2440588; + +/// Number of micro seconds per milli second. +const MICROS_PER_MILLIS: i64 = 1000; + +const MICROS_PER_DAY: i64 = 24_i64 * 60 * 60 * 1000 * 1000; + +pub struct PlainDecoder<T: DataType> { + /// Internal states for this decoder. + inner: PlainDecoderInner, + + /// Marker to allow `T` in the generic parameter of the struct. + _phantom: PhantomData<T>, +} + +impl<T: DataType> PlainDecoder<T> { + pub fn new( + value_data: Buffer, + num_values: usize, + desc: ColumnDescPtr, + read_options: ReadOptions, + ) -> Self { + let len = value_data.len(); + let inner = PlainDecoderInner { + data: value_data.clone(), + offset: 0, + value_count: num_values, + bit_reader: BitReader::new(value_data, len), + read_options, + desc, + }; + Self { + inner, + _phantom: PhantomData, + } + } +} + +macro_rules! make_plain_default_impl { + ($($ty: ident), *) => { + $( + impl PlainDecoding for $ty { + /// Default implementation for PLAIN encoding. Uses `mempcy` when the physical + /// layout is the same between Parquet and Arrow. + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let src_data = &src.data; + let byte_width = src.desc.type_length() as usize; + let num_bytes = byte_width * num; + let dst_offset = byte_width * dst.num_values; + + bit::memcpy( + &src_data[src.offset..src.offset + num_bytes], + &mut dst.value_buffer[dst_offset..]); + src.offset += num_bytes; + } + + fn skip(src: &mut PlainDecoderInner, num: usize) { + let num_bytes = src.desc.type_length() as usize * num; + src.offset += num_bytes; + } + } + )* + }; +} + +make_plain_default_impl! { Int32Type, Int64Type, FloatType, DoubleType, FLBAType } + +macro_rules! make_plain_dict_impl { + ($($ty: ident), *) => { + $( + impl PlainDictDecoding for $ty { + fn decode_dict_one( + idx: usize, + val_idx: usize, + src: &ParquetMutableVector, + dst: &mut ParquetMutableVector, + bit_width: usize, + ) { + let byte_width = bit_width / 8; + bit::memcpy( + &src.value_buffer[val_idx * byte_width..(val_idx+1) * byte_width], + &mut dst.value_buffer[idx * byte_width..], + ); + } + } + )* + }; +} + +make_plain_dict_impl! { Int8Type, UInt8Type, Int16Type, UInt16Type, Int32Type, UInt32Type } +make_plain_dict_impl! { Int32DateType, Int64Type, FloatType, FLBAType } +make_plain_dict_impl! { DoubleType, Int64TimestampMillisType, Int64TimestampMicrosType } + +impl PlainDecoding for Int32To64Type { + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let src_ptr = src.data.as_ptr() as *const i32; + let dst_ptr = dst.value_buffer.as_mut_ptr() as *mut i64; + unsafe { + for i in 0..num { + dst_ptr + .add(dst.num_values + i) + .write_unaligned(src_ptr.add(src.offset + i).read_unaligned() as i64); + } + } + src.offset += 4 * num; + } + + fn skip(src: &mut PlainDecoderInner, num: usize) { + src.offset += 4 * num; + } +} + +impl PlainDictDecoding for Int32To64Type { + fn decode_dict_one( + idx: usize, + val_idx: usize, + src: &ParquetMutableVector, + dst: &mut ParquetMutableVector, + _: usize, + ) { + let src_ptr = src.value_buffer.as_ptr() as *const i32; + let dst_ptr = dst.value_buffer.as_mut_ptr() as *mut i64; + unsafe { + dst_ptr + .add(idx) + .write_unaligned(src_ptr.add(val_idx).read_unaligned() as i64); + } + } +} + +impl PlainDecoding for FloatToDoubleType { + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let src_ptr = src.data.as_ptr() as *const f32; + let dst_ptr = dst.value_buffer.as_mut_ptr() as *mut f64; + unsafe { + for i in 0..num { + dst_ptr + .add(dst.num_values + i) + .write_unaligned(src_ptr.add(src.offset + i).read_unaligned() as f64); + } + } + src.offset += 4 * num; + } + + fn skip(src: &mut PlainDecoderInner, num: usize) { + src.offset += 4 * num; + } +} + +impl PlainDictDecoding for FloatToDoubleType { + fn decode_dict_one( + idx: usize, + val_idx: usize, + src: &ParquetMutableVector, + dst: &mut ParquetMutableVector, + _: usize, + ) { + let src_ptr = src.value_buffer.as_ptr() as *const f32; + let dst_ptr = dst.value_buffer.as_mut_ptr() as *mut f64; + unsafe { + dst_ptr + .add(idx) + .write_unaligned(src_ptr.add(val_idx).read_unaligned() as f64); + } + } +} + +impl PlainDecoding for Int32DateType { + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let src_data = &src.data; + let byte_width = src.desc.type_length() as usize; + let num_bytes = byte_width * num; + let dst_offset = byte_width * dst.num_values; + + if !src.read_options.use_legacy_date_timestamp_or_ntz { + // By default we panic if the date value is before the switch date between Julian + // calendar and Gregorian calendar, which is 1582-10-15, and -141727 days + // before the unix epoch date 1970-01-01. + let mut offset = src.offset; + for _ in 0..num { + let v = &src_data[offset..offset + byte_width] as *const [u8] as *const u8 + as *const i32; + + // TODO: optimize this further as checking value one by one is not very efficient + unsafe { + if unlikely(v.read_unaligned() < JULIAN_GREGORIAN_SWITCH_OFF_DAY) { + panic!( + "Encountered date value {}, which is before 1582-10-15 (counting backwards \ + from Unix eopch date 1970-01-01), and could be ambigous depending on \ + whether a legacy Julian/Gregorian hybrid calendar is used, or a Proleptic \ + Gregorian calendar is used.", + *v + ); + } + } + + offset += byte_width; + } + } + + bit::memcpy( + &src_data[src.offset..src.offset + num_bytes], + &mut dst.value_buffer[dst_offset..], + ); + + src.offset += num_bytes; + } + + fn skip(src: &mut PlainDecoderInner, num: usize) { + let num_bytes = src.desc.type_length() as usize * num; + src.offset += num_bytes; + } +} + +impl PlainDecoding for Int64TimestampMillisType { + #[inline] + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let src_data = &src.data; + let byte_width = src.desc.type_length() as usize; + let num_bytes = byte_width * num; + + if !src.read_options.use_legacy_date_timestamp_or_ntz { + let mut offset = src.offset; + for _ in 0..num { + unsafe { + let v = &src_data[offset..offset + byte_width] as *const [u8] as *const u8 + as *const i64; + let v = v.read_unaligned() * MICROS_PER_MILLIS; + + // TODO: optimize this further as checking value one by one is not very + // efficient + if unlikely(v < JULIAN_GREGORIAN_SWITCH_OFF_TS) { + panic!( + "Encountered timestamp value {}, which is before 1582-10-15 (counting \ + backwards from Unix eopch date 1970-01-01), and could be ambigous \ + depending on whether a legacy Julian/Gregorian hybrid calendar is used, \ + or a Proleptic Gregorian calendar is used.", + v + ); + } + + offset += byte_width; + } + } + } + + unsafe { + let mut offset = src.offset; + let mut dst_offset = byte_width * dst.num_values; + for _ in 0..num { + let v = &src_data[offset..offset + byte_width] as *const [u8] as *const u8 + as *const i64; + let v = v.read_unaligned() * MICROS_PER_MILLIS; + bit::memcpy_value(&v, byte_width, &mut dst.value_buffer[dst_offset..]); + offset += byte_width; + dst_offset += byte_width; + } + } + + src.offset += num_bytes; + } + + #[inline] + fn skip(src: &mut PlainDecoderInner, num: usize) { + let num_bytes = src.desc.type_length() as usize * num; + src.offset += num_bytes; + } +} + +impl PlainDecoding for Int64TimestampMicrosType { + #[inline] + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let src_data = &src.data; + let byte_width = src.desc.type_length() as usize; + let num_bytes = byte_width * num; + let dst_offset = byte_width * dst.num_values; + + if !src.read_options.use_legacy_date_timestamp_or_ntz { + let mut offset = src.offset; + for _ in 0..num { + unsafe { + let v = &src_data[offset..offset + byte_width] as *const [u8] as *const u8 + as *const i64; + + // TODO: optimize this further as checking value one by one is not very + // efficient + if unlikely(v.read_unaligned() < JULIAN_GREGORIAN_SWITCH_OFF_TS) { + panic!( + "Encountered timestamp value {}, which is before 1582-10-15 (counting \ + backwards from Unix eopch date 1970-01-01), and could be ambigous \ + depending on whether a legacy Julian/Gregorian hybrid calendar is used, \ + or a Proleptic Gregorian calendar is used.", + *v + ); + } + + offset += byte_width; + } + } + } + + bit::memcpy( + &src_data[src.offset..src.offset + num_bytes], + &mut dst.value_buffer[dst_offset..], + ); + + src.offset += num_bytes; + } + + #[inline] + fn skip(src: &mut PlainDecoderInner, num: usize) { + let num_bytes = src.desc.type_length() as usize * num; + src.offset += num_bytes; + } +} + +impl PlainDecoding for BoolType { + /// Specific implementation for PLAIN encoding of boolean type. Even though both Parquet and + /// Arrow share the same physical layout for the type (which is 1 bit for each value), we'll + /// need to treat the number of bytes specifically. + #[inline] + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + src.bit_reader + .get_bits(&mut dst.value_buffer, dst.num_values, num); + } + + #[inline] + fn skip(src: &mut PlainDecoderInner, num: usize) { + src.bit_reader.skip_bits(num); + } +} + +// Does it make sense to encode booleans with dictionary? +impl PlainDictDecoding for BoolType { + #[inline] + fn decode_dict_one( + idx: usize, + val_idx: usize, + src: &ParquetMutableVector, + dst: &mut ParquetMutableVector, + _: usize, + ) { + let v = bit::get_bit(src.value_buffer.as_slice(), val_idx); + if v { + bit::set_bit(dst.value_buffer.as_slice_mut(), idx); + } // `dst` should be zero initialized so no need to call `unset_bit`. + } +} + +// Shared implementation for int variants such as Int8 and Int16 +macro_rules! make_int_variant_impl { + ($ty: ident, $native_ty: ty, $type_size: expr) => { + impl PlainDecoding for $ty { + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let num_bytes = 4 * num; // Parquet stores Int8/Int16 using 4 bytes + + let src_data = &src.data; + let mut src_offset = src.offset; + let dst_slice = dst.value_buffer.as_slice_mut(); + let mut dst_offset = dst.num_values * $type_size; + + let mut i = 0; + let mut in_ptr = &src_data[src_offset..] as *const [u8] as *const u8 as *const u32; + + while num - i >= 32 { + unsafe { + let in_slice = std::slice::from_raw_parts(in_ptr, 32); + + for n in 0..32 { + copy_nonoverlapping( + in_slice[n..].as_ptr() as *const $native_ty, + &mut dst_slice[dst_offset] as *mut u8 as *mut $native_ty, + 1, + ); + i += 1; + dst_offset += $type_size; + } + in_ptr = in_ptr.offset(32); + } + } + + src_offset += i * 4; + + (0..(num - i)).for_each(|_| { + unsafe { + copy_nonoverlapping( + &src_data[src_offset..] as *const [u8] as *const u8 + as *const $native_ty, + &mut dst_slice[dst_offset] as *mut u8 as *mut $native_ty, + 1, + ); + } + src_offset += 4; + dst_offset += $type_size; + }); + + src.offset += num_bytes; + } + + fn skip(src: &mut PlainDecoderInner, num: usize) { + let num_bytes = 4 * num; // Parquet stores Int8/Int16 using 4 bytes + src.offset += num_bytes; + } + } + }; +} + +make_int_variant_impl!(Int8Type, i8, 1); +make_int_variant_impl!(UInt8Type, u8, 2); +make_int_variant_impl!(Int16Type, i16, 2); +make_int_variant_impl!(UInt16Type, u16, 4); +make_int_variant_impl!(UInt32Type, u32, 8); + +// Shared implementation for variants of Binary type +macro_rules! make_plain_binary_impl { + ($($ty: ident), *) => { + $( + impl PlainDecoding for $ty { + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let src_data = &src.data; + let mut src_offset = src.offset; + + let mut offset_offset = dst.num_values * 4; + let offset_buf = &mut dst.value_buffer.as_slice_mut(); + let mut offset_value = read_num_bytes!(i32, 4, &offset_buf[offset_offset..]); + offset_offset += 4; + + // The actual content of a byte array is stored contiguously in the child vector + let child = &mut dst.children[0]; + let mut value_offset = child.num_values; // num_values == num of bytes + + (0..num).for_each(|_| { + let len = read_num_bytes!(i32, 4, &src_data[src_offset..]) as usize; + offset_value += len as i32; + + // Copy offset for the current string value into the offset buffer + bit::memcpy_value(&offset_value, 4, &mut offset_buf[offset_offset..]); + + // Reserve additional space in child value buffer if not enough + let value_buf_len = child.value_buffer.len(); + + if unlikely(value_buf_len < value_offset + len) { + let new_capacity = ::std::cmp::max(value_offset + len, value_buf_len * 2); + debug!("Reserving additional space ({} -> {} bytes) for value buffer", + value_buf_len, new_capacity); + child.value_buffer.resize(new_capacity); + } + + // Copy the actual string content into the value buffer + src_offset += mem::size_of::<u32>(); + bit::memcpy( + &src_data[src_offset..src_offset + len], + &mut child.value_buffer.as_slice_mut()[value_offset..], + ); + + value_offset += len; + src_offset += len; + offset_offset += 4; + }); + + src.offset = src_offset; + child.num_values = value_offset; + } + + fn skip(src: &mut PlainDecoderInner, num: usize) { + let src_data = &src.data; + let mut src_offset = src.offset; + + (0..num).for_each(|_| { + let len = read_num_bytes!(i32, 4, &src_data[src_offset..]) as usize; + src_offset += mem::size_of::<u32>(); + src_offset += len; + }); + + src.offset = src_offset; + } + } + )* + }; +} + +make_plain_binary_impl! { ByteArrayType, StringType } + +macro_rules! make_plain_dict_binary_impl { + ($($ty: ident), *) => { + $( + impl PlainDictDecoding for $ty { + #[inline] + fn decode_dict_one( + idx: usize, + val_idx: usize, + src: &ParquetMutableVector, + dst: &mut ParquetMutableVector, + _: usize, + ) { + debug_assert!(src.children.len() == 1); + debug_assert!(dst.children.len() == 1); + + let src_child = &src.children[0]; + let dst_child = &mut dst.children[0]; + + // get the offset & data for the binary value at index `val_idx` + let mut start_slice = &src.value_buffer[val_idx * 4..]; + let start = start_slice.get_u32_le() as usize; + let mut end_slice = &src.value_buffer[(val_idx + 1) * 4..]; + let end = end_slice.get_u32_le() as usize; + + debug_assert!(end >= start); + + let len = end - start; + let curr_offset = read_num_bytes!(u32, 4, &dst.value_buffer[idx * 4..]) as usize; + + // Reserve additional space in child value buffer if not enough + let value_buf_len = dst_child.value_buffer.len(); + + if unlikely(value_buf_len < curr_offset + len) { + let new_capacity = ::std::cmp::max(curr_offset + len, value_buf_len * 2); + debug!("Reserving additional space ({} -> {} bytes) for value buffer \ + during dictionary fallback", value_buf_len, + new_capacity); + dst_child.value_buffer.resize(new_capacity); + } + + bit::memcpy( + &src_child.value_buffer[start..end], + &mut dst_child.value_buffer[curr_offset..], + ); + + bit::memcpy_value( + &((curr_offset + len) as u32), + 4, + &mut dst.value_buffer[(idx + 1) * 4..], + ); + + dst_child.num_values += len; + } + } + )* + }; +} + +make_plain_dict_binary_impl! { ByteArrayType, StringType } + +macro_rules! make_plain_decimal_impl { + ($is_signed: expr, $($ty: ident; $need_convert: expr), *) => { + $( + impl PlainDecoding for $ty { + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let byte_width = src.desc.type_length() as usize; + + let src_data = &src.data[src.offset..]; + let dst_data = &mut dst.value_buffer[dst.num_values * DECIMAL_BYTE_WIDTH..]; + + let mut src_offset = 0; + let mut dst_offset = 0; + + debug_assert!(byte_width <= DECIMAL_BYTE_WIDTH); + + for _ in 0..num { + let s = &mut dst_data[dst_offset..]; + + bit::memcpy( + &src_data[src_offset..src_offset + byte_width], + s, + ); + + // Swap the order of bytes to make it little-endian. + if $need_convert { + for i in 0..byte_width / 2 { + s.swap(i, byte_width - i - 1); + } + } + + if $is_signed { + // Check if the most significant bit is 1 (negative in 2's complement). + // If so, also fill pad the remaining bytes with 0xff. + if s[byte_width - 1] & 0x80 == 0x80 { + s[byte_width..DECIMAL_BYTE_WIDTH].fill(0xff); + } + } + + src_offset += byte_width; + dst_offset += DECIMAL_BYTE_WIDTH; + } + + src.offset += num * byte_width; + } + + #[inline] + fn skip(src: &mut PlainDecoderInner, num: usize) { + let num_bytes_to_skip = num * src.desc.type_length() as usize; + src.offset += num_bytes_to_skip; + } + } + + impl PlainDictDecoding for $ty { + fn decode_dict_one(_: usize, val_idx: usize, src: &ParquetMutableVector, dst: &mut ParquetMutableVector, _: usize) { + let src_offset = val_idx * DECIMAL_BYTE_WIDTH; + let dst_offset = dst.num_values * DECIMAL_BYTE_WIDTH; + + bit::memcpy( + &src.value_buffer[src_offset..src_offset + DECIMAL_BYTE_WIDTH], + &mut dst.value_buffer[dst_offset..dst_offset + DECIMAL_BYTE_WIDTH], + ); + } + } + )* + } +} + +make_plain_decimal_impl!(true, Int32DecimalType; false, Int64DecimalType; false, FLBADecimalType; true); +make_plain_decimal_impl!(false, UInt64Type; false); + +macro_rules! make_plain_decimal_int_impl { + ($($ty: ident; $num_bytes: expr), *) => { + $( + impl PlainDecoding for $ty { + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let byte_width = src.desc.type_length() as usize; + let num_bits = 8 * byte_width; + + let src_data = &src.data[src.offset..]; + let dst_data = &mut dst.value_buffer[dst.num_values * $num_bytes..]; + + let mut src_offset = 0; + + for i in 0..num { + let mut unscaled: i64 = 0; + for _ in 0..byte_width { + unscaled = unscaled << 8 | src_data[src_offset] as i64; + src_offset += 1; + } + unscaled = (unscaled << (64 - num_bits)) >> (64 - num_bits); + bit::memcpy_value(&unscaled, $num_bytes, &mut dst_data[i * + $num_bytes..]); + } + + src.offset += num * byte_width; + } + + fn skip(src: &mut PlainDecoderInner, num: usize) { + let num_bytes_to_skip = num * src.desc.type_length() as usize; + src.offset += num_bytes_to_skip; + } + } + + impl PlainDictDecoding for $ty { + #[inline] + fn decode_dict_one(_: usize, val_idx: usize, src: &ParquetMutableVector, dst: &mut ParquetMutableVector, _: usize) { + bit::memcpy( + &src.value_buffer[val_idx * $num_bytes..(val_idx + 1) * $num_bytes], + &mut dst.value_buffer[dst.num_values * $num_bytes..], + ); + } + } + )* + }; +} + +make_plain_decimal_int_impl!(FLBADecimal32Type; 4, FLBADecimal64Type; 8); + +// Int96 contains 12 bytes +const INT96_SRC_BYTE_WIDTH: usize = 12; +// We convert INT96 to micros and store using i64 +const INT96_DST_BYTE_WIDTH: usize = 8; + +/// Decode timestamps represented as INT96 into i64 with micros precision +impl PlainDecoding for Int96TimestampMicrosType { + #[inline] + fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { + let src_data = &src.data; + + if !src.read_options.use_legacy_date_timestamp_or_ntz { + let mut offset = src.offset; + for _ in 0..num { + let v = &src_data[offset..offset + INT96_SRC_BYTE_WIDTH]; + let nanos = &v[..INT96_DST_BYTE_WIDTH] as *const [u8] as *const u8 as *const i64; + let day = &v[INT96_DST_BYTE_WIDTH..] as *const [u8] as *const u8 as *const i32; + + // TODO: optimize this further as checking value one by one is not very efficient + unsafe { + let micros = (day.read_unaligned() - JULIAN_DAY_OF_EPOCH) as i64 + * MICROS_PER_DAY + + nanos.read_unaligned() / 1000; + + if unlikely(micros < JULIAN_GREGORIAN_SWITCH_OFF_TS) { + panic!( + "Encountered timestamp value {}, which is before 1582-10-15 (counting \ + backwards from Unix eopch date 1970-01-01), and could be ambigous \ + depending on whether a legacy Julian/Gregorian hybrid calendar is used, \ + or a Proleptic Gregorian calendar is used.", + micros + ); + } + + offset += INT96_SRC_BYTE_WIDTH; + } + } + } + + let mut offset = src.offset; + let mut dst_offset = INT96_DST_BYTE_WIDTH * dst.num_values; + unsafe { + for _ in 0..num { + let v = &src_data[offset..offset + INT96_SRC_BYTE_WIDTH]; + let nanos = &v[..INT96_DST_BYTE_WIDTH] as *const [u8] as *const u8 as *const i64; + let day = &v[INT96_DST_BYTE_WIDTH..] as *const [u8] as *const u8 as *const i32; + + let micros = (day.read_unaligned() - JULIAN_DAY_OF_EPOCH) as i64 * MICROS_PER_DAY + + nanos.read_unaligned() / 1000; + + bit::memcpy_value( + µs, + INT96_DST_BYTE_WIDTH, + &mut dst.value_buffer[dst_offset..], + ); + + offset += INT96_SRC_BYTE_WIDTH; + dst_offset += INT96_DST_BYTE_WIDTH; + } + } + + src.offset = offset; + } + + #[inline] + fn skip(src: &mut PlainDecoderInner, num: usize) { + src.offset += INT96_SRC_BYTE_WIDTH * num; + } +} + +impl PlainDictDecoding for Int96TimestampMicrosType { + fn decode_dict_one( + _: usize, + val_idx: usize, + src: &ParquetMutableVector, + dst: &mut ParquetMutableVector, + _: usize, + ) { + let src_offset = val_idx * INT96_DST_BYTE_WIDTH; + let dst_offset = dst.num_values * INT96_DST_BYTE_WIDTH; + + bit::memcpy( + &src.value_buffer[src_offset..src_offset + INT96_DST_BYTE_WIDTH], + &mut dst.value_buffer[dst_offset..dst_offset + INT96_DST_BYTE_WIDTH], + ); + } +} + +impl<T: DataType> Decoder for PlainDecoder<T> { + #[inline] + fn read(&mut self, dst: &mut ParquetMutableVector) { + self.read_batch(dst, 1) + } + + /// Default implementation for PLAIN encoding, which uses a `memcpy` to copy from Parquet to the + /// Arrow vector. NOTE: this only works if the Parquet physical type has the same type width as + /// the Arrow's physical type (e.g., Parquet INT32 vs Arrow INT32). For other cases, we should + /// have special implementations. + #[inline] + fn read_batch(&mut self, dst: &mut ParquetMutableVector, num: usize) { + T::decode(&mut self.inner, dst, num); + } + + #[inline] + fn skip_batch(&mut self, num: usize) { + T::skip(&mut self.inner, num); + } + + #[inline] + fn encoding(&self) -> Encoding { + Encoding::PLAIN + } +} + +/// A decoder for Parquet dictionary indices, which is always of integer type, and encoded with +/// RLE/BitPacked encoding. +pub struct DictDecoder { + /// Number of bits used to represent dictionary indices. Must be between `[0, 64]`. + bit_width: usize, + + /// The number of total values in `data` + value_count: usize, + + /// Bit reader + bit_reader: BitReader, + + /// Number of values left in the current RLE run + rle_left: usize, + + /// Number of values left in the current BIT_PACKED run + bit_packed_left: usize, + + /// Current value in the RLE run. Unused if BIT_PACKED + current_value: u32, +} + +impl DictDecoder { + pub fn new(buf: Buffer, num_values: usize) -> Self { + let bit_width = buf.as_bytes()[0] as usize; + + Self { + bit_width, + value_count: num_values, + bit_reader: BitReader::new_all(buf.slice(1)), + rle_left: 0, + bit_packed_left: 0, + current_value: 0, + } + } +} + +impl DictDecoder { + /// Reads the header of the next RLE/BitPacked run, and update the internal state such as # of + /// values for the next run, as well as the current value in case it's RLE. + fn reload(&mut self) { + if let Some(indicator_value) = self.bit_reader.get_vlq_int() { + if indicator_value & 1 == 1 { + self.bit_packed_left = ((indicator_value >> 1) * 8) as usize; + } else { + self.rle_left = (indicator_value >> 1) as usize; + let value_width = bit::ceil(self.bit_width, 8); + self.current_value = self.bit_reader.get_aligned::<u32>(value_width).unwrap(); + } + } else { + panic!("Can't read VLQ int from BitReader"); + } + } +} + +impl Decoder for DictDecoder { + fn read(&mut self, dst: &mut ParquetMutableVector) { + let dst_slice = dst.value_buffer.as_slice_mut(); + let dst_offset = dst.num_values * 4; + + // We've finished the current run. Now load the next. + if self.rle_left == 0 && self.bit_packed_left == 0 { + self.reload(); + } + + let value = if self.rle_left > 0 { + self.rle_left -= 1; + self.current_value + } else { + self.bit_packed_left -= 1; + self.bit_reader.get_u32_value(self.bit_width) + }; + + // Directly copy the value into the destination buffer + unsafe { + let dst = &mut dst_slice[dst_offset..] as *mut [u8] as *mut u8 as *mut u32; + *dst = value; + } + } + + fn read_batch(&mut self, dst: &mut ParquetMutableVector, num: usize) { + let mut values_read = 0; + let dst_slice = dst.value_buffer.as_slice_mut(); + let mut dst_offset = dst.num_values * 4; + + while values_read < num { + let num_to_read = num - values_read; + let mut dst = &mut dst_slice[dst_offset..] as *mut [u8] as *mut u8 as *mut u32; + + if self.rle_left > 0 { + let n = std::cmp::min(num_to_read, self.rle_left); + unsafe { + // Copy the current RLE value into the destination buffer. + for _ in 0..n { + *dst = self.current_value; + dst = dst.offset(1); + } + dst_offset += 4 * n; + } + self.rle_left -= n; + values_read += n; + } else if self.bit_packed_left > 0 { + let n = std::cmp::min(num_to_read, self.bit_packed_left); + unsafe { + // Decode the next `n` BitPacked values into u32 and put the result directly to + // `dst`. + self.bit_reader.get_u32_batch(dst, n, self.bit_width); + } + dst_offset += 4 * n; + self.bit_packed_left -= n; + values_read += n; + } else { + self.reload(); + } + } + } + + fn skip_batch(&mut self, num: usize) { + let mut values_skipped = 0; + + while values_skipped < num { + let num_to_skip = num - values_skipped; + + if self.rle_left > 0 { + let n = std::cmp::min(num_to_skip, self.rle_left); + self.rle_left -= n; + values_skipped += n; + } else if self.bit_packed_left > 0 { + let n = std::cmp::min(num_to_skip, self.bit_packed_left); + self.bit_reader.skip_bits(n * self.bit_width); + self.bit_packed_left -= n; + values_skipped += n; + } else { + self.reload(); + } + } + } + + fn encoding(&self) -> Encoding { + Encoding::RLE_DICTIONARY + } +} diff --git a/core/src/parquet/util/bit_packing.rs b/core/src/parquet/util/bit_packing.rs new file mode 100644 index 0000000000..fd39ba8fa3 --- /dev/null +++ b/core/src/parquet/util/bit_packing.rs @@ -0,0 +1,3658 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +/// Unpack 32 values with bit width `num_bits` from `in_ptr`, and write to `out_ptr`. +/// Return the `in_ptr` where the starting offset points to the first byte after all the +/// bytes that were consumed. +// TODO: may be better to make these more compact using if-else conditions. +// However, this may require const generics: +// https://github.com/rust-lang/rust/issues/44580 +// to eliminate the branching cost. +// TODO: we should use SIMD instructions to further optimize this. I have explored +// https://github.com/tantivy-search/bitpacking +// but the layout it uses for SIMD is different from Parquet. +// TODO: support packing as well, which is used for encoding. +pub unsafe fn unpack32(mut in_ptr: *const u32, out_ptr: *mut u32, num_bits: usize) -> *const u32 { + in_ptr = match num_bits { + 0 => nullunpacker32(in_ptr, out_ptr), + 1 => unpack1_32(in_ptr, out_ptr), + 2 => unpack2_32(in_ptr, out_ptr), + 3 => unpack3_32(in_ptr, out_ptr), + 4 => unpack4_32(in_ptr, out_ptr), + 5 => unpack5_32(in_ptr, out_ptr), + 6 => unpack6_32(in_ptr, out_ptr), + 7 => unpack7_32(in_ptr, out_ptr), + 8 => unpack8_32(in_ptr, out_ptr), + 9 => unpack9_32(in_ptr, out_ptr), + 10 => unpack10_32(in_ptr, out_ptr), + 11 => unpack11_32(in_ptr, out_ptr), + 12 => unpack12_32(in_ptr, out_ptr), + 13 => unpack13_32(in_ptr, out_ptr), + 14 => unpack14_32(in_ptr, out_ptr), + 15 => unpack15_32(in_ptr, out_ptr), + 16 => unpack16_32(in_ptr, out_ptr), + 17 => unpack17_32(in_ptr, out_ptr), + 18 => unpack18_32(in_ptr, out_ptr), + 19 => unpack19_32(in_ptr, out_ptr), + 20 => unpack20_32(in_ptr, out_ptr), + 21 => unpack21_32(in_ptr, out_ptr), + 22 => unpack22_32(in_ptr, out_ptr), + 23 => unpack23_32(in_ptr, out_ptr), + 24 => unpack24_32(in_ptr, out_ptr), + 25 => unpack25_32(in_ptr, out_ptr), + 26 => unpack26_32(in_ptr, out_ptr), + 27 => unpack27_32(in_ptr, out_ptr), + 28 => unpack28_32(in_ptr, out_ptr), + 29 => unpack29_32(in_ptr, out_ptr), + 30 => unpack30_32(in_ptr, out_ptr), + 31 => unpack31_32(in_ptr, out_ptr), + 32 => unpack32_32(in_ptr, out_ptr), + _ => unimplemented!(), + }; + in_ptr +} + +unsafe fn nullunpacker32(in_buf: *const u32, mut out: *mut u32) -> *const u32 { + for _ in 0..32 { + *out = 0; + out = out.offset(1); + } + in_buf +} + +unsafe fn unpack1_32(in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 1) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 2) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 3) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 4) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 5) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 6) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 7) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 9) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 11) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 13) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 15) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 17) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 19) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 21) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 23) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 25) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 26) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 27) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 29) & 1; + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 30) & 1; + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + + in_buf.offset(1) +} + +unsafe fn unpack2_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 26) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 2); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + out = out.offset(1); + in_buf = in_buf.offset(1); + *out = (in_buf.read_unaligned()) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 26) % (1u32 << 2); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 2); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + + in_buf.offset(1) +} + +unsafe fn unpack3_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 21) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 27) % (1u32 << 3); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (3 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 25) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 3); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (3 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 23) % (1u32 << 3); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 26) % (1u32 << 3); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + + in_buf.offset(1) +} + +unsafe fn unpack4_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 4); + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 4); + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 4); + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 4); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 4); + + in_buf.offset(1) +} + +unsafe fn unpack5_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 25) % (1u32 << 5); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (5 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 23) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 5); + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (5 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 21) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 26) % (1u32 << 5); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (5 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 5); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (5 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 5); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 5); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + + in_buf.offset(1) +} + +unsafe fn unpack6_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 6); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (6 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 6); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (6 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 6); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 6); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (6 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 6); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (6 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 6); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 6); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + + in_buf.offset(1) +} + +unsafe fn unpack7_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 21) % (1u32 << 7); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (7 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 7); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (7 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 7); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (7 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 23) % (1u32 << 7); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (7 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 7); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (7 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 7); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (7 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 7); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 7); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + + in_buf.offset(1) +} + +unsafe fn unpack8_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + + in_buf.offset(1) +} + +unsafe fn unpack9_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (9 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (9 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (9 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 21) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (9 - 7); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (9 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (9 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (9 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (9 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 9); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 9); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 23; + + in_buf.offset(1) +} + +unsafe fn unpack10_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (10 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (10 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (10 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (10 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (10 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (10 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (10 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (10 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 10); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 10); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + + in_buf.offset(1) +} + +unsafe fn unpack11_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (11 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (11 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (11 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (11 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (11 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (11 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (11 - 7); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (11 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (11 - 9); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 11); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (11 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 11); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 21; + + in_buf.offset(1) +} + +unsafe fn unpack12_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 12); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (12 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 12); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (12 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 12); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (12 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 12); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (12 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 12); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (12 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 12); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (12 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 12); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (12 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 12); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (12 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 12); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + + in_buf.offset(1) +} + +unsafe fn unpack13_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 13); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (13 - 7); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (13 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 13); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (13 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (13 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 13); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (13 - 9); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (13 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 13); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (13 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (13 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 13); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (13 - 11); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (13 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 13); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (13 - 12); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (13 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 13); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 19; + + in_buf.offset(1) +} + +unsafe fn unpack14_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 14); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (14 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (14 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (14 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 14); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (14 - 12); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (14 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (14 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 14); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (14 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (14 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (14 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 14); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (14 - 12); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (14 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (14 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 14); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 18; + + in_buf.offset(1) +} + +unsafe fn unpack15_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 15); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (15 - 13); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (15 - 11); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (15 - 9); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (15 - 7); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (15 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (15 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (15 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 15); + out = out.offset(1); + *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (15 - 14); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (15 - 12); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (15 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (15 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (15 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (15 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (15 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 15); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 17; + + in_buf.offset(1) +} + +unsafe fn unpack16_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + out = out.offset(1); + in_buf = in_buf.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 16); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 16; + + in_buf.offset(1) +} + +unsafe fn unpack17_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 17; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (17 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (17 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (17 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (17 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (17 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (17 - 12); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (17 - 14); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (17 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (17 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (17 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (17 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (17 - 7); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (17 - 9); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (17 - 11); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (17 - 13); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 17); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (17 - 15); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 15; + + in_buf.offset(1) +} + +unsafe fn unpack18_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (18 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (18 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (18 - 12); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (18 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (18 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (18 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (18 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (18 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (18 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (18 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (18 - 12); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (18 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (18 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (18 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (18 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 18); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (18 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + + in_buf.offset(1) +} + +unsafe fn unpack19_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (19 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (19 - 12); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (19 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (19 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (19 - 11); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (19 - 17); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 17; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (19 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (19 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (19 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (19 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (19 - 9); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (19 - 15); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 15; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (19 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (19 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (19 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (19 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (19 - 7); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 19); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (19 - 13); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 13; + + in_buf.offset(1) +} + +unsafe fn unpack20_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (20 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (20 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (20 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (20 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (20 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (20 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (20 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (20 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (20 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (20 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (20 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (20 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (20 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (20 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (20 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 20); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (20 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + + in_buf.offset(1) +} + +unsafe fn unpack21_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (21 - 10); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (21 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (21 - 9); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (21 - 19); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (21 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (21 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (21 - 7); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (21 - 17); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 17; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (21 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (21 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (21 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (21 - 15); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 15; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (21 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (21 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (21 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (21 - 13); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 13; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (21 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (21 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (21 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 21); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (21 - 11); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 11; + + in_buf.offset(1) +} + +unsafe fn unpack22_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (22 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (22 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (22 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (22 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (22 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (22 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (22 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (22 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (22 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (22 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (22 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (22 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (22 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (22 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (22 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (22 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (22 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (22 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 22); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (22 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (22 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + + in_buf.offset(1) +} + +unsafe fn unpack23_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (23 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (23 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (23 - 19); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (23 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (23 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (23 - 15); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 15; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (23 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (23 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (23 - 11); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 11; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (23 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (23 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (23 - 7); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (23 - 21); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (23 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (23 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (23 - 17); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 17; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (23 - 8); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (23 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (23 - 13); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 13; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (23 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 23); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (23 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (23 - 9); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 9; + + in_buf.offset(1) +} + +unsafe fn unpack24_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 24); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 24); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 24); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 24); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 24); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 24); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 24); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 24); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + + in_buf.offset(1) +} + +unsafe fn unpack25_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 25); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (25 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (25 - 11); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 11; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (25 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 25); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (25 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (25 - 15); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 15; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (25 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (25 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 25); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (25 - 19); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (25 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (25 - 5); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 25); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 23)) << (25 - 23); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (25 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (25 - 9); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 9; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (25 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 25); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (25 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (25 - 13); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 13; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (25 - 6); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 25); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (25 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (25 - 17); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 17; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (25 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (25 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 25); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (25 - 21); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (25 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (25 - 7); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 7; + + in_buf.offset(1) +} + +unsafe fn unpack26_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 26); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (26 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (26 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (26 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (26 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 26); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (26 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (26 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (26 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (26 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 26); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (26 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (26 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (26 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (26 - 6); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 6; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 26); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (26 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (26 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (26 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (26 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 26); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (26 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (26 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (26 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (26 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 26); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (26 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (26 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (26 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (26 - 6); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 6; + + in_buf.offset(1) +} + +unsafe fn unpack27_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 27); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (27 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (27 - 17); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 17; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (27 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (27 - 7); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 7; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (27 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 27); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (27 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (27 - 19); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (27 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (27 - 9); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 9; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (27 - 4); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 27); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (27 - 26); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (27 - 21); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (27 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (27 - 11); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 11; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (27 - 6); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 6; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (27 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 27); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 23)) << (27 - 23); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (27 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (27 - 13); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 13; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (27 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (27 - 3); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 27); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 25)) << (27 - 25); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (27 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (27 - 15); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 15; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (27 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (27 - 5); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 5; + + in_buf.offset(1) +} + +unsafe fn unpack28_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 28); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (28 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (28 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (28 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (28 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (28 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (28 - 4); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 4; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 28); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (28 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (28 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (28 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (28 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (28 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (28 - 4); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 4; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 28); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (28 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (28 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (28 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (28 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (28 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (28 - 4); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 4; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 28); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (28 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (28 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (28 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (28 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (28 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (28 - 4); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 4; + + in_buf.offset(1) +} + +unsafe fn unpack29_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 29); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (29 - 26); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 23)) << (29 - 23); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (29 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (29 - 17); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 17; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (29 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (29 - 11); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 11; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (29 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (29 - 5); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 5; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (29 - 2); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 29); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 28)) << (29 - 28); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 25)) << (29 - 25); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (29 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (29 - 19); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (29 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (29 - 13); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 13; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (29 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (29 - 7); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 7; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (29 - 4); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 4; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (29 - 1); + out = out.offset(1); + + *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 29); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 27)) << (29 - 27); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (29 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (29 - 21); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (29 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (29 - 15); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 15; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (29 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (29 - 9); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 9; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (29 - 6); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 6; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (29 - 3); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 3; + + in_buf.offset(1) +} + +unsafe fn unpack30_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 30); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 28)) << (30 - 28); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (30 - 26); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (30 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (30 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (30 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (30 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (30 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (30 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (30 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (30 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (30 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (30 - 6); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 6; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (30 - 4); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 4; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (30 - 2); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 2; + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) % (1u32 << 30); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 28)) << (30 - 28); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (30 - 26); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (30 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (30 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (30 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (30 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (30 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (30 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (30 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (30 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (30 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (30 - 6); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 6; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (30 - 4); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 4; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (30 - 2); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 2; + + in_buf.offset(1) +} + +unsafe fn unpack31_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = (in_buf.read_unaligned()) % (1u32 << 31); + out = out.offset(1); + *out = (in_buf.read_unaligned()) >> 31; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 30)) << (31 - 30); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 30; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 29)) << (31 - 29); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 29; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 28)) << (31 - 28); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 28; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 27)) << (31 - 27); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 27; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (31 - 26); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 26; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 25)) << (31 - 25); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 25; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (31 - 24); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 24; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 23)) << (31 - 23); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 23; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (31 - 22); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 22; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (31 - 21); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 21; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (31 - 20); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 20; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (31 - 19); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 19; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (31 - 18); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 18; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (31 - 17); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 17; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (31 - 16); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 16; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (31 - 15); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 15; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (31 - 14); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 14; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (31 - 13); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 13; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (31 - 12); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 12; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (31 - 11); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 11; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (31 - 10); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 10; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (31 - 9); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 9; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (31 - 8); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 8; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (31 - 7); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 7; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (31 - 6); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 6; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (31 - 5); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 5; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (31 - 4); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 4; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (31 - 3); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 3; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (31 - 2); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 2; + in_buf = in_buf.offset(1); + *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (31 - 1); + out = out.offset(1); + + *out = (in_buf.read_unaligned()) >> 1; + + in_buf.offset(1) +} + +unsafe fn unpack32_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + in_buf = in_buf.offset(1); + out = out.offset(1); + + *out = in_buf.read_unaligned(); + + in_buf.offset(1) +} diff --git a/core/src/parquet/util/buffer.rs b/core/src/parquet/util/buffer.rs new file mode 100644 index 0000000000..d584ac0b12 --- /dev/null +++ b/core/src/parquet/util/buffer.rs @@ -0,0 +1,128 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::{ops::Index, slice::SliceIndex, sync::Arc}; + +/// An immutable byte buffer. +pub trait Buffer { + /// Returns the length (in bytes) of this buffer. + fn len(&self) -> usize; + + /// Returns the byte array of this buffer, in range `[0, len)`. + fn data(&self) -> &[u8]; + + /// Returns whether this buffer is empty or not. + fn is_empty(&self) -> bool { + self.len() == 0 + } +} + +impl Buffer for Vec<u8> { + fn len(&self) -> usize { + self.len() + } + + fn data(&self) -> &[u8] { + self + } +} + +pub struct BufferRef { + inner: Arc<dyn Buffer>, + offset: usize, + len: usize, +} + +impl BufferRef { + pub fn new(inner: Arc<dyn Buffer>) -> Self { + let len = inner.len(); + Self { + inner, + offset: 0, + len, + } + } + + /// Returns the length of this buffer. + #[inline] + pub fn len(&self) -> usize { + self.len + } + + #[inline] + pub fn is_empty(&self) -> bool { + self.len == 0 + } + + #[inline] + pub fn data(&self) -> &[u8] { + self.inner.data() + } + + /// Creates a new byte buffer containing elements in `[offset, offset+len)` + #[inline] + pub fn slice(&self, offset: usize, len: usize) -> BufferRef { + assert!( + self.offset + offset + len <= self.inner.len(), + "can't create a buffer slice with offset exceeding original \ + JNI array len {}, self.offset: {}, offset: {}, len: {}", + self.inner.len(), + self.offset, + offset, + len + ); + + Self { + inner: self.inner.clone(), + offset: self.offset + offset, + len, + } + } + + /// Creates a new byte buffer containing all elements starting from `offset` in this byte array. + #[inline] + pub fn start(&self, offset: usize) -> BufferRef { + assert!( + self.offset + offset <= self.inner.len(), + "can't create a buffer slice with offset exceeding original \ + JNI array len {}, self.offset: {}, offset: {}", + self.inner.len(), + self.offset, + offset + ); + let len = self.inner.len() - offset - self.offset; + self.slice(offset, len) + } +} + +impl AsRef<[u8]> for BufferRef { + fn as_ref(&self) -> &[u8] { + let slice = self.inner.as_ref().data(); + &slice[self.offset..self.offset + self.len] + } +} + +impl<Idx> Index<Idx> for BufferRef +where + Idx: SliceIndex<[u8]>, +{ + type Output = Idx::Output; + + fn index(&self, index: Idx) -> &Self::Output { + &self.as_ref()[index] + } +} diff --git a/core/src/parquet/util/hash_util.rs b/core/src/parquet/util/hash_util.rs new file mode 100644 index 0000000000..102e01ffc8 --- /dev/null +++ b/core/src/parquet/util/hash_util.rs @@ -0,0 +1,167 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +fn hash_(data: &[u8], seed: u32) -> u32 { + #[cfg(any(target_arch = "x86", target_arch = "x86_64"))] + unsafe { + if is_x86_feature_detected!("sse4.2") { + crc32_hash(data, seed) + } else { + murmur_hash2_64a(data, seed as u64) as u32 + } + } + + #[cfg(any( + target_arch = "aarch64", + target_arch = "arm", + target_arch = "riscv64", + target_arch = "wasm32" + ))] + unsafe { + murmur_hash2_64a(data, seed as u64) as u32 + } +} + +const MURMUR_PRIME: u64 = 0xc6a4a7935bd1e995; +const MURMUR_R: i32 = 47; + +/// Rust implementation of MurmurHash2, 64-bit version for 64-bit platforms +/// +/// SAFTETY Only safe on platforms which support unaligned loads (like x86_64) +unsafe fn murmur_hash2_64a(data_bytes: &[u8], seed: u64) -> u64 { + let len = data_bytes.len(); + let len_64 = (len / 8) * 8; + let data_bytes_64 = + std::slice::from_raw_parts(&data_bytes[0..len_64] as *const [u8] as *const u64, len / 8); + + let mut h = seed ^ (MURMUR_PRIME.wrapping_mul(data_bytes.len() as u64)); + for v in data_bytes_64 { + let mut k = *v; + k = k.wrapping_mul(MURMUR_PRIME); + k ^= k >> MURMUR_R; + k = k.wrapping_mul(MURMUR_PRIME); + h ^= k; + h = h.wrapping_mul(MURMUR_PRIME); + } + + let data2 = &data_bytes[len_64..]; + + let v = len & 7; + if v == 7 { + h ^= (data2[6] as u64) << 48; + } + if v >= 6 { + h ^= (data2[5] as u64) << 40; + } + if v >= 5 { + h ^= (data2[4] as u64) << 32; + } + if v >= 4 { + h ^= (data2[3] as u64) << 24; + } + if v >= 3 { + h ^= (data2[2] as u64) << 16; + } + if v >= 2 { + h ^= (data2[1] as u64) << 8; + } + if v >= 1 { + h ^= data2[0] as u64; + } + if v > 0 { + h = h.wrapping_mul(MURMUR_PRIME); + } + + h ^= h >> MURMUR_R; + h = h.wrapping_mul(MURMUR_PRIME); + h ^= h >> MURMUR_R; + h +} + +/// CRC32 hash implementation using SSE4 instructions. Borrowed from Impala. +#[cfg(any(target_arch = "x86", target_arch = "x86_64"))] +#[target_feature(enable = "sse4.2")] +unsafe fn crc32_hash(bytes: &[u8], seed: u32) -> u32 { + #[cfg(target_arch = "x86")] + use std::arch::x86::*; + #[cfg(target_arch = "x86_64")] + use std::arch::x86_64::*; + + let u32_num_bytes = std::mem::size_of::<u32>(); + let mut num_bytes = bytes.len(); + let num_words = num_bytes / u32_num_bytes; + num_bytes %= u32_num_bytes; + + let bytes_u32: &[u32] = std::slice::from_raw_parts( + &bytes[0..num_words * u32_num_bytes] as *const [u8] as *const u32, + num_words, + ); + + let mut offset = 0; + let mut hash = seed; + while offset < num_words { + hash = _mm_crc32_u32(hash, bytes_u32[offset]); + offset += 1; + } + + offset = num_words * u32_num_bytes; + while offset < num_bytes { + hash = _mm_crc32_u8(hash, bytes[offset]); + offset += 1; + } + + // The lower half of the CRC hash has poor uniformity, so swap the halves + // for anyone who only uses the first several bits of the hash. + hash = (hash << 16) | (hash >> 16); + hash +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_murmur2_64a() { + unsafe { + let result = murmur_hash2_64a(b"hello", 123); + assert_eq!(result, 2597646618390559622); + + let result = murmur_hash2_64a(b"helloworld", 123); + assert_eq!(result, 4934371746140206573); + + let result = murmur_hash2_64a(b"helloworldparquet", 123); + assert_eq!(result, 2392198230801491746); + } + } + + #[test] + #[cfg(any(target_arch = "x86", target_arch = "x86_64"))] + fn test_crc32() { + if is_x86_feature_detected!("sse4.2") { + unsafe { + let result = crc32_hash(b"hello", 123); + assert_eq!(result, 2927487359); + + let result = crc32_hash(b"helloworld", 123); + assert_eq!(result, 314229527); + + let result = crc32_hash(b"helloworldparquet", 123); + assert_eq!(result, 667078870); + } + } + } +} diff --git a/core/src/parquet/util/jni.rs b/core/src/parquet/util/jni.rs new file mode 100644 index 0000000000..000eeee0bf --- /dev/null +++ b/core/src/parquet/util/jni.rs @@ -0,0 +1,199 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 jni::{ + errors::Result as JNIResult, + objects::{JMethodID, JString}, + sys::{jboolean, jint, jobjectArray, jstring}, + JNIEnv, +}; + +use parquet::{ + basic::{Encoding, LogicalType, TimeUnit, Type as PhysicalType}, + format::{MicroSeconds, MilliSeconds, NanoSeconds}, + schema::types::{ColumnDescriptor, ColumnPath, PrimitiveTypeBuilder}, +}; + +/// Convert primitives from Spark side into a `ColumnDescriptor`. +#[allow(clippy::too_many_arguments)] +pub fn convert_column_descriptor( + env: &JNIEnv, + physical_type_id: jint, + logical_type_id: jint, + max_dl: jint, + max_rl: jint, + bit_width: jint, + is_signed: jboolean, + type_length: jint, + precision: jint, + scale: jint, + time_unit: jint, + is_adjusted_utc: jboolean, + jni_path: jobjectArray, +) -> JNIResult<ColumnDescriptor> { + let physical_type = convert_physical_type(physical_type_id); + let type_length = fix_type_length(&physical_type, type_length); + let logical_type = if logical_type_id >= 0 { + Some(convert_logical_type( + logical_type_id, + bit_width, + is_signed, + precision, + scale, + time_unit, + is_adjusted_utc, + )) + } else { + // id < 0 means there is no logical type associated + None + }; + + // We don't care the column name here + let ty = PrimitiveTypeBuilder::new("f", physical_type) + .with_logical_type(logical_type) + .with_length(type_length) + .with_precision(precision) // Parquet crate requires to set this even with logical type + .with_scale(scale) + .build() + .unwrap(); // TODO: convert Parquet errot to JNI error + let path = convert_column_path(env, jni_path).unwrap(); + + let result = ColumnDescriptor::new(Arc::new(ty), max_dl as i16, max_rl as i16, path); + Ok(result) +} + +pub fn convert_encoding(ordinal: jint) -> Encoding { + match ordinal { + 0 => Encoding::PLAIN, + 1 => Encoding::RLE, + 3 => Encoding::BIT_PACKED, + 4 => Encoding::PLAIN_DICTIONARY, + 5 => Encoding::DELTA_BINARY_PACKED, + 6 => Encoding::DELTA_LENGTH_BYTE_ARRAY, + 7 => Encoding::DELTA_BYTE_ARRAY, + 8 => Encoding::RLE_DICTIONARY, + _ => panic!("Invalid Java Encoding ordinal: {}", ordinal), + } +} + +pub struct TypePromotionInfo { + pub(crate) physical_type: PhysicalType, + pub(crate) precision: i32, +} + +impl TypePromotionInfo { + pub fn new_from_jni(physical_type_id: jint, precision: jint) -> Self { + let physical_type = convert_physical_type(physical_type_id); + Self { + physical_type, + precision, + } + } + + pub fn new(physical_type: PhysicalType, precision: i32) -> Self { + Self { + physical_type, + precision, + } + } +} + +fn convert_column_path(env: &JNIEnv, path: jobjectArray) -> JNIResult<ColumnPath> { + let array_len = env.get_array_length(path)?; + let mut res: Vec<String> = Vec::new(); + for i in 0..array_len { + let p: JString = (env.get_object_array_element(path, i)?.into_inner() as jstring).into(); + res.push(env.get_string(p)?.into()); + } + Ok(ColumnPath::new(res)) +} + +fn convert_physical_type(id: jint) -> PhysicalType { + match id { + 0 => PhysicalType::BOOLEAN, + 1 => PhysicalType::INT32, + 2 => PhysicalType::INT64, + 3 => PhysicalType::INT96, + 4 => PhysicalType::FLOAT, + 5 => PhysicalType::DOUBLE, + 6 => PhysicalType::BYTE_ARRAY, + 7 => PhysicalType::FIXED_LEN_BYTE_ARRAY, + _ => panic!("Invalid id for Parquet physical type: {} ", id), + } +} + +fn convert_logical_type( + id: jint, + bit_width: jint, + is_signed: jboolean, + precision: jint, + scale: jint, + time_unit: jint, + is_adjusted_utc: jboolean, +) -> LogicalType { + match id { + 0 => LogicalType::Integer { + bit_width: bit_width as i8, + is_signed: is_signed != 0, + }, + 1 => LogicalType::String, + 2 => LogicalType::Decimal { scale, precision }, + 3 => LogicalType::Date, + 4 => LogicalType::Timestamp { + is_adjusted_to_u_t_c: is_adjusted_utc != 0, + unit: convert_time_unit(time_unit), + }, + 5 => LogicalType::Enum, + 6 => LogicalType::Uuid, + _ => panic!("Invalid id for Parquet logical type: {}", id), + } +} + +fn convert_time_unit(time_unit: jint) -> TimeUnit { + match time_unit { + 0 => TimeUnit::MILLIS(MilliSeconds::new()), + 1 => TimeUnit::MICROS(MicroSeconds::new()), + 2 => TimeUnit::NANOS(NanoSeconds::new()), + _ => panic!("Invalid time unit id for Parquet: {}", time_unit), + } +} + +/// Fixes the type length in case they are not set (Parquet only explicitly set it for +/// FIXED_LEN_BYTE_ARRAY type). +fn fix_type_length(t: &PhysicalType, type_length: i32) -> i32 { + match t { + PhysicalType::INT32 | PhysicalType::FLOAT => 4, + PhysicalType::INT64 | PhysicalType::DOUBLE => 8, + PhysicalType::INT96 => 12, + _ => type_length, + } +} + +fn get_method_id<'a>(env: &'a JNIEnv, class: &'a str, method: &str, sig: &str) -> JMethodID<'a> { + // first verify the class exists + let _ = env + .find_class(class) + .unwrap_or_else(|_| panic!("Class '{}' not found", class)); + env.get_method_id(class, method, sig).unwrap_or_else(|_| { + panic!( + "Method '{}' with signature '{}' of class '{}' not found", + method, sig, class + ) + }) +} diff --git a/core/src/parquet/util/jni_buffer.rs b/core/src/parquet/util/jni_buffer.rs new file mode 100644 index 0000000000..33f36ed9dd --- /dev/null +++ b/core/src/parquet/util/jni_buffer.rs @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use core::slice; +use std::ptr::NonNull; + +use jni::{ + objects::{ReleaseMode, TypeArray}, + sys::{jbyte, jbyteArray, JNI_TRUE}, + JavaVM, +}; + +use crate::errors::{CometError, CometResult as Result}; + +use super::Buffer; + +/// An immutable byte buffer wrapping a JNI byte array allocated on heap. +/// +/// Unlike `AutoArray`, this doesn't have a lifetime and can be used across different JNI calls. +pub struct JniBuffer { + /// A pointer for the JVM instance, used to obtain byte array elements (via + /// `GetByteArrayElements`) and release byte array elements (via `ReleaseByteArrayElements`). + jvm: JavaVM, + /// The original JNI byte array that backs this buffer + inner: jbyteArray, + /// The raw pointer from the JNI byte array + ptr: NonNull<i8>, + /// Total number of bytes in the original array (i.e., `inner`). + len: usize, + /// Whether the JNI byte array is copied or not. + is_copy: bool, +} + +impl JniBuffer { + pub fn try_new(jvm: JavaVM, array: jbyteArray, len: usize) -> Result<Self> { + let env = jvm.get_env()?; + let mut is_copy = 0xff; + let ptr = jbyte::get(&env, array.into(), &mut is_copy)?; + let res = Self { + jvm, + inner: array, + ptr: NonNull::new(ptr) + .ok_or_else(|| CometError::NullPointer("null byte array pointer".to_string()))?, + len, + is_copy: is_copy == JNI_TRUE, + }; + Ok(res) + } + + /// Whether the JNI byte array is copied or not, i.e., whether the JVM pinned down the original + /// Java byte array, or made a new copy of it. + pub fn is_copy(&self) -> bool { + self.is_copy + } +} + +impl Buffer for JniBuffer { + fn len(&self) -> usize { + self.len + } + + fn data(&self) -> &[u8] { + self.as_ref() + } +} + +impl AsRef<[u8]> for JniBuffer { + fn as_ref(&self) -> &[u8] { + unsafe { slice::from_raw_parts(self.ptr.as_ptr() as *mut u8 as *const u8, self.len) } + } +} + +impl Drop for JniBuffer { + fn drop(&mut self) { + let env = self.jvm.get_env().unwrap(); // TODO: log error here + jbyte::release( + &env, + self.inner.into(), + self.ptr, + ReleaseMode::NoCopyBack as i32, // don't copy back since it's read-only here + ) + .unwrap(); + } +} diff --git a/core/src/parquet/util/memory.rs b/core/src/parquet/util/memory.rs new file mode 100644 index 0000000000..a2bbbfdde9 --- /dev/null +++ b/core/src/parquet/util/memory.rs @@ -0,0 +1,557 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Utility methods and structs for working with memory. + +use std::{ + fmt::{Debug, Display, Formatter, Result as FmtResult}, + io::{Result as IoResult, Write}, + mem, + ops::{Index, IndexMut}, + sync::{ + atomic::{AtomicI64, Ordering}, + Arc, Weak, + }, +}; + +// ---------------------------------------------------------------------- +// Memory Tracker classes + +/// Reference counted pointer for [`MemTracker`]. +pub type MemTrackerPtr = Arc<MemTracker>; +/// Non-owning reference for [`MemTracker`]. +pub type WeakMemTrackerPtr = Weak<MemTracker>; + +/// Struct to track memory usage information. +#[derive(Debug)] +pub struct MemTracker { + // In the tuple, the first element is the current memory allocated (in bytes), + // and the second element is the maximum memory allocated so far (in bytes). + current_memory_usage: AtomicI64, + max_memory_usage: AtomicI64, +} + +impl MemTracker { + /// Creates new memory tracker. + #[inline] + pub fn new() -> MemTracker { + MemTracker { + current_memory_usage: Default::default(), + max_memory_usage: Default::default(), + } + } + + /// Returns the current memory consumption, in bytes. + pub fn memory_usage(&self) -> i64 { + self.current_memory_usage.load(Ordering::Acquire) + } + + /// Returns the maximum memory consumption so far, in bytes. + pub fn max_memory_usage(&self) -> i64 { + self.max_memory_usage.load(Ordering::Acquire) + } + + /// Adds `num_bytes` to the memory consumption tracked by this memory tracker. + #[inline] + pub fn alloc(&self, num_bytes: i64) { + let new_current = self + .current_memory_usage + .fetch_add(num_bytes, Ordering::Acquire) + + num_bytes; + self.max_memory_usage + .fetch_max(new_current, Ordering::Acquire); + } +} + +impl Default for MemTracker { + fn default() -> Self { + Self::new() + } +} + +// ---------------------------------------------------------------------- +// Buffer classes + +/// Type alias for [`Buffer`]. +pub type ByteBuffer = Buffer<u8>; +/// Type alias for [`BufferPtr`]. +pub type ByteBufferPtr = BufferPtr<u8>; + +/// A resize-able buffer class with generic member, with optional memory tracker. +/// +/// Note that a buffer has two attributes: +/// `capacity` and `size`: the former is the total number of space reserved for +/// the buffer, while the latter is the actual number of elements. +/// Invariant: `capacity` >= `size`. +/// The total allocated bytes for a buffer equals to `capacity * sizeof<T>()`. +pub struct Buffer<T: Clone> { + data: Vec<T>, + mem_tracker: Option<MemTrackerPtr>, + type_length: usize, +} + +impl<T: Clone> Buffer<T> { + /// Creates new empty buffer. + pub fn new() -> Self { + Buffer { + data: vec![], + mem_tracker: None, + type_length: std::mem::size_of::<T>(), + } + } + + /// Adds [`MemTracker`] for this buffer. + #[inline] + pub fn with_mem_tracker(mut self, mc: MemTrackerPtr) -> Self { + mc.alloc((self.data.capacity() * self.type_length) as i64); + self.mem_tracker = Some(mc); + self + } + + /// Returns slice of data in this buffer. + #[inline] + pub fn data(&self) -> &[T] { + self.data.as_slice() + } + + /// Sets data for this buffer. + #[inline] + pub fn set_data(&mut self, new_data: Vec<T>) { + if let Some(ref mc) = self.mem_tracker { + let capacity_diff = new_data.capacity() as i64 - self.data.capacity() as i64; + mc.alloc(capacity_diff * self.type_length as i64); + } + self.data = new_data; + } + + /// Resizes underlying data in place to a new length `new_size`. + /// + /// If `new_size` is less than current length, data is truncated, otherwise, it is + /// extended to `new_size` with provided default value `init_value`. + /// + /// Memory tracker is also updated, if available. + #[inline] + pub fn resize(&mut self, new_size: usize, init_value: T) { + let old_capacity = self.data.capacity(); + self.data.resize(new_size, init_value); + if let Some(ref mc) = self.mem_tracker { + let capacity_diff = self.data.capacity() as i64 - old_capacity as i64; + mc.alloc(capacity_diff * self.type_length as i64); + } + } + + /// Clears underlying data. + #[inline] + pub fn clear(&mut self) { + self.data.clear() + } + + /// Reserves capacity `additional_capacity` for underlying data vector. + /// + /// Memory tracker is also updated, if available. + #[inline] + pub fn reserve(&mut self, additional_capacity: usize) { + let old_capacity = self.data.capacity(); + self.data.reserve(additional_capacity); + if self.data.capacity() > old_capacity { + if let Some(ref mc) = self.mem_tracker { + let capacity_diff = self.data.capacity() as i64 - old_capacity as i64; + mc.alloc(capacity_diff * self.type_length as i64); + } + } + } + + /// Returns [`BufferPtr`] with buffer data. + /// Buffer data is reset. + #[inline] + pub fn consume(&mut self) -> BufferPtr<T> { + let old_data = mem::take(&mut self.data); + let mut result = BufferPtr::new(old_data); + if let Some(ref mc) = self.mem_tracker { + result = result.with_mem_tracker(mc.clone()); + } + result + } + + /// Adds `value` to the buffer. + #[inline] + pub fn push(&mut self, value: T) { + self.data.push(value) + } + + /// Returns current capacity for the buffer. + #[inline] + pub fn capacity(&self) -> usize { + self.data.capacity() + } + + /// Returns current size for the buffer. + #[inline] + pub fn size(&self) -> usize { + self.data.len() + } + + /// Returns `true` if memory tracker is added to buffer, `false` otherwise. + #[inline] + pub fn is_mem_tracked(&self) -> bool { + self.mem_tracker.is_some() + } + + /// Returns memory tracker associated with this buffer. + /// This may panic, if memory tracker is not set, use method above to check if + /// memory tracker is available. + #[inline] + pub fn mem_tracker(&self) -> &MemTrackerPtr { + self.mem_tracker.as_ref().unwrap() + } +} + +impl<T: Clone> Default for Buffer<T> { + fn default() -> Self { + Self::new() + } +} + +impl<T: Sized + Clone> Index<usize> for Buffer<T> { + type Output = T; + + fn index(&self, index: usize) -> &T { + &self.data[index] + } +} + +impl<T: Sized + Clone> IndexMut<usize> for Buffer<T> { + fn index_mut(&mut self, index: usize) -> &mut T { + &mut self.data[index] + } +} + +// TODO: implement this for other types +impl Write for Buffer<u8> { + #[inline] + fn write(&mut self, buf: &[u8]) -> IoResult<usize> { + let old_capacity = self.data.capacity(); + let bytes_written = self.data.write(buf)?; + if let Some(ref mc) = self.mem_tracker { + if self.data.capacity() - old_capacity > 0 { + mc.alloc((self.data.capacity() - old_capacity) as i64) + } + } + Ok(bytes_written) + } + + fn flush(&mut self) -> IoResult<()> { + // No-op + self.data.flush() + } +} + +impl AsRef<[u8]> for Buffer<u8> { + fn as_ref(&self) -> &[u8] { + self.data.as_slice() + } +} + +impl<T: Clone> Drop for Buffer<T> { + #[inline] + fn drop(&mut self) { + if let Some(ref mc) = self.mem_tracker { + mc.alloc(-((self.data.capacity() * self.type_length) as i64)); + } + } +} + +// ---------------------------------------------------------------------- +// Immutable Buffer (BufferPtr) classes + +/// An representation of a slice on a reference-counting and read-only byte array. +/// Sub-slices can be further created from this. The byte array will be released +/// when all slices are dropped. +#[allow(clippy::rc_buffer)] +#[derive(Clone, Debug)] +pub struct BufferPtr<T> { + data: Arc<Vec<T>>, + start: usize, + len: usize, + // TODO: will this create too many references? rethink about this. + mem_tracker: Option<MemTrackerPtr>, +} + +impl<T> BufferPtr<T> { + /// Creates new buffer from a vector. + pub fn new(v: Vec<T>) -> Self { + let len = v.len(); + Self { + data: Arc::new(v), + start: 0, + len, + mem_tracker: None, + } + } + + /// Returns slice of data in this buffer. + #[inline] + pub fn data(&self) -> &[T] { + &self.data[self.start..self.start + self.len] + } + + /// Updates this buffer with new `start` position and length `len`. + /// + /// Range should be within current start position and length. + #[inline] + pub fn with_range(mut self, start: usize, len: usize) -> Self { + self.set_range(start, len); + self + } + + /// Updates this buffer with new `start` position and length `len`. + /// + /// Range should be within current start position and length. + #[inline] + pub fn set_range(&mut self, start: usize, len: usize) { + assert!(self.start <= start && start + len <= self.start + self.len); + self.start = start; + self.len = len; + } + + /// Adds memory tracker to this buffer. + pub fn with_mem_tracker(mut self, mc: MemTrackerPtr) -> Self { + self.mem_tracker = Some(mc); + self + } + + /// Returns start position of this buffer. + #[inline] + pub fn start(&self) -> usize { + self.start + } + + /// Returns length of this buffer + #[inline] + pub fn len(&self) -> usize { + self.len + } + + /// Returns whether this buffer is empty + #[inline] + pub fn is_empty(&self) -> bool { + self.len == 0 + } + + /// Returns `true` if this buffer has memory tracker, `false` otherwise. + pub fn is_mem_tracked(&self) -> bool { + self.mem_tracker.is_some() + } + + /// Returns a shallow copy of the buffer. + /// Reference counted pointer to the data is copied. + pub fn all(&self) -> BufferPtr<T> { + BufferPtr { + data: self.data.clone(), + start: self.start, + len: self.len, + mem_tracker: self.mem_tracker.as_ref().cloned(), + } + } + + /// Returns a shallow copy of the buffer that starts with `start` position. + pub fn start_from(&self, start: usize) -> BufferPtr<T> { + assert!(start <= self.len); + BufferPtr { + data: self.data.clone(), + start: self.start + start, + len: self.len - start, + mem_tracker: self.mem_tracker.as_ref().cloned(), + } + } + + /// Returns a shallow copy that is a range slice within this buffer. + pub fn range(&self, start: usize, len: usize) -> BufferPtr<T> { + assert!(start + len <= self.len); + BufferPtr { + data: self.data.clone(), + start: self.start + start, + len, + mem_tracker: self.mem_tracker.as_ref().cloned(), + } + } +} + +impl<T: Sized> Index<usize> for BufferPtr<T> { + type Output = T; + + fn index(&self, index: usize) -> &T { + assert!(index < self.len); + &self.data[self.start + index] + } +} + +impl<T: Debug> Display for BufferPtr<T> { + fn fmt(&self, f: &mut Formatter) -> FmtResult { + write!(f, "{:?}", self.data) + } +} + +impl<T> Drop for BufferPtr<T> { + fn drop(&mut self) { + if let Some(ref mc) = self.mem_tracker { + if Arc::strong_count(&self.data) == 1 && Arc::weak_count(&self.data) == 0 { + mc.alloc(-(self.data.capacity() as i64)); + } + } + } +} + +impl AsRef<[u8]> for BufferPtr<u8> { + #[inline] + fn as_ref(&self) -> &[u8] { + &self.data[self.start..self.start + self.len] + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_byte_buffer_mem_tracker() { + let mem_tracker = Arc::new(MemTracker::new()); + + let mut buffer = ByteBuffer::new().with_mem_tracker(mem_tracker.clone()); + buffer.set_data(vec![0; 10]); + assert_eq!(mem_tracker.memory_usage(), buffer.capacity() as i64); + buffer.set_data(vec![0; 20]); + let capacity = buffer.capacity() as i64; + assert_eq!(mem_tracker.memory_usage(), capacity); + + let max_capacity = { + let mut buffer2 = ByteBuffer::new().with_mem_tracker(mem_tracker.clone()); + buffer2.reserve(30); + assert_eq!( + mem_tracker.memory_usage(), + buffer2.capacity() as i64 + capacity + ); + buffer2.set_data(vec![0; 100]); + assert_eq!( + mem_tracker.memory_usage(), + buffer2.capacity() as i64 + capacity + ); + buffer2.capacity() as i64 + capacity + }; + + assert_eq!(mem_tracker.memory_usage(), capacity); + assert_eq!(mem_tracker.max_memory_usage(), max_capacity); + + buffer.reserve(40); + assert_eq!(mem_tracker.memory_usage(), buffer.capacity() as i64); + + buffer.consume(); + assert_eq!(mem_tracker.memory_usage(), buffer.capacity() as i64); + } + + #[test] + fn test_byte_ptr_mem_tracker() { + let mem_tracker = Arc::new(MemTracker::new()); + + let mut buffer = ByteBuffer::new().with_mem_tracker(mem_tracker.clone()); + buffer.set_data(vec![0; 60]); + + { + let buffer_capacity = buffer.capacity() as i64; + let buf_ptr = buffer.consume(); + assert_eq!(mem_tracker.memory_usage(), buffer_capacity); + { + let buf_ptr1 = buf_ptr.all(); + { + let _ = buf_ptr.start_from(20); + assert_eq!(mem_tracker.memory_usage(), buffer_capacity); + } + assert_eq!(mem_tracker.memory_usage(), buffer_capacity); + let _ = buf_ptr1.range(30, 20); + assert_eq!(mem_tracker.memory_usage(), buffer_capacity); + } + assert_eq!(mem_tracker.memory_usage(), buffer_capacity); + } + assert_eq!(mem_tracker.memory_usage(), buffer.capacity() as i64); + } + + #[test] + fn test_byte_buffer() { + let mut buffer = ByteBuffer::new(); + assert_eq!(buffer.size(), 0); + assert_eq!(buffer.capacity(), 0); + + let mut buffer2 = ByteBuffer::new(); + buffer2.reserve(40); + assert_eq!(buffer2.size(), 0); + assert_eq!(buffer2.capacity(), 40); + + buffer.set_data((0..5).collect()); + assert_eq!(buffer.size(), 5); + assert_eq!(buffer[4], 4); + + buffer.set_data((0..20).collect()); + assert_eq!(buffer.size(), 20); + assert_eq!(buffer[10], 10); + + let expected: Vec<u8> = (0..20).collect(); + { + let data = buffer.data(); + assert_eq!(data, expected.as_slice()); + } + + buffer.reserve(40); + assert!(buffer.capacity() >= 40); + + let byte_ptr = buffer.consume(); + assert_eq!(buffer.size(), 0); + assert_eq!(byte_ptr.as_ref(), expected.as_slice()); + + let values: Vec<u8> = (0..30).collect(); + let _ = buffer.write(values.as_slice()); + let _ = buffer.flush(); + + assert_eq!(buffer.data(), values.as_slice()); + } + + #[test] + fn test_byte_ptr() { + let values = (0..50).collect(); + let ptr = ByteBufferPtr::new(values); + assert_eq!(ptr.len(), 50); + assert_eq!(ptr.start(), 0); + assert_eq!(ptr[40], 40); + + let ptr2 = ptr.all(); + assert_eq!(ptr2.len(), 50); + assert_eq!(ptr2.start(), 0); + assert_eq!(ptr2[40], 40); + + let ptr3 = ptr.start_from(20); + assert_eq!(ptr3.len(), 30); + assert_eq!(ptr3.start(), 20); + assert_eq!(ptr3[0], 20); + + let ptr4 = ptr3.range(10, 10); + assert_eq!(ptr4.len(), 10); + assert_eq!(ptr4.start(), 30); + assert_eq!(ptr4[0], 30); + + let expected: Vec<u8> = (30..40).collect(); + assert_eq!(ptr4.as_ref(), expected.as_slice()); + } +} diff --git a/core/src/parquet/util/mod.rs b/core/src/parquet/util/mod.rs new file mode 100644 index 0000000000..6a8c731d4e --- /dev/null +++ b/core/src/parquet/util/mod.rs @@ -0,0 +1,28 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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(crate) mod bit_packing; +pub mod hash_util; +pub mod jni; +pub mod memory; + +mod buffer; +pub use buffer::*; +mod jni_buffer; +pub use jni_buffer::*; + +pub mod test_common; diff --git a/core/src/parquet/util/test_common/file_util.rs b/core/src/parquet/util/test_common/file_util.rs new file mode 100644 index 0000000000..78e42d29e6 --- /dev/null +++ b/core/src/parquet/util/test_common/file_util.rs @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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::{env, fs, io::Write, path::PathBuf}; + +/// Returns file handle for a temp file in 'target' directory with a provided content +pub fn get_temp_file(file_name: &str, content: &[u8]) -> fs::File { + // build tmp path to a file in "target/debug/testdata" + let mut path_buf = env::current_dir().unwrap(); + path_buf.push("target"); + path_buf.push("debug"); + path_buf.push("testdata"); + fs::create_dir_all(&path_buf).unwrap(); + path_buf.push(file_name); + + // write file content + let mut tmp_file = fs::File::create(path_buf.as_path()).unwrap(); + tmp_file.write_all(content).unwrap(); + tmp_file.sync_all().unwrap(); + + // return file handle for both read and write + let file = fs::OpenOptions::new() + .read(true) + .write(true) + .open(path_buf.as_path()); + assert!(file.is_ok()); + file.unwrap() +} + +pub fn get_temp_filename() -> PathBuf { + let mut path_buf = env::current_dir().unwrap(); + path_buf.push("target"); + path_buf.push("debug"); + path_buf.push("testdata"); + fs::create_dir_all(&path_buf).unwrap(); + path_buf.push(rand::random::<i16>().to_string()); + + path_buf +} diff --git a/core/src/parquet/util/test_common/mod.rs b/core/src/parquet/util/test_common/mod.rs new file mode 100644 index 0000000000..e46d732239 --- /dev/null +++ b/core/src/parquet/util/test_common/mod.rs @@ -0,0 +1,24 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 file_util; +pub mod page_util; +pub mod rand_gen; + +pub use self::rand_gen::{random_bools, random_bytes, random_numbers, random_numbers_range}; + +pub use self::file_util::{get_temp_file, get_temp_filename}; diff --git a/core/src/parquet/util/test_common/page_util.rs b/core/src/parquet/util/test_common/page_util.rs new file mode 100644 index 0000000000..b366994e55 --- /dev/null +++ b/core/src/parquet/util/test_common/page_util.rs @@ -0,0 +1,317 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::{collections::VecDeque, mem}; + +use rand::distributions::uniform::SampleUniform; + +use parquet::{ + basic::Encoding, + column::page::{Page, PageIterator, PageMetadata, PageReader}, + data_type::DataType, + encodings::{ + encoding::{get_encoder, DictEncoder, Encoder}, + levels::{max_buffer_size, LevelEncoder}, + }, + errors::Result, + schema::types::{ColumnDescPtr, SchemaDescPtr}, + util::memory::ByteBufferPtr, +}; + +use super::random_numbers_range; + +pub trait DataPageBuilder { + fn add_rep_levels(&mut self, max_level: i16, rep_levels: &[i16]); + fn add_def_levels(&mut self, max_level: i16, def_levels: &[i16]); + fn add_values<T: DataType>(&mut self, encoding: Encoding, values: &[T::T]); + fn add_indices(&mut self, indices: ByteBufferPtr); + fn consume(self) -> Page; +} + +/// A utility struct for building data pages (v1 or v2). Callers must call: +/// - add_rep_levels() +/// - add_def_levels() +/// - add_values() for normal data page / add_indices() for dictionary data page +/// - consume() +/// in order to populate and obtain a data page. +pub struct DataPageBuilderImpl { + desc: ColumnDescPtr, + encoding: Option<Encoding>, + num_values: u32, + buffer: Vec<u8>, + rep_levels_byte_len: u32, + def_levels_byte_len: u32, + datapage_v2: bool, +} + +impl DataPageBuilderImpl { + // `num_values` is the number of non-null values to put in the data page. + // `datapage_v2` flag is used to indicate if the generated data page should use V2 + // format or not. + pub fn new(desc: ColumnDescPtr, num_values: u32, datapage_v2: bool) -> Self { + DataPageBuilderImpl { + desc, + encoding: None, + num_values, + buffer: vec![], + rep_levels_byte_len: 0, + def_levels_byte_len: 0, + datapage_v2, + } + } + + // Adds levels to the buffer and return number of encoded bytes + fn add_levels(&mut self, max_level: i16, levels: &[i16]) -> u32 { + if max_level <= 0 { + return 0; + } + let size = max_buffer_size(Encoding::RLE, max_level, levels.len()); + let mut level_encoder = LevelEncoder::v1(Encoding::RLE, max_level, size); + level_encoder.put(levels); + let encoded_levels = level_encoder.consume(); + // Actual encoded bytes (without length offset) + let encoded_bytes = &encoded_levels[mem::size_of::<i32>()..]; + if self.datapage_v2 { + // Level encoder always initializes with offset of i32, where it stores + // length of encoded data; for data page v2 we explicitly + // store length, therefore we should skip i32 bytes. + self.buffer.extend_from_slice(encoded_bytes); + } else { + self.buffer.extend_from_slice(encoded_levels.as_slice()); + } + encoded_bytes.len() as u32 + } +} + +impl DataPageBuilder for DataPageBuilderImpl { + fn add_rep_levels(&mut self, max_levels: i16, rep_levels: &[i16]) { + self.num_values = rep_levels.len() as u32; + self.rep_levels_byte_len = self.add_levels(max_levels, rep_levels); + } + + fn add_def_levels(&mut self, max_levels: i16, def_levels: &[i16]) { + assert!( + self.num_values == def_levels.len() as u32, + "Must call `add_rep_levels() first!`" + ); + + self.def_levels_byte_len = self.add_levels(max_levels, def_levels); + } + + fn add_values<T: DataType>(&mut self, encoding: Encoding, values: &[T::T]) { + assert!( + self.num_values >= values.len() as u32, + "num_values: {}, values.len(): {}", + self.num_values, + values.len() + ); + self.encoding = Some(encoding); + let mut encoder: Box<dyn Encoder<T>> = + get_encoder::<T>(encoding).expect("get_encoder() should be OK"); + encoder.put(values).expect("put() should be OK"); + let encoded_values = encoder + .flush_buffer() + .expect("consume_buffer() should be OK"); + self.buffer.extend_from_slice(encoded_values.data()); + } + + fn add_indices(&mut self, indices: ByteBufferPtr) { + self.encoding = Some(Encoding::RLE_DICTIONARY); + self.buffer.extend_from_slice(indices.data()); + } + + fn consume(self) -> Page { + if self.datapage_v2 { + Page::DataPageV2 { + buf: ByteBufferPtr::new(self.buffer), + num_values: self.num_values, + encoding: self.encoding.unwrap(), + num_nulls: 0, /* set to dummy value - don't need this when reading + * data page */ + num_rows: self.num_values, /* also don't need this when reading + * data page */ + def_levels_byte_len: self.def_levels_byte_len, + rep_levels_byte_len: self.rep_levels_byte_len, + is_compressed: false, + statistics: None, // set to None, we do not need statistics for tests + } + } else { + Page::DataPage { + buf: ByteBufferPtr::new(self.buffer), + num_values: self.num_values, + encoding: self.encoding.unwrap(), + def_level_encoding: Encoding::RLE, + rep_level_encoding: Encoding::RLE, + statistics: None, // set to None, we do not need statistics for tests + } + } + } +} + +/// A utility page reader which stores pages in memory. +pub struct InMemoryPageReader<P: Iterator<Item = Page>> { + page_iter: P, +} + +impl<P: Iterator<Item = Page>> InMemoryPageReader<P> { + pub fn new(pages: impl IntoIterator<Item = Page, IntoIter = P>) -> Self { + Self { + page_iter: pages.into_iter(), + } + } +} + +impl<P: Iterator<Item = Page> + Send> PageReader for InMemoryPageReader<P> { + fn get_next_page(&mut self) -> Result<Option<Page>> { + Ok(self.page_iter.next()) + } + + fn peek_next_page(&mut self) -> Result<Option<PageMetadata>> { + unimplemented!() + } + + fn skip_next_page(&mut self) -> Result<()> { + unimplemented!() + } +} + +impl<P: Iterator<Item = Page> + Send> Iterator for InMemoryPageReader<P> { + type Item = Result<Page>; + + fn next(&mut self) -> Option<Self::Item> { + self.get_next_page().transpose() + } +} + +/// A utility page iterator which stores page readers in memory, used for tests. +#[derive(Clone)] +pub struct InMemoryPageIterator<I: Iterator<Item = Vec<Page>>> { + schema: SchemaDescPtr, + column_desc: ColumnDescPtr, + page_reader_iter: I, +} + +impl<I: Iterator<Item = Vec<Page>>> InMemoryPageIterator<I> { + pub fn new( + schema: SchemaDescPtr, + column_desc: ColumnDescPtr, + pages: impl IntoIterator<Item = Vec<Page>, IntoIter = I>, + ) -> Self { + Self { + schema, + column_desc, + page_reader_iter: pages.into_iter(), + } + } +} + +impl<I: Iterator<Item = Vec<Page>>> Iterator for InMemoryPageIterator<I> { + type Item = Result<Box<dyn PageReader>>; + + fn next(&mut self) -> Option<Self::Item> { + self.page_reader_iter + .next() + .map(|x| Ok(Box::new(InMemoryPageReader::new(x)) as Box<dyn PageReader>)) + } +} + +impl<I: Iterator<Item = Vec<Page>> + Send> PageIterator for InMemoryPageIterator<I> {} + +#[allow(clippy::too_many_arguments)] +pub fn make_pages<T: DataType>( + desc: ColumnDescPtr, + encoding: Encoding, + num_pages: usize, + levels_per_page: usize, + min: T::T, + max: T::T, + def_levels: &mut Vec<i16>, + rep_levels: &mut Vec<i16>, + values: &mut Vec<T::T>, + pages: &mut VecDeque<Page>, + use_v2: bool, +) where + T::T: PartialOrd + SampleUniform + Copy, +{ + let mut num_values = 0; + let max_def_level = desc.max_def_level(); + let max_rep_level = desc.max_rep_level(); + + let mut dict_encoder = DictEncoder::<T>::new(desc.clone()); + + for i in 0..num_pages { + let mut num_values_cur_page = 0; + let level_range = i * levels_per_page..(i + 1) * levels_per_page; + + if max_def_level > 0 { + random_numbers_range(levels_per_page, 0, max_def_level + 1, def_levels); + for dl in &def_levels[level_range.clone()] { + if *dl == max_def_level { + num_values_cur_page += 1; + } + } + } else { + num_values_cur_page = levels_per_page; + } + if max_rep_level > 0 { + random_numbers_range(levels_per_page, 0, max_rep_level + 1, rep_levels); + } + random_numbers_range(num_values_cur_page, min, max, values); + + // Generate the current page + + let mut pb = DataPageBuilderImpl::new(desc.clone(), num_values_cur_page as u32, use_v2); + if max_rep_level > 0 { + pb.add_rep_levels(max_rep_level, &rep_levels[level_range.clone()]); + } + if max_def_level > 0 { + pb.add_def_levels(max_def_level, &def_levels[level_range]); + } + + let value_range = num_values..num_values + num_values_cur_page; + match encoding { + Encoding::PLAIN_DICTIONARY | Encoding::RLE_DICTIONARY => { + let _ = dict_encoder.put(&values[value_range.clone()]); + let indices = dict_encoder + .write_indices() + .expect("write_indices() should be OK"); + pb.add_indices(indices); + } + Encoding::PLAIN => { + pb.add_values::<T>(encoding, &values[value_range]); + } + enc => panic!("Unexpected encoding {}", enc), + } + + let data_page = pb.consume(); + pages.push_back(data_page); + num_values += num_values_cur_page; + } + + if encoding == Encoding::PLAIN_DICTIONARY || encoding == Encoding::RLE_DICTIONARY { + let dict = dict_encoder + .write_dict() + .expect("write_dict() should be OK"); + let dict_page = Page::DictionaryPage { + buf: dict, + num_values: dict_encoder.num_entries() as u32, + encoding: Encoding::RLE_DICTIONARY, + is_sorted: false, + }; + pages.push_front(dict_page); + } +} diff --git a/core/src/parquet/util/test_common/rand_gen.rs b/core/src/parquet/util/test_common/rand_gen.rs new file mode 100644 index 0000000000..c876520abc --- /dev/null +++ b/core/src/parquet/util/test_common/rand_gen.rs @@ -0,0 +1,57 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 rand::{ + distributions::{uniform::SampleUniform, Distribution, Standard}, + thread_rng, Rng, +}; + +pub fn random_bytes(n: usize) -> Vec<u8> { + let mut result = vec![]; + let mut rng = thread_rng(); + for _ in 0..n { + result.push(rng.gen_range(0..255)); + } + result +} + +pub fn random_bools(n: usize) -> Vec<bool> { + let mut result = vec![]; + let mut rng = thread_rng(); + for _ in 0..n { + result.push(rng.gen::<bool>()); + } + result +} + +pub fn random_numbers<T>(n: usize) -> Vec<T> +where + Standard: Distribution<T>, +{ + let mut rng = thread_rng(); + Standard.sample_iter(&mut rng).take(n).collect() +} + +pub fn random_numbers_range<T>(n: usize, low: T, high: T, result: &mut Vec<T>) +where + T: PartialOrd + SampleUniform + Copy, +{ + let mut rng = thread_rng(); + for _ in 0..n { + result.push(rng.gen_range(low..high)); + } +} diff --git a/dev/cargo.config b/dev/cargo.config new file mode 100644 index 0000000000..642b842515 --- /dev/null +++ b/dev/cargo.config @@ -0,0 +1,27 @@ +<?xml version='1.0'?> +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +[target.x86_64-apple-darwin] +linker = "x86_64-apple-darwin21.4-clang" +ar = "x86_64-apple-darwin21.4-ar" + +[target.aarch64-apple-darwin] +linker = "aarch64-apple-darwin21.4-clang" +ar = "aarch64-apple-darwin21.4-ar" diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml new file mode 100644 index 0000000000..35976948c8 --- /dev/null +++ b/dev/checkstyle-suppressions.xml @@ -0,0 +1,34 @@ +<?xml version='1.0'?> +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +<!DOCTYPE suppressions PUBLIC +"-//Puppy Crawl//DTD Suppressions 1.1//EN" +"https://checkstyle.org/dtds/suppressions_1_1.dtd"> + +<!-- + + This file contains suppression rules for Checkstyle checks. + Ideally only files that cannot be modified (e.g. third-party code) + should be added here. All other violations should be fixed. + +--> + +<suppressions> +</suppressions> diff --git a/dev/ensure-jars-have-correct-contents.sh b/dev/ensure-jars-have-correct-contents.sh new file mode 100644 index 0000000000..d364ae1f4e --- /dev/null +++ b/dev/ensure-jars-have-correct-contents.sh @@ -0,0 +1,114 @@ +#!/usr/bin/env bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Borrowed from Hadoop + +# Usage: $0 [/path/to/some/example.jar;/path/to/another/example/created.jar] +# +# accepts a single command line argument with a colon separated list of +# paths to jars to check. Iterates through each such passed jar and checks +# all the contained paths to make sure they follow the below constructed +# safe list. + +# We use +=, which is a bash 3.1+ feature +if [[ -z "${BASH_VERSINFO[0]}" ]] \ + || [[ "${BASH_VERSINFO[0]}" -lt 3 ]] \ + || [[ "${BASH_VERSINFO[0]}" -eq 3 && "${BASH_VERSINFO[1]}" -lt 1 ]]; then + echo "bash v3.1+ is required. Sorry." + exit 1 +fi + +set -e +set -o pipefail + +allowed_expr="(^org/$|^org/apache/$" +# we have to allow the directories that lead to the org/apache/comet dir +# We allow all the classes under the following packages: +# * org.apache.comet +# * org.apache.spark.sql.comet +# * org.apache.arrow.c +allowed_expr+="|^org/apache/comet/" +allowed_expr+="|^org/apache/spark/sql/comet/" +allowed_expr+="|^org/apache/arrow/c/" +# * whatever in the "META-INF" directory +allowed_expr+="|^META-INF/" +# * whatever under the "conf" directory +allowed_expr+="|^conf/" +# * whatever under the "lib" directory +allowed_expr+="|^lib/" +# Native dynamic library from Arrow +allowed_expr+="|^x86_64/" +allowed_expr+="|^aarch_64/" +allowed_expr+="|^x86_64/libarrow_cdata_jni.so$" +allowed_expr+="|^x86_64/libarrow_cdata_jni.dylib$" +allowed_expr+="|^x86_64/arrow_cdata_jni.dll$" +allowed_expr+="|^aarch_64/libarrow_cdata_jni.dylib$" +# Two classes in Arrow C module: StructVectorLoader and StructVectorUnloader, are not +# under org/apache/arrow/c, so we'll need to treat them specially. +allowed_expr+="|^org/apache/arrow/$" +allowed_expr+="|^org/apache/arrow/vector/$" +allowed_expr+="|^org/apache/arrow/vector/StructVectorLoader.class$" +allowed_expr+="|^org/apache/arrow/vector/StructVectorUnloader.class$" +# Log4J stuff +allowed_expr+="|log4j2.properties" +# Git Info properties +allowed_expr+="|comet-git-info.properties" +# For some reason org/apache/spark/sql directory is also included, but with no content +allowed_expr+="|^org/apache/spark/$" +allowed_expr+="|^org/apache/spark/sql/$" +allowed_expr+="|^org/apache/spark/CometPlugin.class$" +allowed_expr+="|^org/apache/spark/CometDriverPlugin.*$" + +allowed_expr+=")" +declare -i bad_artifacts=0 +declare -a bad_contents +declare -a artifact_list +while IFS='' read -r -d ';' line; do artifact_list+=("$line"); done < <(printf '%s;' "$1") +if [ "${#artifact_list[@]}" -eq 0 ]; then + echo "[ERROR] No artifacts passed in." + exit 1 +fi + +jar_list_failed () +{ + echo "[ERROR] Listing jar contents for file '${artifact}' failed." + exit 1 +} +trap jar_list_failed SIGUSR1 + +for artifact in "${artifact_list[@]}"; do + bad_contents=() + # Note: On Windows the output from jar tf may contain \r\n's. Normalize to \n. + while IFS='' read -r line; do bad_contents+=("$line"); done < <( ( jar tf "${artifact}" | sed 's/\\r//' || kill -SIGUSR1 $$ ) | grep -v -E "${allowed_expr}" ) + if [ ${#bad_contents[@]} -gt 0 ]; then + echo "[ERROR] Found artifact with unexpected contents: '${artifact}'" + echo " Please check the following and either correct the build or update" + echo " the allowed list with reasoning." + echo "" + for bad_line in "${bad_contents[@]}"; do + echo " ${bad_line}" + done + bad_artifacts=${bad_artifacts}+1 + else + echo "[INFO] Artifact looks correct: '$(basename "${artifact}")'" + fi +done + +if [ "${bad_artifacts}" -gt 0 ]; then + exit 1 +fi diff --git a/dev/scalastyle-config.xml b/dev/scalastyle-config.xml new file mode 100644 index 0000000000..7d01f726dd --- /dev/null +++ b/dev/scalastyle-config.xml @@ -0,0 +1,374 @@ +<!-- + +If you wish to turn off checking for a section of code, you can put a comment in the source +before and after the section, with the following syntax: + + // scalastyle:off + ... // stuff that breaks the styles + // scalastyle:on + +You can also disable only one rule, by specifying its rule id, as specified in: + http://www.scalastyle.org/rules-0.7.0.html + + // scalastyle:off no.finalize + override def finalize(): Unit = ... + // scalastyle:on no.finalize + +This file is divided into 3 sections: + (1) rules that we enforce. + (2) rules that we would like to enforce, but haven't cleaned up the codebase to turn on yet + (or we need to make the scalastyle rule more configurable). + (3) rules that we don't want to enforce. +--> + +<scalastyle> + <name>Scalastyle standard configuration</name> + + <!-- ================================================================================ --> + <!-- rules we enforce --> + <!-- ================================================================================ --> + + <check level="error" class="org.scalastyle.file.FileTabChecker" enabled="true"></check> + + <!-- disabled for now since we don't have ASF header in our project + <check level="error" class="org.scalastyle.file.HeaderMatchesChecker" enabled="true"> + <parameters> + <parameter name="header"><![CDATA[/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */]]></parameter> + </parameters> + </check> + --> + + <check level="error" class="org.scalastyle.scalariform.SpacesAfterPlusChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.SpacesBeforePlusChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.file.WhitespaceEndOfLineChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.file.FileLineLengthChecker" enabled="true"> + <parameters> + <parameter name="maxLineLength"><![CDATA[100]]></parameter> + <parameter name="tabSize"><![CDATA[2]]></parameter> + <parameter name="ignoreImports">true</parameter> + </parameters> + </check> + + <check level="error" class="org.scalastyle.scalariform.ClassNamesChecker" enabled="true"> + <parameters><parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter></parameters> + </check> + + <check level="error" class="org.scalastyle.scalariform.ObjectNamesChecker" enabled="true"> + <parameters><parameter name="regex"><![CDATA[(config|[A-Z][A-Za-z]*)]]></parameter></parameters> + </check> + + <check level="error" class="org.scalastyle.scalariform.PackageObjectNamesChecker" enabled="true"> + <parameters><parameter name="regex"><![CDATA[^[a-z][A-Za-z]*$]]></parameter></parameters> + </check> + + <check customId="argcount" level="error" class="org.scalastyle.scalariform.ParameterNumberChecker" enabled="true"> + <parameters><parameter name="maxParameters"><![CDATA[10]]></parameter></parameters> + </check> + + <check level="error" class="org.scalastyle.scalariform.NoFinalizeChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.CovariantEqualsChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.StructuralTypeChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.UppercaseLChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.IfBraceChecker" enabled="true"> + <parameters> + <parameter name="singleLineAllowed"><![CDATA[true]]></parameter> + <parameter name="doubleLineAllowed"><![CDATA[true]]></parameter> + </parameters> + </check> + + <check level="error" class="org.scalastyle.scalariform.PublicMethodsHaveTypeChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.file.NewLineAtEofChecker" enabled="true"></check> + + <check customId="nonascii" level="error" class="org.scalastyle.scalariform.NonASCIICharacterChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.SpaceAfterCommentStartChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.EnsureSingleSpaceBeforeTokenChecker" enabled="true"> + <parameters> + <parameter name="tokens">ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW</parameter> + </parameters> + </check> + + <check level="error" class="org.scalastyle.scalariform.EnsureSingleSpaceAfterTokenChecker" enabled="true"> + <parameters> + <parameter name="tokens">ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW</parameter> + </parameters> + </check> + + <!-- ??? usually shouldn't be checked into the code base. --> + <check level="error" class="org.scalastyle.scalariform.NotImplementedErrorUsage" enabled="true"></check> + + <check customId="println" level="error" class="org.scalastyle.scalariform.TokenChecker" enabled="true"> + <parameters><parameter name="regex">^println$</parameter></parameters> + <customMessage><![CDATA[Are you sure you want to println? If yes, wrap the code block with + // scalastyle:off println + println(...) + // scalastyle:on println]]></customMessage> + </check> + + <check customId="visiblefortesting" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">@VisibleForTesting</parameter></parameters> + <customMessage><![CDATA[ + @VisibleForTesting causes classpath issues. Please note this in the java doc instead (SPARK-11615). + ]]></customMessage> + </check> + + <check customId="runtimeaddshutdownhook" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">Runtime\.getRuntime\.addShutdownHook</parameter></parameters> + <customMessage><![CDATA[ + Are you sure that you want to use Runtime.getRuntime.addShutdownHook? In most cases, you should use + ShutdownHookManager.addShutdownHook instead. + If you must use Runtime.getRuntime.addShutdownHook, wrap the code block with + // scalastyle:off runtimeaddshutdownhook + Runtime.getRuntime.addShutdownHook(...) + // scalastyle:on runtimeaddshutdownhook + ]]></customMessage> + </check> + + <check customId="mutablesynchronizedbuffer" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">mutable\.SynchronizedBuffer</parameter></parameters> + <customMessage><![CDATA[ + Are you sure that you want to use mutable.SynchronizedBuffer? In most cases, you should use + java.util.concurrent.ConcurrentLinkedQueue instead. + If you must use mutable.SynchronizedBuffer, wrap the code block with + // scalastyle:off mutablesynchronizedbuffer + mutable.SynchronizedBuffer[...] + // scalastyle:on mutablesynchronizedbuffer + ]]></customMessage> + </check> + + <check customId="classforname" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">Class\.forName</parameter></parameters> + <customMessage><![CDATA[ + Are you sure that you want to use Class.forName? In most cases, you should use Utils.classForName instead. + If you must use Class.forName, wrap the code block with + // scalastyle:off classforname + Class.forName(...) + // scalastyle:on classforname + ]]></customMessage> + </check> + + <check customId="awaitresult" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">Await\.result</parameter></parameters> + <customMessage><![CDATA[ + Are you sure that you want to use Await.result? In most cases, you should use ThreadUtils.awaitResult instead. + If you must use Await.result, wrap the code block with + // scalastyle:off awaitresult + Await.result(...) + // scalastyle:on awaitresult + ]]></customMessage> + </check> + + <check customId="awaitready" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">Await\.ready</parameter></parameters> + <customMessage><![CDATA[ + Are you sure that you want to use Await.ready? In most cases, you should use ThreadUtils.awaitReady instead. + If you must use Await.ready, wrap the code block with + // scalastyle:off awaitready + Await.ready(...) + // scalastyle:on awaitready + ]]></customMessage> + </check> + + <check customId="caselocale" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">(\.toUpperCase|\.toLowerCase)(?!(\(|\(Locale.ROOT\)))</parameter></parameters> + <customMessage><![CDATA[ + Are you sure that you want to use toUpperCase or toLowerCase without the root locale? In most cases, you + should use toUpperCase(Locale.ROOT) or toLowerCase(Locale.ROOT) instead. + If you must use toUpperCase or toLowerCase without the root locale, wrap the code block with + // scalastyle:off caselocale + .toUpperCase + .toLowerCase + // scalastyle:on caselocale + ]]></customMessage> + </check> + + <check customId="throwerror" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">throw new \w+Error\(</parameter></parameters> + <customMessage><![CDATA[ + Are you sure that you want to throw Error? In most cases, you should use appropriate Exception instead. + If you must throw Error, wrap the code block with + // scalastyle:off throwerror + throw new XXXError(...) + // scalastyle:on throwerror + ]]></customMessage> + </check> + + <check customId="commonslang2" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">org\.apache\.commons\.lang\.</parameter></parameters> + <customMessage>Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead + of Commons Lang 2 (package org.apache.commons.lang.*)</customMessage> + </check> + + <check customId="FileSystemGet" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">FileSystem.get\([a-zA-Z_$][a-zA-Z_$0-9]*\)</parameter></parameters> + <customMessage><![CDATA[ + Are you sure that you want to use "FileSystem.get(Configuration conf)"? If the input + configuration is not set properly, a default FileSystem instance will be returned. It can + lead to errors when you deal with multiple file systems. Please consider using + "FileSystem.get(URI uri, Configuration conf)" or "Path.getFileSystem(Configuration conf)" instead. + If you must use the method "FileSystem.get(Configuration conf)", wrap the code block with + // scalastyle:off FileSystemGet + FileSystem.get(...) + // scalastyle:on FileSystemGet + ]]></customMessage> + </check> + + <check level="error" class="org.scalastyle.scalariform.ImportOrderChecker" enabled="true"> + <parameters> + <parameter name="groups">java,scala,org,apache,3rdParty,comet</parameter> + <parameter name="group.java">javax?\..*</parameter> + <parameter name="group.scala">scala\..*</parameter> + <parameter name="group.org">org\..*</parameter> + <parameter name="group.apache">org\.apache\..*</parameter> + <parameter name="group.3rdParty">(?!org\.apache\.comet\.).*</parameter> + <parameter name="group.comet">org\.apache\.comet\..*</parameter> + </parameters> + </check> + + <check level="error" class="org.scalastyle.scalariform.DisallowSpaceBeforeTokenChecker" enabled="true"> + <parameters> + <parameter name="tokens">COMMA</parameter> + </parameters> + </check> + + <!-- SPARK-3854: Single Space between ')' and '{' --> + <check customId="SingleSpaceBetweenRParenAndLCurlyBrace" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">\)\{</parameter></parameters> + <customMessage><![CDATA[ + Single Space between ')' and `{`. + ]]></customMessage> + </check> + + <check customId="NoScalaDoc" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">(?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*]</parameter></parameters> + <customMessage>Use Javadoc style indentation for multiline comments</customMessage> + </check> + + <check customId="OmitBracesInCase" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">case[^\n>]*=>\s*\{</parameter></parameters> + <customMessage>Omit braces in case clauses.</customMessage> + </check> + + <check level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">new (java\.lang\.)?(Byte|Integer|Long|Short)\(</parameter></parameters> + <customMessage>Use static factory 'valueOf' or 'parseXXX' instead of the deprecated constructors.</customMessage> + </check> + + <!-- SPARK-16877: Avoid Java annotations --> + <check level="error" class="org.scalastyle.scalariform.OverrideJavaChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.DeprecatedJavaChecker" enabled="true"></check> + + <check level="error" class="org.scalastyle.scalariform.IllegalImportsChecker" enabled="true"> + <parameters><parameter name="illegalImports"><![CDATA[scala.collection.Seq,scala.collection.IndexedSeq]]></parameter></parameters> + <customMessage><![CDATA[ + Don't import scala.collection.Seq and scala.collection.IndexedSeq as it may bring some problems with cross-build between Scala 2.12 and 2.13. + + Please refer below page to see the details of changes around Seq / IndexedSeq. + https://docs.scala-lang.org/overviews/core/collections-migration-213.html + + If you really need to use scala.collection.Seq or scala.collection.IndexedSeq, please use the fully-qualified name instead. + ]]></customMessage> + </check> + + <check level="error" class="org.scalastyle.scalariform.IllegalImportsChecker" enabled="true"> + <parameters><parameter name="illegalImports"><![CDATA[org.apache.log4j]]></parameter></parameters> + <customMessage>Please use Apache Log4j 2 instead.</customMessage> + </check> + + <check level="error" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="true"></check> + <check level="error" class="org.scalastyle.scalariform.NoWhitespaceAfterLeftBracketChecker" enabled="true"></check> + + <!-- This breaks symbolic method names so we don't turn it on. --> + <!-- Maybe we should update it to allow basic symbolic names, and then we are good to go. --> + <check level="error" class="org.scalastyle.scalariform.MethodNamesChecker" enabled="false"> + <parameters> + <parameter name="regex"><![CDATA[^[a-z][A-Za-z0-9]*$]]></parameter> + </parameters> + </check> + + <check level="error" class="org.scalastyle.scalariform.EqualsHashCodeChecker" enabled="true"></check> + + <!-- ================================================================================ --> + <!-- rules we don't want --> + <!-- ================================================================================ --> + + <check level="error" class="org.scalastyle.scalariform.IllegalImportsChecker" enabled="false"> + <parameters><parameter name="illegalImports"><![CDATA[sun._,java.awt._]]></parameter></parameters> + </check> + + <!-- We want the opposite of this: NewLineAtEofChecker --> + <check level="error" class="org.scalastyle.file.NoNewLineAtEofChecker" enabled="false"></check> + + <!-- This one complains about all kinds of random things. Disable. --> + <check level="error" class="org.scalastyle.scalariform.SimplifyBooleanExpressionChecker" enabled="false"></check> + + <!-- We use return quite a bit for control flows and guards --> + <check level="error" class="org.scalastyle.scalariform.ReturnChecker" enabled="false"></check> + + <!-- We use null a lot in low level code and to interface with 3rd party code --> + <check level="error" class="org.scalastyle.scalariform.NullChecker" enabled="false"></check> + + <!-- Doesn't seem super big deal here ... --> + <check level="error" class="org.scalastyle.scalariform.NoCloneChecker" enabled="false"></check> + + <!-- Doesn't seem super big deal here ... --> + <check level="error" class="org.scalastyle.file.FileLengthChecker" enabled="false"> + <parameters><parameter name="maxFileLength">800></parameter></parameters> + </check> + + <!-- Doesn't seem super big deal here ... --> + <check level="error" class="org.scalastyle.scalariform.NumberOfTypesChecker" enabled="false"> + <parameters><parameter name="maxTypes">30</parameter></parameters> + </check> + + <!-- Doesn't seem super big deal here ... --> + <check level="error" class="org.scalastyle.scalariform.CyclomaticComplexityChecker" enabled="false"> + <parameters><parameter name="maximum">10</parameter></parameters> + </check> + + <!-- Doesn't seem super big deal here ... --> + <check level="error" class="org.scalastyle.scalariform.MethodLengthChecker" enabled="false"> + <parameters><parameter name="maxLength">50</parameter></parameters> + </check> + + <!-- Not exactly feasible to enforce this right now. --> + <!-- It is also infrequent that somebody introduces a new class with a lot of methods. --> + <check level="error" class="org.scalastyle.scalariform.NumberOfMethodsInTypeChecker" enabled="false"> + <parameters><parameter name="maxMethods"><![CDATA[30]]></parameter></parameters> + </check> + + <!-- Doesn't seem super big deal here, and we have a lot of magic numbers ... --> + <check level="error" class="org.scalastyle.scalariform.MagicNumberChecker" enabled="false"> + <parameters><parameter name="ignore">-1,0,1,2,3</parameter></parameters> + </check> + + <check customId="GuavaToStringHelper" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> + <parameters><parameter name="regex">Objects.toStringHelper</parameter></parameters> + <customMessage>Avoid using Object.toStringHelper. Use ToStringBuilder instead.</customMessage> + </check> +</scalastyle> diff --git a/doc/comet-overview.png b/doc/comet-overview.png new file mode 100644 index 0000000000000000000000000000000000000000..a3148d8f84b9998c8a3c93e38fe2e975366f2662 GIT binary patch literal 172624 zcmeFZd0f(I+bC{LQ&ZD2E!t)}Q<;fNnVFd;HErWkmWumUWR|#-8p=|(IAv;Cxa2~c z8lvKo3ocMulA_Y0xPW3RQlg>|A|iOcp!L-A^qlv1-rsV5=l#6>F`p6c`?|0FzP6j| z$B!LZvS{5R6%~~wmVX_zQ&Cy8P(@_{<CposC!5EbdsI{|s#qS}?{L9$sEZ!E0hXyT z;eIxvErN*ImmDS3+qQV!wz|vJ#UFP3iJ*_=ZXa=6c0ivMzJ6!+BKpq79vUagR_}5! zY&F0Zto?ZOlFpT`H+92U9okQUZTj164m>DYySRJt66ovtpv;xIueE#QR%&Z;_JS=m zFMPDClh$2d8I;@dcI^Dij<o<f|NZa3Iq=^c_}|NcXPy_PCdOp3v9SvXt%KKDM+=(8 z+_3Is8s`W}<11>xuw|MS)lON@IH3~%Qr8+%Ll&}x<QnS)QEx1r`(@chmt6G;PO_Ip zQ}n$di08Is+7VLt%$Mp#+sG5WeOB^xCTWVZ%z`}i$|=EsBClV$eEIFmmNKSD8%)AT zCXaYpFJv_gguNghu{u|bL!8KJc0V!$BL4W^PmdD8zT_~&QO<iB(Thx;;y5L5+_-T; z29XkZ;_CA$@zVQQo=`KpKkw|dt%-I%@tbmI&%8W_t@J_$*td!Xo^lfx&@Q<Y{I-0# z=`{zJ(Lr|Qlz7>F+x?@3_J^Lk+s~f$SIwD<D${;!Z57Gj7qm>!djI$z8#jj9;4!>a zKVQ1mBzl>qm+6D%nNY$c3&8Rbj^^}GRA`Juis2Hy<RcN3z0L`Zp(Ny8*sn^6&u%|S z+aI`i_}KpA2M-<td0|8wr}gJ;b!z*KUL4G@RsCuDGkdC*Mw{m9O6Ci3Wci1%liS;J zl1-dok`o2nX0SSP-&4P}YmML@UZP;#o&A^oJb17_MHow*0<TmtalrdO_7Bg94E0uh zU!1pv?VtNJ71qk3ZKELX!4}V;acO-Op8IJz(*t~HG})_W?OpAI2M@-ZdQp%W(_7}| z<cn542mK;Xo*sR)fB(_(P5Ii3CNb-NQtsXTr_5=IIB+q->W!LcbJqS4#o?O;mlK5S zLp|p`JiyiUyz31LK3qO^Z^%|Y<OJ5pbKyt6nSQ@!=5WJL6u*iH``7O`l~qhl5?0r~ zyYlznuNU^O-d1yC#RXlW76>?gdQua!V{3LjniD~h4nOY<C|Do^$IVefP>(H|K_T+x z6@>z<ySoKI|JWZ+)AI)?vCqy$Dpa{T{z(hLzvR{yX);E3gsqv^wPjmq{tSTK$Dzb^ zBfX!Nzn_0~A)t9Ht>jg|q-Q%?jxIw_b?yJ-==ipLr=?NF<}(N{cQHt&3$GpL*S!&{ z(}_MV3X&fSCXZz9Ufe>shU*VI$VtGA*C;j`k-|N`9<xtjF3Bd*JB+t*i*{|b{`>F0 z)9vK_+F~L5aL?zwyYrS=e7Ah{>eV=I+<lw4dPLrmh82KL=d9A&-(zDrx+W3Urq#Ai ziO<EKc8YR87<ihpzJR<t?~;+$bLd|M9?9-7<r<Zxjp<HHBE}j%gj_CGuvh1h<!`&~ zSD0Vx?+mLiRYU<QDm~8{5v{?iiRv0Av$*}I0p&RoOb{#2ZVU>wu!;sOr1!*%dChm{ zCH(O%eZK?rEfdUtQ`VBsqT|=!)lx`J$13_+YGRM!=C%Us0^|2;v9CBct3TFZ1ohgQ zasHTXZCy~Q{PTNTFu%-8#+8!xQdta%MDm`+e<-d|iu0mbvAY3|v*dmN-jsy1{~;~e zajW|3LLccxh`8J$S&e%HX7W=`!YPU<{9;ti@FT6~dosHN136PuQ=->s>^vpho`U@F zC?$WE{)mTXI{z{Z)N@&ZyajEJ^dU3X=?r#l)jW209{ZD!N~UkIidHf?I5|+Llos;+ zeLaQqu=<WQsb$W3nK%1g?sDeaE^7SKYwyq7sY9poRGf*L-Ma=*NHmJVV*g)+)zS6W zTE1mG9w(xv<8YJ)!>TnpgG&}gBHU#Ud(DvY8iFLKw<L|K#7hOakHD<)tB7@4Dqj2p zUAxy6cXupvAf+UXG;$a}QbM<o?q1NebL*hThFY%a*`bO_Otbc~<`cM;Qe>;NB=L78 zR2G(+pvk7M2Ulf}-<Ix${E%@GE_E<0M{Vpg<#1{+L2uB*=kxU2Z3!?Tx195WiA3L> zi}T_f;?p0o*DHZesF0@wv>ZgAZ(EB+C?&`5c#^_zXKh_L*VihVddt1afW#Owpk2r> zX1%U>8_M2wq7jxP69@<CvYdqsUJShXZlu6_h}dvrRw$!Ko%0=u!aBNs@%yG|=nw5E zJlUrjU+k_bJT-J#-AF4o63c*H7FG<>MK}7UT$*9^ZK}9@uc*AuzbKK67>)(*w%tLK ztX-u<*0j#LAE|d0ZAZ6*%ZT65qcu`<s!jdF?v^R@f`Z@H5ErA>-rrNJaZ-)IRUKb3 zPXY2`mDs}5nDt6Grft5D!cE!VNc=rGd-H|G^`*Q-Skp-H&GIZgCtEfzA;4l`aHE0k z#>T!utLPIB>7s_dCc;&zbg~*v{tB%;60Gog*jj*T|6-yY9!X<mSWZD1IQB)?BXy$z zzoE<WYS7|=6ta0iH{oi~D%GkN?7M^zdK5g@`9hEouC-rDcZD@y2&wg`4t2Iv+-c_D znX`i4vSOAm4mBQvnJO#x@?Fk(K`^g3L?PU361|h;_lSB1qQ|upta9R3Pe7gukk7jO zx+<cmHr?lU|NM>qz`S5v7it^}9j@$mEveG$gM`}b0fRBEnqIG-tFN3-)G5Gm?gPkf zUD$mA^r7P=xmWa(r1=#~8b<)>9z_5eB!nMp`jIIrSg<Rr%2ITLogy6ngoGA9DGxx_ z_YEdX8(QhzpUaHB5bm~nhUz%>%T|A}?uaBHL_(BsQCZ^|xCpC%Cn%?S!%c(r7lW)L zML3?X>61IQ8jUa5Ddq*~@KSnCS6L?p<EHHuSN*q>`nJ2X?q@^~h@Ado(B;mn<z<!Y z#Cc$1ra=nSN=+m<ht@~9OP)Ss@wIWt9TS+}8ljDbzkXqbnPx8_uwyc8?Kj;0dNrH@ ztTW9miI(UI^PIkp!8)dHk5g}p%TY<XfE|6$B3DSB);=F=rDvDT!~^{L7-Va3vOStJ z`a`AyJ}D^N#lUM*y6vFuiIy~^ehB+@$P%k)(fQ|rQ5FwI2Yr`K3aJg(zv8!EQ=q_4 z^6&85<Wt?@LjrZbVy!<!zCBEzWaivdZ@a(QIqT&b;eDnfu2m}esC)OTh)f&_t+CZ_ zg>yoSr>XM&%G>+y!_B{>>nzkbFsn`xIsz=HCk%chb3j~8;YS_{!AS3#YCT_p8yj!$ ztC&yRXz&HUiBSy;<b`cP0_2kxUk(=+{UeCDN)Q9`N1#2LkjP@Z@WKKRNoUp)eS`>^ zq|H{i9?`%)rd3VSdcM`vlhRxf{{UPLIhF^l93GpAP>ej>-I19l2n&b75`ob6Il$T@ z85P%ewHpW?2yD?4VD>~80nAM~@S2X9j7pxa^_t=nDYg75MsgcMB4caZlupaLYzruT z7%^9##yby%p-t-8)V<fgbs?z1!<maNc-*6fpkmXAoSM8)&-m0GcJr$Zws;P6PYgxm zzytt}cI=XAQ(13sD+Z12Gy4Z00M!>(*9#xm;jlQ&B0%+1GeC17A98N&n`W}77|a#N zIhs?%={^x178OS$u^oE1bb434j%j$1Jdei@aa!8g!j{O}y=1YNV2{O(A(9%n8#<eH z;=3rM0QWGW53>MY?74IhKNj?BI2I)1+z(JQp@o+khzDD1Yj6;k5N)%hdD25vu8f`u z(l+>FLA8}MAP6#j<+OBJ!-`^Kh)V7XZ`Q4QHrQV<65Ht!Z?A|9^8yyY($J4u&&}oN z;?2&OW<x03teoT9u;IIi>CAB?LHkpUMpW0xz`~34K8$b2WMr@5Cfn`RW9MH)7JD^! zoJmgfMp4#CcloZgwR8XC%wocl0k0c+1BkNjYCTSkOmxP|1ljKlBm|6P8|sm*T4P`R z@J3CLOR?GGL~mI}llD)J%L{K%AnWAx>A{#FDg!6vyU4_cWz~Xdxff+R+&`X1$C`bA z7n~)H>=AW#frJ5z!c44?9(4^@;QGDSugF*fpRR0TKXO^2H`f2c;7%6x>g-L8bdHb! zaY@cZ!v{ct58G?qXAlnSmgZX>fFK7wP1Ca;8i<5w{6~X3)I{bX3x}no&CWj0rlgtB z%6@TPw0HccUVdNZ!Ed9=GsA-z@rA_cs@FGd)CS8^CmP*6V)(8r{E_m@>TTs;E}~_+ zK_e`PV0U~Mpw*Xe?p>Q_cj#FsBXWngY>Eq()tn&}P9H3)veGyZxLR_&Uf?e4dTl7Z zXFi_Ho*sUlVjuwt@NGAx(hBW=L{d4CsPfoDedjL4Rom|`^!M~im*(53iCn|=7Y=aS z?)(h5as4U~;pm6js02+g8LW+1`2?HZW`zpZpc?}5X#e3lPb1e3k<M8c^<101Cx;Gb zCHzqyi+pnWZNZ{4(=%Ros1_nIrJ)v1$)O}ft@t)Jzn7MK)`_^bdAjDYwRll)dW4&k z6saL)ySBhON;f;J)x2Z3`huH=@nEtX9=JexcaHqkJ<m@|nHs!~JN+SN(PjHA9R8;2 z&p{=`d+)W%1FX^LKz8!b<lR@1C}BV|wamft>A2LwYncZ--;Khs-<n|P@)VZe7Jw-S zpBA)>@tfOaBNRIO?VZ@Kb`IZAQ3dLP{4okj7zA@?67h>cHvc%8*#i5s<fRd0X=f4= zbh75!1*WbOtQkB_ou-A2JrQ;GQYMq9-wpLOkVrLlxK`W1AfM*y$fLaW6i{<3=}C@m zIQNO%HxAF=<+v_y%;oi9GYr*SHrI1{_arePzqr-;K?#M=YI7KDhc&ma)TwG&7bx}k z_-;=OlvvhGD!1H#$w(-Id$93kt-e8KY?r6J26z|$f<{yp+L5x7Io>5RIG^g4!jlKa zHSzdH3HfpsPTYp>t7lH9^W0%fO{N!jii|))Kgly&&~UUEA*u<@wE2f>l9nptDV5zD zmGY)`0(uu@e<&R-$!-8M8?2aioD2}6NyiX!9YnLV$A`XgPA$Nt-(hm&*V57vki@|R zd|y8N_N!c*OzEB&(=GN~s9((_lvgq68`ca5Lp#d&2KmJ&bxXUMXcK`tgI;{GjUCQx z5yOVdv6EyH&VWQ5vAmIA>;`5w)7r`KMrVQhIG1CQBUmk;!kA6C=*s>K3Z(V%a+)C; ze1`#fe1j7ZNR?s}d!B2zHHZDKRF`MV>(ln-_nR{SS4%pEg%zJS5}!lyH$k>E()o5* z@s9XFChry;8Av>RPPe>#hcDYDG&je1fLm&p)zo=pucdx@=?;i1FM-f^&)zY;ir?UH zJH&4ZjOjqqrGIcJGUPTN#hSQLol;#(7>N3J9vdRmLUQb=cckf^5dbUNN&drdkY?rP z3LgJV@;xI+!QG=@eRT$JpAyN7n-Wbfxy~f8p&3Pqp7E07_-IFTIJP+VcpJvfTYRHi zZyBUsPu!s;PGjo2q&E!bv(B(RI`YT|P{||GFyj3t)Oc(JWIF-=PIWMMv9aSGnCggN zccrksW_~56z|E(Qac4cxIn-aA+aXV9yq|PERArS}D}TR;w+4ps2pwVdBl6((ytmuZ zTZ`HdYolCG$;`#6>=KLx%#CutZ4Z#$@8GtJPpnAqoNggyRMD-_6jry-(DjF5-7vPS zZ5Ou^BG5_S@ec#%?wKtj{oQ}`zGJ8c3<#avwXOHtzp4^v=y?Q01$=C`uXFXu)%%Sz zZk3cC8l_v_5#cj<&X=^*Birh{%K#zNEgs)*p=-~rL}&WsJG%G}=|t!MdD3U-)8D4q z)|s_|!)`EiT7S_~vTw3pPM9rf$k$)Le6M5&-I~WgITGcEll7DQn`koIj0BQ2?=uw= z8l^?w?=vW-SZu@Cx?vWaUisA_{TqBeAyf5+nj4|E_hDGZBwo8*#C_V}K;pAJ>!VL7 z?3!G%53)9D&^@_GXejFJu(G&R2zk;VbIiVh#OlSyVLyNnOWmm3jdLLB>Si$_>Ak+d zG-PWyFG7dw@Vq!apUiW<Zb0|&@ps?yYat2c&$fS6jV`{$yq7G+?HO^$F)VatA^H3y z%4#?|*>CS8H=uc4=K23XiS#C`=_?ox#p&&!2XR_72Po&UeB3G!XLd#pKEOd_*n2j} z0^MRKyJchf(Y_-Q$0Q)J$EynU_gPlxE}l1c)n53p8>ikp25TzCr4_Y#P!e)hop~EL z0t*&O<7J*)GPUle&q%d;gp=0uDO8$JoMo*6=ZP)Jb}Qx;<j<683Vxc2%;LZC*<r;W zjUbC6arN}BtaL09xxsAU8^pv+#|c8C4tU(C+!;%)3ZP(%wAT-$Ft=EHKzUB}@OIFC zvb6x)g!MqLrY5q3Mnbsa4QBCr&dn|~D-C;?XFM$bs8&LGDcuU=u2s(Ej@YehrYwNV z3o`uwLyp;Cqq}{JceX@whMbF!?;Jl}&B7=-hG-6zwx>iI2gld<r<>a1(u5mA8_S%Q zln-#mO+SHs3a!bLWg@d=?jz$7kHoAhgHlg8-F^!O%+(O5^V6W&HxROZ!_omxMG^F4 zF|A0chXu9ilx-e@>$l+6soHD*11+SPR+&HNc@*dG+=<oO5r>T)7{WBC6rDB&l{*Q} z#n}Ix>oJ~|D%>z|b1khvIA}~P-6h&vm-(j6itu+5x9>pp;j$Yk_Ec9}-#j}gYCQej z87%-iwQhklJ|kJ(h%3%bq_*5k?yHM`Kr^7e({!pICJ`g*+cBE2fkMh`vFXP?H7R=J zp|`ycL2tZ21@CYY#x$gNOs5+{>;JS5B=#dRK*xuR*^TM^?Y@o91I+|cqg}tjIn}oU zWTR<Jx#iyB;M}e4S)L0#u09X2t*LJ6behE}f(=j5p~h`Ekn9nk1ujTra?__tYo+|3 zI{xRkRGUO%i{T%Bo_*uE=>RIn%F$)0-q0Su<6W8En{*j{tx-~BC94eKQg#n?Jgoj5 zBv6x<&VR!u(<YlcB2;(5aHeI}s2l@POT&oeWtn0AM%VZ>_)@l(sIzlVojbe80iO?1 z`}cSj2S}?<C$q=Ciza5u9l-|$TaN3RCX@ofV}bpvCwd)?sQkgI&0wQ)Tdml*2#*!% zM%!qPfG`bQDjSBOpAv|@u7PwzC^r`yz=i{<g`F*v5MWBS&P;P$u1}|?!zajg9BRPT z=b%S%DEbr{o?u4dw<GEe;o`mGlu%Z?;rR}Z^JE{0v=_J8nRvE9*0qa-`41Qvbm!0$ z5Q3A!eB6M{e{l?bCJ<+(mRffsh-qaBr``kt8S^>@;-j{;6A7g!8-sHHptd0B!}<Qw zOi@%Kw9X2iI+Z!gHt@kgvsS*KedS7?G0jxI3+7eNHw*U+Z43~1-AkrggfFfdDVbF_ zJRKYH_ctNfzcQ^mJ2NFoE7R162q@YUrjsf@6yvh!9%zkE@oNVv4l0oLkCUJ~WPD@e zZ3!XGVj4V=2`Do-XY=j|3gGs};#CLg5{0f;3)M}y>gd-u^JVRXfS{6btSQ@s>VPn$ z-qf-tL8S%#Mdgg_MC!XvU!~^pB78@<4QBsyy$doL?`)4i-1Dv}*llg;@;fc(O)KvZ z9Gu@zu%zCwlc0dB2e&x+P+d_!A>K{eT-@q#Dc3rwAXEwnh+nxmg&X9*VF#1~%m33k zGr^QQdLRdFE1Z}XQYJsd8Is1~&DK86;-)`?cr1}UMpKwV9rvh*;yS0(`-T&7IHnNp zMxhDf{}JE+$os~fK1VFJ4k22rCKkVR@+sEq@#!cS=qK0haNI#dIfdE;5{K+U2NO9| z2D#+(PG2^{Tq)%IQ?2QJ8)#)HU(3<t>CgP7b*Jmlbk{>q%{Af89}Mv9v`$}?JRQnU zq_7e0r15-ya;gwj@_6_7swNu^TNoP(&51`fkA>IM>BI18&x9<idm^^Bh`pC|bR*zk zu!=4kLUvqoSrZd+{O}za4JR9<UM^+!A1W|b#z-m)&h{UEb9N?-VEv`)U;G(@*H&rP zXSY;%Sb4it(6o!=w5)CNwKWC|`^T*;u=%A#14wj(tZY1kY6`hOn3zRiStQ9PiZ*)1 zr&h7kP0IM@-ONl1vf(srT0uoUTWl&NvfrA^r=+)G>`58DQ9?#6<?)Zlf65VwBmLP} zkFP{vr}*&<{voiW<sB%vS$}orY2bd*RisF`wq;oMWK+<RMdylvoY`wSxG+3zHj`3O zzkGZB{zo&R-j2Wa9@VFrV?iHU?o_&JkwJqFsb}+U?u2}<2THJ03}1)hXAb97qeI;T z*&-oqs2zxn%@V?L_qA%x+FpK6Lz8KD(P!>W?7WiGE?n(j;U1IRO}Lj%!ce4^!a-D6 z7vJ4A{*q>+A$7Lg)`reK+eT04n>kLxhAHS|;f0)Wj}A^dS3M|sYKrB#HtJqaS?@WG zT9$TVrsBekBfzeiAeNkvyvGQaEy&v0L2Q9NfOqXeB=Tn4VsqA!mhZfmu1pXf)%UWT zbN{yIP(UC1->XPF97pk|Kj7u=(t@^J$4^h}4rS^66u`v9Hw}B##ZwxB!ZhkEsw|9R zok(=6r2K*EN>d6nP}3TNB;4xIG>V{@2IXx|u`MOEOtzZlzBrdryC+#Dgh<y$t@CQa z!x`Jo4M<Wz54yHMvz}UQ2-*;V@)U!ZQyLz14C5J6S^P99v8A((4m-BNUPh8uwW4s$ zp{T)ju11hnf=PKqeeanpc|0Seh2PRHPK#PBz$X4a$^hDrNo~KW64kr;Q*F}|cX-+s z1C@{h#^)0SMzh+uHE3RGqR<)0fFGxV`fjQQhg<US;i-{3I-MncZe_z@DtFufZgtzE z@}Yauqfi`H&#g&kTT@5SnW2dtlD$B%3m?zml;B(TMA)wLm!U;qZKb?mx%1lhBn6zq zI>3i>ZiLGkWn@TTdQ@ZAdqZK0tF$@9o@yb!pbHpT4bJ?UDW30wNrro^0L*@ufq{WW zzwyE3$c%ub_ou;jTFvf61E7Zc$lZ4a>7~cJfBBL-pU(FDgJmUNMG7GLZ=kV?T~Kun zYw_02v|*V|ja7xU>ch%tHvn-AR&P&}>42%Uso!v5TwNg)Vkv(_ycxXdJs2o;n(_;* zqp_2yeC_v~qNnfnb{L4?1De;;jt-ZQ?TIT!-l@)`#^X!*>Ly`BRq8<+Zh`a`a$Tmt zG-*#f1NK4-s4DuKBPT{ZJrA76Yq!c~JE&w*%gTA>tLEW2vrW|C`+_rHZqGA0Q7c@q zAt1=H+_`o#?uc;_<S8%tS>T<?^vD}0#V(K)gP6mrF$3tsaeVD^-%5TvCN5M)A1N)7 z6^T5`hpPrYQyX=THFpGMgUO=JrScS#v@6VWda9K~VBcm)cy`>!QP?0%T-7kbVj#UX z!crF4O7O2@3k$9=WCd*zu#7%*=&&0bT*4C%$Co#IY+$#93>cP<XY^&qqmuiE6A1m| zMJ-|iT~iv54Z!Ja0t6H}(~{ZnHtR+-W~zkmk^H{6a7#ooSlXJfZCa#>nid4f$AgwK zo@m-G|C)Xq0p%8@b<Os{B6r!YL}s+DkO3`t{&WTfEn8IuTGG%p)AWtJ`O1E#7OoO+ z)BW45w3SU%z?_3aLdJe4YV`*L-9Il#hf$mAO#b;(lF8NC{ByQ}>izQ7t5;ma-an+8 zKhHERE^fk_T>E~}{gd{qkayZti*b0T#k#-DOf#tCt4&+1nxO&z=s2we1{h)!2!Ktg z6+0nh6JveSuhR{;Z>Jrc^@m@Mz6Oj+yv^48SRWmAD>8o+q5-*X)TS8A0{T0f?wZ{E z9*G{c!TK;Pfp=3o)C6zmogoDI4@4>#s|5EuZo2xcXeRrgZ3fMJqcT*k_fy%7{myn& zXTK<97cXxA9{rhbo}p<wyl0~~pqH(vhR<%1>eDXHmKgr~kePlI(2@FYb}4U%|DS8u zkT=O4eI9$nscrSP0y&pZJ#cH;y*7dffe9CZaqjN!=C4IFNqo;=aV+z7N53;{HwVK} z#gI(ys)}R;3UDrng4yZBBWL;#znzWNW=APf3K4!iY1L)7YG`T7j<;y08`ksJP1f)K z1bCC8Uqrw2)3R-upxHd&v7qjY`=2$xc|Sj|jeq#N(h8p~Z%F&|dn5d<pkRY%f9Vx< zM@7GpR4`+$>V?&R{F8gy3*3<>Jdd14bIsUfq$CEz7bM6(b2yYK%Cs%1c4~+%|0u`- zPe12Gki{nRc`FdNzsx4hSx@@oOZ4-_KNl3N_vD5hOa^7^%pbAM-9F)j?sK~wHrQKQ zJA=ZJ%|Y{LV}~!(9EY(Df^`pRV$5{VGAlnp6Bj3wYMR7G1DGKS3}?120R79hIv`NN zw*7{UjP?d@e2g6t-E7S4Kl*itU;mjeAu}KZOU?pKE+y{SxdvX0fykAq2aF13iFQ6$ z#NMRlS@ztoUan-1vvOV_qhB!`<Z6(t$q^j@$QKSPCw)YFKDt*mv{qWszKq4FaN?9T zzGTHblyX4m&g)jnB?*6H@I4JM8VM$9f;^3pYb2lOnC;G0M?L#pXQUr|@o?b0IAGQ% zTtZ8x&(Bge?O(Tf=_IJM0RfE#ofp$EJ!UHx&rS-ZRev4DQHf*yqR^XRfvx=Q<^AUr zBSxzPvB|cIv8tUO9^1_8c`-K<dp>ObmvS(ASfH5UTBV%f+DpWIJ|Ry7rkQ?RJ4WYc zKHj<Y7;uTIhk07iFv~8zFST1?KGEO#&sB1whx`M(GD0GvwbjnO3b6ddKwQo*b5Xil z&oA2+{ByaNbzVZ#L?S)ztx?kf6d_(GXpmh!74h5dfFG}4th49_0Fui~Ca#)y$?(UE zZn{w-C$w!1us<I<*zat|dFF)_ZJloV>ErC^RKZWFeZXW;4aHf{APR9CsucUhy|<JF zXqzg7U*%Oq3yQ&Fw5(M~s9tl|0KjbAP!!|{3>wA~H9W}jw?`yF>ib-QPB1cb*nL5> zupa1`KLti|4r=sQkSt`M-_xbiTp2e7_2r6OurX*Zpzy#^4OW(;J$GQ-J;9l`6WuvY zt=Ung;qjmjkr*PS8%j_2OLb=l8Y~)=b$j5-FaE;obl=CD+c=2$j7I~*2P}hyHMxZo z4>JQ(m}bDNL<hC2qCqj_q&}lpLzk)-0*Va$vGoh1o*{r*Ryk-wpFv!U&Ouc#3sJWX zHB&Mm*jrheS*D_L|0|!ksi1|Kz)X+@bF8#B%3;M2W!%|vrlh^pL)i)f9OK<DD$`@W zN$a`E7&l{4xMD3PA%i@FNMXzmK-VYswBj@mS&j;nLqgorYGR^yleGVitwjTleuBE5 z{o!}(XnyEYC3Za@XDa9_D*If%vfapTMKuPEmhZF*-xlgg3$TiYTp=Jtk$n>^o<CZS zWReGW>pJy$#Whm89Z8*ZZddxz2cyGflOtVaO^7uaCe#5IIm4^oaIaX62N#<sQ0&+b z;(e1}Y*C)6j{O1u>a3!jz~tnsHIXm8n>Uo`RC46^4llaS6g(82uen{6UyUmEMo)KQ zg4`@1A-Vk~!NzizMobd4VPK8Gn4VwVQJOd7&5!)?1C7wD56dDGrl)?VodZsZfRDd9 z6_eO=%K2f{7-AEcx5JgOer;PrUSiL{!Wff*%%<H7GG2|EnXg$3??W~noxUD3Bgz#& z<hMJeMKQ>B;#gF7+8#f{lFb!sl71tKy=?1w6uSuFo}Qu|AD9olax7p%ROnPmpj+1T z<`+})*sZ@aa8f4M;#JUcHZij?1?uND)`117S$x#wopD!Ns(Tr(JXYn1?hi>4S)<al zKXq-*DYe36fC`${Qr4(Rw%{C6q#Ja*!b1IJrquHS>9&WPciZ(dUGM6}PN1Q-a+=c~ zFV0cQ?`ua|eYq3rGs(ceIl%|dAEYNd9CG9UGah7?SI(lq6Bl4=l3lr*+ceqkO!I4a z6j$uE*vrR!oYcf;ySx<hz%_+8?=H((6dk00jT^zZvqX8v_w;a-otp3T?2Z<x$c_zI z7GdjCW9x`bT4hqxY=6&3%$m6qZ#$}cJ;;+{bi58?u{|}7O|jTS)$7GyWZpo;ejr@) zio03Qx8r)pgY?PzzsC>H%%;WL{lI!$64&c?x2_zSgo5&Oy7kW_%?pAW>M^?LhFS>^ zfXdPyiHv6y+PKjQ6RS^D_iWs%Y7mZ$gL`d&Vn#Y{Mr+L|ZS@b7ws><Jh>(H9fq|s# zp=-QO?6Tu4SmptvXp6UnrF`>ilkoUY{bJmNr->EHcjc>UQ)r)2#N^(l(z6*eE$7`J zWe*hi<G7<$^a7l_S=%MyY0JQe!Qp&h`Z4V0hU$$GA+dm7uJQBSdx>9X4-6}GiqCk- ziump;)O7}yWG;19tIb(zK?8;#&g9vhMDbnf!zvmsPt6ojcJKM&0R@n1UC80s(9X1y z3s&8y$`gBP2!?bhuH=2Exc8^K^N!p=Jpx$ql5lPHrk}~T>BzR76wMhv#Qlg5g$z)4 zm|CqJI}RApV+;%tGykmkyG<}&f^hY;IK;1;V31d90y9B^g=pVC;&OD<O=FAl<faHs zBsbzOp;W0N&)&ZNSHG-n9h%?k!3CU`<gDQd2H}M(%uO3ylI&YLt<HAIi;cb`H9#yD zan;|iHBCBWxt)f>)#=8p9((*sF*FnfEN!?VDq)@(O#h#p@hc1faK|F9m^E-6(PrD~ z*0^<{!8O}WNTZyOWfRw|qtl{KWBe@+jnxsZxv|gn8_6<fM&ow<h`A^8A=TDgE|>W5 zW4Oml!ZaxWBkAlWn9~fQ0wbWZDp*qqU4Ms-F+8zC53$&n0*q?a$muY5j3-+;nTO0~ z+bdb!zU7BzUwR|rT;9hrNe3g*ySb|2vicptZnv*o=-{8W9QB*x8p<czg<KIgow-_i z7QI>t(0kJl0S?<angOGG>wL;1H;guGJ!ilW(p^%rVbI&Hs|O5QaBv%K3I=s7exXU` znu>PwpF@@tj~9d(@~^40hkfHgzc-uC6oG;re&`-q&iJ4gHOR1FIl>>a397*XW*F<b zefKsM`55Xg+!c%2Bn^xo)(cdd>P|PlaoIk%e>J=S_cEo=Zu<JwFpctznR|5ix*svy zGBB^jgyYwIS+6R9$=;Z`$l0e{{k;oIZ>r`xf*6YYwE9za{G)!cSEKI2fr#dux$zbs zwtK`ysR_GnRDY&=j?E5?KCAyX%D1EI?yJJhCL~JH3iiOKXxm+fDi(<DJ_F4euQR7X zyBechr928cVn_!S>qS{4v7BCbWs{WUx<W5@t6q32$UM6)g!~DU(=^r^vdpub1+tvc zoD)BYK>%HcW|0GS`INtX=MKX_v&tQJ`BX$<Sq$2p_UVsf{(`b!QrHwqK*OntRnK@& zJuk8BvR)%)g7nYLz;iO-U;4El87JVRL;?21nx>7+BpKubQ8Mp^XYNyR+2V4<yB_B` zqlYbnPq~-VVG@~?nSHZnviu$9;fJLInXv|Rzhfga7eu`44;*V+j*2JcT6rifky$E7 z4~!74B1NBTUJ-G6&acRmbh_KmwN0=KguBkKDjV+9wS(9ml-vm6Q*vieP1%Gna36YF zUKTJZ+y&R<q<A=t6^_3((+o4H6po!L!Ep8|DLQ8VFX#TQ{Nl^>#kW!1*TR-!j_5m| zs@$RiWJmuTEPg*n@a^q1TXMSnFC5*c1omaH0{FuxYx9NC`cn9Y@7hY=1$;{9qcYp9 z2LAXO+x@Nt_J24rKv8Wgkc3%9$J}uT@-OZ`<zEVVm03BEZ}E{_=H(sthP=?@L71IC z%GMOC0e~kw!RIB9z$nwLS-`6kMT+d*1#;p|>V187an~d|I}r{nZ&0SH%dp9q3GsYI zBLT^c!axfH)6DC`W}j9l@;SdPTtKD@*a~E)-+M(xO17az2@78B>`1Vw_L~s&_V(`o zwhDh#OSwb&NU9qBQ6OVaGcSkULkcCn({gGuhGRlDb|wM@f2|`Q4xDd%M}gBGh#n30 zw*h3;=Y!8>mNoMuAO7xayL`&LZN?K{{0VDvc?SDEGmV|!WK+|?+fSN50<AfIG~5;D zHNDTV<mp1!EBk*bzP|Uo(R-QyibF<YPuFbu%@SU8^YBaeH9OZ@j(a@i{DL12Kl01E zlYNIb6mB{i3LQPw`P<^*`i(o+ZXsW}1ibuwA8dr<TI5W!B2c!i_?%{Z4$qCax;k~l zEe!3@D(TLZWJciZvz~$x{HXHs3K=h<yF;g-zEZLpJb??O-IF$4ijah$%K%+{mRcj9 z<WWY2Qw?;X6g&;)1<{*e#F}xDd)taZ%GSmJeZXJYg_>Ds1>$crUH?Rdlk&BS8JH!N z48tOZAu>YuZpN_w?b7(`tY$kGUWzks4f!Q>V|nRa<=<;Ol^=QX_@!2XFdb4xR&UCM z5AWiyuf-5K3m>2m=Q_S_t#HGTonkkamtHxb9b{{nZ&qG@h1DD_2@QlTG~sPnNtoJO zG++oIVHN0$&%3ag7-o|fy4Y-sMJ%-Us%sF`O}W=N;e%?z^^5@hy8vKl!UxT^9Yl(J zGKk$2Apf)Ajzx<vYhsEMUCdxBqVZ>oBkrV8kCEd_8tNaX{hGGf06?*r(O=(m4ksn& z8CzBZWHuNMy?kyxOI*cQ;>P|#oH6xf#(L_@T>x+LB;w8+)R)a=Y7MT`m#)-8Xz#H= z-{<a~h9AWA3<XnW_HovP+aT4bFMloVt}9cs1Zae5)aq)iG-JAMg>6`=tywG5+)ost z1;OY~$>>jDiNse*>F4Sv-!7Hcf&kQeE>>A$EB$(!r%!@+5YlZbNrjfZiA)y|?5JP2 z-u6#>ZkKMk-nM6pqv^Gb-4}EKn6M4Ct0Qds%aRnUbTw*cK7z?=22v{EeJSl*0M;8% zd%BIKw0AK6oS`?|hE+eTmtt+UWlB!E^K~CVy*B<j9gyV%2D6>9u|7cmWE1qlE9fa- zw_knop{I(V6$R!_@cX1C2FVY5d=BqdIA)F4UHKp69&(&W+FyT1o8C71`O|wqQrOb| z@T7(kUwWv0>=U{{dFjEy61Q3<_kvM@q<laifY<@Tlq3M>0?TgzGB1^T67salOB@(` zt^tGs|J^p6G6B41iYQZjwgsprRhKBV)o@HM<@5bv_`K&4UA{`f_@$*}p887ME5orD z_W(Ho8V=xn0YE(;+Rn$(flVR96in{%0kxMXlI|jTIGQ0K*>o*pO#sRI#h75H-=;$P zrVH7+u#DXmPrq<}g;~hxMu5+-%qdr1%Ijf0ETDy@@h7P-eSs4chq+>i5Qb7rz^X3< z%^1Z(b{I~<JOv*7J;0U_o9e;=!=j{bO^N)JU#Zw0e|@#%o-OFSCoQ;18|=^EWc=8; zv_L~~XDxdo*QKRhE%<%+>HL)obdUbheMTQp>r;snSC!a&Ft$bKhuJ8Y{zXNNW-Rk$ zKBWx=0|XAB{oyO-02;yPs8{%v3a$V~U(JZffcBQ`-s0&jOmVhWSYw#d8p9TQbw+pq z+f@O*28atgw7*<q<h_T>s8gUPi(DP(RMW0c@NpcBo)`A&8C%}qL>Ax%xQgnB`Nhd? z6YHk6(P>8#G`fzxL`{uRO~JsWuc9(avWKS{=eWSrOxdQ*O6Qcm?PuVNiv`t_gM_s7 zHS2xzrze%3*gF7+p8O;fvoRRoZg~1O)F*37iZELN=1pAM-g1|T(b{G3V@nht^z^p4 z`!-{i;@ut&RO=`{IsZY_<~4ma{kQeKi=q`D?6d7giMuioZ%&*TT^%Z4rTEOCn+G@) zUDLIWh|eESG~<BVK}E&=X{Bg%eNc6gk!GaRiYaV5Sczb)c#=n)MjyKdaX+LCO+KDz zLJc54dRV^Ek5O#=NN;LtVgj%v6MYHr5oMCs)4Mln+~3#Yvq2DdN{N?yby`}QrpcDt z&~D|5`|C|LbCpe{edX&`o}76tv#%}G!a-Sex)^A@<;BdD{>1_g@=4<F3KYBiY_90c zl*~M6lH)V)RVrIkezBi<5LKBZd@uv)Ji%~z_Ds*}ygv89%!9p;j?j%ZGexbBuFWwZ zC8Rg`bq7;sz`rR?Uj|V^y6+Zueftdf`>_9eh_;=+^F?@*L}-jTLeMW!^6{c>Q9VML zR(-T<zBwa;OGlfriYRYia_E^EMlSLXEl|p=@~#n7!2=a1?>q1;#s9p?N?)t2x>d|g zOV?Ej%Yf6Ud_((8wY}ecMjv#1>lBMs3Z8A@S}9JsZ*6L%j6s5b-eu~f>@<2bq$<nq z7j+BkNx)A*07CtL6s4)pIbf9hk*4fy+N2mh*+EB-zZ=I0?%?|E9ZkB?O4LWbC3J6Y zV#k41O$Mz|{2KApuU-=`c8U00hpg(caf&EEi1Tl@B@ZnCPdpczuz`4}jXYkOk=Rxd zJl<@^H=q8n43iQIKVo@YDgU1D*qZ#L;S#Z0&~U7Xcib=4H7QwV;f!v$e=ConcSBWv zJuD>9H$lita56n7RH;Vg=85+>`RDMo0~!xsr&9k)AWyv7)=e&p*?LkTN959PsY-qv zsGsgr0J3PswGCfU(C?3PbIKur34eMU*yD@;*k*p-a#Z+xLPEF0a`*+H$O(XWhVA@D zlm>@X<;AZdhbX2_^YXY3hM;83-?|KxhR(uquN8Sv)Q>m4GUHbZ?XL5%PM)cd-_&V@ zMT|YiSLoFw&!hGoQ8J{$?i&?q&w8yp2R$ui({t2!k~w}^fe?30cwS|GNLO>k_=AEs z;kdBJu3lLTC#{+f?Btz!B|8k9JtHj(J!zu|mbk{uVTC%wg-0Z~YPcQu_;Z7AEm}RQ zF^)<{Sw+`InOD=XliV?VvaMlt8Pu!Iy|jChb&bA_s?fVPIXSktK;g%?kGv=(l0*P_ zS1jc=e8&4>)3;Wwf_z+`ToJE-Y01E<E5)~VuS)joK>GhR?B#(UO?l^#UtTbpMAWxu zrVAr^rhS}i#a_b^9>yV4?H5pDorPq{K-xBaIQxZ9y)4uI6d|BhiuzbX;0{Ui7LZu? zyJPVIEm3<SWa1M<LpGO%-T=1GdWDCUVs%PKs^|LZmg4eJ?Y_<t8fjtc$3BED5v<8D z(#U;%;p5&EGes<Vk#-&qf8W3~^;O!{YT_31HKoPhvQG5Dfwc0q%8lI(^?yB_@9@C? z*one|H7PU=E7jcb`eF}NLiV`yZlbqZ+W>h$hs2HnYTWXBppjBL`d){5G#{((-bfI{ z&#dhCa{7Z%Z{5;mlC}LI)!X!`BL`~eVeWK8am|MKSLTPss7#Sdqt32)V0;BI_=~i6 zTYs7Lbvbnh+XD<NRcnje;cVTu4T+pwN1glvwBxHhR*u!)2&C%MyIB<>ei@M92Gp8j z;<IF4SA=S{G-;lYl%qD!dPGc4>~RRmAAgiYu&Eh99V%qNg>=6zL~GR_)~Y|lHrPwO znC<1u5XMwI^a=JUAEgmbXQ2Bnp4#MBG1^>%P-+s_Y^&%ny#YZgp4z6Tp>SZ!Hht@% zhkC7|^L2YSfcm<e)Do|~V-(hetj$|=B7ZJa^>+4YQt+TvL}>;qxxrzfE-k!oxJB;J zk(+5RHS;{8HVwee?`7eCex3-tEnd;ch2$18BEfzXSH95U_O>>&$Bf&!rWRjMOn9#2 zC4&T#-Lxkr^XiMrKfP!ZruKaPePaf&7#CJ!LRu!uJbDgwt#o8+vC!`ZDGn*WZ|v#H zp>!rVx&(XORygGf^Y8A4)lMt)Tqf7q9V*hIMdv?!`#^V~N&gLmq=Soi;+{l|%6-`e z@`hyJfL;b34l;OdxyP{pmZxa+EQov^VPh9dY~ba_dDA+l<V`to%^oLm9bUGfB?OE{ zZ^+T_<J#0vE?29Ll0$v!>BJqPDI|TohMG?g`pX&y9^bI3RF2G-E{X@xdd8{$_HwLx zo!CQu_vRL8v13tggT1}edb7>3u2|En6`kW3QXNgVS$n{5__(}D16h<>*})l@C%{84 zYIPzrT?{Q?@g1-0<BTBJtPzlE{0ruU=>b=|wS@fi#sg{v-J&JLk@_;2Ri3n#SR#_g zb3)Ro2dv|1)sv_2vIa5l&6<24!Cw5+({n=n(jU7Ql=2nm`850!dfut_tJHI>EFqB% zZbKvQK?}=nIO))kF*p26n@y#650&E+ozy;o;0;KrRWxihpobAWk<V>d;GDev03SD} z40W(49(pETx@vH|9+gq08FRC(BNCI-GG@wcr$e8Fg*B>c!WZRffu&Z~7QW5xAVe4) z8m|u^(<A&YJQzA<9eo)TS=xY%vHSV{0nP(=KI@@lQXI%SwWM2*30YdhNc@@j;G-k0 zz~eVP5<7o|o7w@lLlhmC9WYf%U4o`?Psv2M-*Y{2*!8fau}N1xAD<on07XxgRtg5w zuy?6%9z+MO+h$f?tq60G^6z3B52R)A@k3>x<&Zh}23r@uv<C7D=cIXVS95^06WRj9 z7r(#NN$m_zB4jh7*bf~FDDs-HS9PSU0w@PvZsjkTOdx6blgW;sgk&DBgik%I+ItOE z(0B6KrxG7LN?j9&tvuDUeY}o&HZFBa-IadsTvzCBb`0>pxG@XTG=R}Q&F@06N}Yve zi+!p<fB3u&NXHZ2`kXu)n2PV;RouAKQR@7viq)O|^z;e8z_ofbH{gzGZkFzf@DLmN zQn=M}9YiU`l!tke=zY%PXmI+Z|De&msss*M>a6uF65Tyta@-ZVC8&v2sWo=j9YY>B zzh-;1>x`mnQi1%=t~`?G*{h0NUlA(bc@kbGXVpL(Y+MRxJM8hgk0ABBVs6S)R(avB z9B&A})%6wLRGJ}$-!yK+@iL=Cbq5BGQtcZF+?FvHYKujmk1G+}>YG6o7@ot<gS(g3 z)d6Q4L=9eO=$}+N%e&u(kl_%m=likv;>9hM_j+`EvJcQ4C4RY65SnAIxA8H8N^9cP zQd6J89XXqw@$cFn?C3-_cJNur&^(gdU2I5eK0);*^mJNn(6+Cm!)oM3jNSN_5H}h_ z-fwu1Dto8(ygKfJ0dw7^oXu@OaZbfO``ZAtkf66Psd|qq!5Qn5?fBuGRZ^V5>MCi( z3R@sRK;f}xCDjLf3ZFYz7LM!ETtvRsX(en@5MY(V*SJZ4z}d-_iQc)gwZNM}t@l<n zjHClZ%yH_;f)da74+!UgD*8Gj#E|VU%zDH=X(Ch#Q_le~_-EJA3B19-p1U)pxUmIf zezbnaduNYB+ck0acN;dk7DU{lGo9wIr->i$xceeB11^@VA;mG!z9hI8eg}sK%LjWw zlip-Ec8HBzc=W!fJ0N!HRSEe7wPqa~#rO&0Tq+}_o**mDG}035fN`ccMHCv0`c4B4 z3V?Up_dx|4?DhQcgd(30TYV(CVWrP?q>K!wr@-?W@1_oJDEo!0TAn~O&<8n4c+qD! zFcg8_;<AFNEe3metp%eSCxs>FhNZ59?e&7wS)z`{E-Dg7m@+wH(6%l|V)``meIoUU z78s}QCiUE;z+vzAIAm{bublKG%rkcKY|4XdX?S7Z)g2bEXT%A0NvsT-LwJ|S1PaJW zETSVqSNx#i9DGqcT%ie6$G1UNYM)%4{!~{Yalx-90+w5JtNcn}*SZT+7kIJ_{rNZD z_z=IYhZQ1=7IYl$edP^bYu?~Bmlgi9y&;XQTNGMp@x7a{u`E0taxRcmp!F*hvd_xp zZ-5P3&YaYXJ>zmd%|5YVZr80O<Oq33yYq+>p$Br6p!X<4V0yEugO5C>wms+RBL94W zvxV=-r0_zn?yucWHFc4EUW)lZO;g{<am!$KNHR3wm}McPH*L=~0V3u`w&;RJ1QyUn z_40hwS-dQ_?KrOMflQH27+8OsNVOlr&J#L5#DY$2u>Y7^X$&LG!}>fr86|xRSgc;- zL$@r5^ZFJ+J}T4Ipp6Y;mNx1DDqb3x1^LU2!ICx80oi2f6&|fALwdj&I6jqi>cibV zb#*+Ls<S=eLL2B!6XICCc;Zb6dtDk9Pv4%vJwP-BQ&3VI^=&%yRJq)9_h*tKrbzxi zrg#X%1siR%tXv?IeyRiWOZ~BESDLNr{o<|0r*q%5bW$8o=0QF-5SA?U!^ONinIF}x z+FiWa7}8<|w~p2#x;mLUyxeno1kDwK^2H}_{ag|W&bg6A>OQ~=>Kce(OLrYfTB-|q zk>1IDx5v><Fp(OWAas86)PmiaGbT-=PPd@@^<#ANLySE!HFVN+exi5180vLmyI1Y# z6pC-6P&t?HRW5h4v+JSikijC*FOd3k6+SRolp5)p!>6;ki|-zK6mt7X5+U?tz@tUB zB6@y+O;`9XqM8$)16!XjY6UxywkH$7nx6dQiH$3|9G2_(7!Dop&<Dq$_#AcWP()x< zJGuq~`A9$!#UN{+7*8h)lV5}TMBK3xF1hZENEG%%NkN}qdkiDQM6|ACu}NNu9l!N> zC9KP_(oPDY`tn3=mAumOBs~3;`%{)le2myslh~HdOb5wZd1r?)`d&Y`MJ^FhNO6DI zXB`Kq>)G+W2D@7pIy~P>`%`R*y)v#^8M+=~-mtNYJfyp>HQY@r+p*4RKg`1?@b&T0 zJ=y*yNyLEMv%Ai!)@K(q?4}k;yDIN}Ah(c5q&8Lt_JZ9h4P-*=210y--j(K~yc@{N zc;?(?YrRs5VWkxO&2m#nHH{7jm~Dg%l(-C)@OO1dLqD!jYOn40+M6_Q0c5V9$Jzj| zYXs5JuI%%AzgpJrh-o+%cBsts&?5$Rj#23{7xOewWJVPrAE_OuSXx0xCj?Qew?WjH z5oUw?VN$pi9^Qt@i`>Rl>l#GB<Et3HyEan~VeQ<Upz1qr=7e;F=T5avn)67~f{#wL zo5Z?QT&;Rc9q-n<Sc&oyvgrS|;^A@LGC1$X`99|TczcYaX_?C1tB=P7kid08K^04e zQ8!N2lATiV49IlUSqw~CtMALh@JI(v5ai`@)L%)9b6Us=$!Sv8I+}V(jRP&7&3DB6 zTMhFj)T(O((x`lzy43TCEGi!}W!mbSNq4I)9R4ugabS?WY=f@82)(5Is0Q(o$TKdX z6!b1GsH7`W?p+<nkap}qPrLCn66!(QG#)Wbhv*YEgf>I5v!dQ*@cw;{Z8=a|dP`)p zSV)Pkm!yIHTB0m=R9tXbI)vX`aSf%lKPke>@)mD9xwnVR1Sza`=xa%&jzA<=t`!V* zlyDE61%7*kSm~OP*b}z4z_X*$&pVeJpO9c$?x?6^oNe!)Eqwu{j&`8&<r(HvF(&mW zFU_(IBMglBdN}=J?yxGa?_~8&w|R<F#81Deauiuxj#hgjIN0QFzPLMT8}3hS0ps0) z6NMaYu%zC)d~LVb9oTbm?YG5}aGiw(d?|ZcJ`CXqY6@DQp9!MWw$`}>byqxdAy;*Q zuGduWY7b?>Z1<~g3NLj9dWJd+4YvL2-M6v*0h1ouF}UcG$dv1g*$STeunZ0Xj#8n1 zTf6}%svzrKL=b^zO#6Acmn4#MyYfelzbQJ4YTuS?078OMA`K^??m_HIdNw#-9H<qA z8Z;sndv^t1SQ8Ou-U6lo`EQ-#=lTjC|JSTEDcJX)Eq-Zqu~<(9;mxUv%FVM^QKD8S zlLMJRArq<WqV6NE`+5INO>nX$<(DdD0d^npJFop;qGI`fOlkFh0nuTgS;It%2*6Lt zfPNv5cf!2r|8PeRGu7jQ#x2~Vid*6SdJ_{9FAV|E+Tk8hXp~C9%P(L3e_uRRod2Iy z>HnANx&Iqb+8Zh8uixb*8+4lP9T*rGZu5*9=FOJbJnFCSP@r`2pDY+=bEQjJ>Avsu zT|W6=x7qN&U=iPyJ^q)vA{V1Bj1RfUmjlB+SJ@9$sQ=uFsfYpkUmVRjD$iWZ(Nhx= zU|>2Cv-p|DKg-}qu5Nh?JENdyuF5kK#Iz6{i=Oxp12jjXz6*bVhL8FunV3zHRh7tr z4q5Z3Pw#WeDTmmRynN&@7r)Qc00$iJP`<8!V%8P9M&%-WJvx>=R=56#KtJ>)fYG1$ z^vMVnmEAE@`jbGgfn21ra**HnM-W9e&PFP30I$h+$!J5at->+%^!k_r4TPTEwfWa# zDmdF!RJe(hLp(;U3XArwgLrnbcDU=>*9Dqjbh_;v@62zn+>0{7c;Eq+0?E?WVMAfW zSq?C$-jp*pxc!@OVg568m|3~Gj$23?kRve0N2>TEQ%m!a3O+fgEM<Pv8Bl2_26B_M zeRKnb^HrX8c3<d=%V0mif|kt<e)`?P=Ws|tee=SOkiifY73bO`ny9BjDf}<&f;qc8 zzdQQrGK&ds?v<h4qTMPgc_#-QYY$k+C;HsyU6g$r3#nuu?ee?ux7*U6R0@;uZcCPp zY38Y@2YlP4x<CBp3&X6XK=(2N9dRbp9FZ71r;z>ak^7hrfc+sq15!>JXSf9oPmTC1 z*>Y%3O4&Ct+nhI!PG{=x6y{e%13#N<o!jdrA4qQiD7hQ^)AsL9_HHxOKBKEtF}nel z--Xvzd~({47m1JnJ6^EdmN;qbjr=!&6l-?30t#I$bQ2!;TdQ)W<69A2=qM^*r_q~K z3z){><z*J_k_^Pw(#0wz-*$mis)yPYRMJkWs2mdk&jT~qsWH8|43(0AZ(DuJRc5BY zsytgU0w;X<L#46!TMmpCn}3M30aTK#{8keMyw+WM(c_yg&KeaQ{tf~0LX@&w0R^=; z^7C1)n8T``mP<){RKmYI4LQ&i!R|5zK>8s75*buYOtp|HNC6CZx6{IB1>b29y&5nn zGi~<-OMRefMa|WnGY~|BDqcqP`^~w~@vTIk{hHG~s~o|3_JPLc{k6-$jjeOtYkcXM z0m5-+&~%fN7b;Ac1@yUB!<%PXorx=bRH}mqzvv8gw+PU^ZouB%ji0sBD7yTAL}^uu zF3~NZX=I@}_nGgpn%2?Z%GKwq^sKR#Q9<aKxkC-W@|mX@`!GPGxqME~)hqyymx|_f z3JgE(oMl)3eqN7}MBMK&6TW@4g>D?$rBXern7EkQfUxpI0qObcchf~Ts^+Uy>rH1? z13#XnxNv0W6gSMolmvLn5J;4dJzt>dAGE}P$6e2!sB325&Uws8X5JDBKE!Bt$EU3W zt2~oQ^dsd{&o!r>HwV?*3*0CoH#6p~u)gxGNHT|gCjxjl3&6b(j=NsSRKYFN#7vD1 zUvCFS)%Z8<mz!D`Ck>v~R|!sGKio*G3pz1@tT&8YJsXq0SHz^Fg@b-86EyaAu1(um zY~zQ7e43-u&MANLx^m2Im4#-ge>d=>b-)Q1<vX{$2>i+XrY%_ZSppbEmC=?%r!y42 zs&LjE6?E*4bTf49i`oW5#CPz6(P5#l^H~R*aqA+EHO~zmo>Egqq)g~9TBc3y`;LFl z5T-i1f<pY*0+qIgpcA)a;@A&Iw}<L|k3f1Gi&?o>-W6nlnM`J@-rgkW$4oUbi|+gn z_TD?H$*f%*jRgx96cL@Fi3$i(R8*u!6j1>|I?^3P6vQY<3khLBL`MY_AtEK9(n|mh zk&*-~M2G@X6_9`k0U|^QAtWKm-2rDt=R5PAIqSE6_pWvBm$m!@llR^GdG^!W9#mxl zY<_N88d<oDjUT}i`EO~csXn&J=j#YwH7Q6xIKaSrPCAB3xg7fA4vpsJ+h$qXJJ?)0 zO<|6uU?lu}R5;CH3K=NrJK@70P_u(Q&pqSH6k&jn$3vG!`UQ;OM|1={EEGla5Wb+H z`0w<ah4Dgm^O4UP912R9i;t<3^%5bZ;a`^HmGn+cK=~hO7y%hoVH+%YhnHJ#ABR(9 zkc1Iytsx`bMCwD-Y^U(+97^~OGk!1K<g+0Ea+23un04r8-DN*gXOkB-$)(`+y&>iG zr<83Sr4Vzom)F`tMmoVu?9Vg?Bwklf%-<=_)exs8d41JfFlDB5cdrkY={GQ%?T=2M zbPOA0%oXfh@mtaTDr87pwIa|6wYd9S4X&SX99tEY`TDo~o)FqIri)aQ7Mc2&Q!_wv zulUT_X9$KuLd;HVEx#ZyoX`oWch45z0J*d#C*r1{A)MNyy6?v8a!x0nFC7z`JjnRV zK7ed5*A}QX3tlKNq#l@@`GEM^g?gqvqqa|TQUA{td}+Q2r}{Af#{G;ry67v>Q8rG7 z{CJe|wkl`o4tb}}N#60+4x$VIKA@cxi_|>9mo9e6I-7OeB>8Q-Ml9F_v*BN!)t#yE zDzyfW4kdauv$sIHwv&X{&ZHbS={z21NIEi56Y%BZcg$dM?8Q$*?|b6vOu<i*t>O%+ z(Gb_fIf2aGBVd~<x(hoj`+kSoiferCdBOCT$#{dG80pR+fU-R?#e*G_PBA%?{`tKK z!#l}+-(mR8)xBxfafI>B7w&<daKKLtSJdunV2~6~osql~bl0mHW_<uF8~&Xj*tMbc zlIa34Xa54soj-vQq5Q2QiQ7S%<;;fja9!>^coSy9lcn4^2j1cS-@U`wKcH_x{!We+ zVlQ4Z2%q`FS=Vncb*5$J@ySPTwr{1VkTW(s`4_Y-$9-SOLZ>7e_aq*=y$6sgMmF5? z&A;G8zWsX@QKV(H%->3UU!%p=azwxv-QBDHijdHg->bD!qQOs34OQEBXy14})lf!z zss8O~|HKA><j<m+TJApq;JdPy{{Zlx6Bzgp3H>)rJTKV#nsgT`*vu^NEu#Bbn34|s z!I*ftXvih8|8Jz>|0>b`A7K8!Rl0Y*nSB^O0ev@~611J1!7zrD9FGvLY1o>mj)8{K zLfp-aG`sG|h6D33Z0LRUcfLVN!Zq#2GQbEx8LFphAqG}yDJh`o$9^y_P8o4r_AdPI zzqLw6?;+@<aDSS~XGzXGA!oR$LQ{$3JaxG;?guMEZ=dk3jt7-9jx!OwBJm!j`@5M< z;XV%r$N^o6-;w4RN2p$y&((qfm9Hz$A9y4i-n0At4()VFrMi!w{Tj$adcY3(!(zHK z+P-7ugtCYEKY+4L)mpOm1Th)mTC(j2bI9`c-TsUJa2t2kL7g7K10@h)J(1Zaq!rHt z@$$mL4v5PaJyYiJb++HM$W=CvLb0o<p@QX%xuyn=hme5}sR&YSpg!b0b3GdHg)?s_ ze{Y|jJ@j+a4su`c>ivxtx!cX7&<L^G?7J#wng5E&Q<62GcS1Ci<}@BG96j}+$cmcy z&Ye_#Ja8-Ei$B@((VHt~_m+kv{+BN@i>m&oJNhiEREL@KKq0F86j=;TT7GZH2l&M7 z44EoKqfj&DVZQ{=Iej#(>|{4V1-E<H6Z%v^M-K`+&i?`xJV_6dGt*#3leh$Q=c#;D z36wWL;xE^ySSV2o5s3Ge<IGSQP#9uC*o14UI>Kt_xnN;6R)Mg?hU$)VIg-{3AKehc z+#iU%EnoNO@vnRd;dC$oBn6qewv~)>R`29^yl?~q%^+M5NXtPT@YW?PL%~r}7_Se> z42c+@q56djd*THbeB5GfD07J@)*m}!qL^oVKn+oNQoKY@K2jd$HNm@^&wki>3ZIt& zN6}fg+2<$x^J0g534?6!S#??o?cTg8EixOw#pef)j{si)iUKiV|48QRZw6<ZD%fgi zXNroxOH8@N#ya9GM(CrZfyeH3d{wS?XQ-k+zsu!Sg==t{>)BbRTH%n_V9gXB71l{F zXeT;UYI=Bp0nLYbt_{rrvrOlimf$hm5I4uBIo;V~+>cO__0lnhI5`;WPX0>#40p+= zeayYDtD`%Uc3-Uj!Dlu#L48m9gTQ-B{n9!55AAnQ?ACulf6{3twz_Vl6qqFU{G96! z+t?HE;c`IAKyzVHa{O(k$6T>T2CGU9hqW)Ebaj~(vs;WDr=G|PYLX3~ro;U!iTUk; zxt)$YXRP0jN&76=+-z3Y?i#JiK8LX|w(8lY6Z5XjpO!l)693cQy<%4GRAG9&^sZPn zmfs~DY6{#_2@4XdK{O?(NjMm5SoNLA7C&1Vr^xB$KB;_#Q)?dIxyPa4y#p4CHoS)f zh+z9YYxOAcQNtlhw6)pDysj=a{|S9!foIqko=vj{y&&o@;HxJWTT2$5shgk(JE+}Z zmdfnnl1Oy(Pc}n!YI#Gjq<p(N^{(C9F)pb6_-777mB=IYW*d9PShVukau{YOCt{5w zm%_5hPHql5J4}>dah<c@BFT;$)#H&KV;ny>`(5M=5J$-TJ^<;74BK-q+$PWZte(;- z^&ZDhdizd63KyR;@;SC~yuP`3aeur0V1aA0%9ny7onG;00v!dcMbgAn*x*AK!ZqiM z&u(A#`xq;pz%jDBw)1O861(8NV|Js%=w**ZyP=et->8?W`x@KrSF@3V;;8;o>WkuA zWZp@vHYLlJ4s+WV;B>S&Ftpi)aa==(EgnY(-1Dc8lw+ItsvN0QVw)&ds+KSmc+KwJ z#kpQJh^5=Ul16V1?{OpiMR6n@#hh%k8`Z?<XjXOj<qciy*^KW*=0Q>zhrORI_HYZs z{)V(KxR4~z-nfqW<W1r18a%z&!tK>1DvhePVlB_Gw=uLvf~EIFRu3oWwBHsR)UFrH zu3=@g2LxjKLoMdzpoH)DA=yr}N_2%v%+KCAws&8Z=Sy};=5KYBu5)($wQ!H85eyW; z(l30}!*`}+SDNup8R=lO&x|=#;$W2$w!G9KGLl}5Y?A!NUj?JH9ZFGrVb7o96z^o8 z*r?)i<V`+pq8dKX&TCcMVp$BuS~`6QNowFJ{|jFJUt0Cw(@Uz_WsM~DG2F$=sWD{^ zqp)MiBRRi$AD&`WlyG$-5XTBF4p>Te<;-mJaB=%d3!`nB#0&7qDS3d#7PrDt<c333 zyOq1yypyV;>f#`Tz4cy<4ok4+{6<-qm^NFWNN=3_-Ck(RZ}{bd-3`C|GuGIkn?35c zz?6)`<t94z>N0n+_@EBb<ltdv@8_4xpY6vfaJt7DLpf=e3wqJKJwY8c2>+x`p1!3t zH^=O{s5m^VvQM#iu<*BsoSJ+#>{wy{d$qKTyYp5^^!FVDk*uMH%<<z{rYWQM&Xiq1 z?EJ`6#@}i1k6RULzf}g>sW!Rhd=kak@r{g1mq$auDWizI*NzRy6g8TCSc!NrE7Y4h z9vyqe80MRT@Yq>w<F1T8Plye~-aF9L%Wvok$=h-OP=GGEZ}$(r_-On8^~LZz4D!B@ zOFMaFtnSW`^1YX0dA#s-kZ*uYN=^DDeqhp<vF5@A9rvzwd#0MbxD1rt(lLI`PPOc} zVEa8_%=r}SjRpqv;Laiwv4Os*sVBXTrM-Apq6)a@rmEXBCum=tYlwGKV9p-mgZ^Ir zgVY~_8kQ&iFRum6%D+EQ;|n)F?|06>+ZUSB9bf7FEFTG-o~vx=(#AgpbHE1KZ+qEF z)OaNRd>;U)Pkp7Oo6_#KD*J$v>buPiWVRBU5VY~-zp$oQnbO^!bJ=F_L2qB^aPmOT z48Mg*Q}$@?Z3WY@;;A~ohCo1PKIVi`132&}+YbaKy|kb^G=SPvBNYWIx|iK)zm5U$ z^l$gxP*Cz_f3G;3*x+xGZFV8d=Hlx#R(oJQ<*E2N_VEexzPM1eN>?MJq$=lMl9WNU zCUN6rLQU1D-n;#?`<r-oDMP#i_{Z<fh$@v6>RUQHAhA6T&Buv4m4bg^ySw&oDgMtk zP9PI|?wcjOpTajL=+g6Px6tP|woD}UqHE+P?Lf;mLM;gQQtgBT2Zq@BNii(w6}a1G z(m#!@gr=}BUp!uxgs=+uc<NwL;i9$p;*3NDEPyXEmj0c9i536FFDEE`pz#uK01<Jv z7}?BkAm;|W39w)NhL1_jzw2HCo8aA{3~{YgR}NPNV#>FN`Yycodo2=}x258&Gk#Oa zYP>=d&8M@Q-HQR+C96BWs_hk4rl^e$afgWtm98a1&_Yt8&sH^$%2n<{nf-gwACUJF z7aZ}`k%SKn7=O#)bDNmX`GxW&WsFKFXVz|56P;b{rET9@&G1P9TMPtx3fT}uGa9P0 zIssPt{66p(=}5o3R3OG)U-4a=^sKW9%5kLja11BGt&F$R!Nn4ehOS=JK-AP^{u?(< z^APgcg2y4$<~?W%izXNz<lUt=Asd4*4_t++^Zv=-vUEe1WY>erdN4D?NFx82GaHA; z_O1S@FIv6zMACG?$3SYlYsqh^gJjx8Qj%Af8wLrweA+ku9H=ny7j}L7fAF4vOU^g^ zE|y)mQNxT!4QBht+d8g%L-Q$c!q*dN#Lt7ax=&r{omEuM3|3y<<m{b`^H`W_2?)ef z&D_U#qxk~T98b&89Uvq$?=N^Qf4L9uRAC}7d_+Z@?-^{jU(DpBsC5>DQ_=#bv{s|5 z3OAvHqIIp6-umR2&irwOovW<ufhl2U?o$Nsps2alL7Y})T&&se_TqmPAH8C>cH_&= zMhW3KH1vG6y>j96-r45zlgSNHr}*JUG$YXCaW~#-s3m~I9V-aqq&Wms=Cg;0n4zp8 za52(QlPlS-o`Rbp*r&Y-GO+t0l&iN-#0W#p!ncX&3K#EkwKung>Z`m{)bDqYXn_-n zbN=`KCvOIB-QSus0f1FZ$M(OlPmqD(FK*Ch(JJQ>z&c4{Ox@!4;Hl>Nh7B?EXhSnt zy?nzf`-u6hX1=<F2AaX_E#!Cj=(vMMm7tBU5KURlG}phVHCeUm#doUSUfTG%zNfh7 zr6oU16S{rX*&y#agyl|L`rMPtQ+R72v<_2-!VN=E5^lF)zWhKa1^mr3VEdw2A8w=l z#>3r??8XF+Fm)aOI|c>7%((r84T`DDZ*}5xW-9}+Z`h4g!4Nu`SLea+Gmno~DEvGn zA~5MKblh;@o>MJ_XWm_WjTEhkYmBhPG#n`IIgXz(q@w2x<kkLSu^$#I+gHwL3@zTd z+A^tLfST!J4Bh=gY$i;8teoKsmIq{;%UP_b+k9B_7lpyopI>jfu~n)%#=8H#L*o0p zr60Gb@4VZ)$xH5LUUrK7>5-e)SKKa3dQV2KR?bV>Y;HKXO50?|#i-L4Euv-m9yRNS zkDp@X68$;~2#y_nunuNz^w~aVoNq@wuJCE6Pr)?1<^nyHBluuNMM(1-6;LzvQ;3vl zmU&z;ZPBsuLiA<BOcld(XoDHoxZ+JQd^6g19d6o{_F$-Hgi%vA+c4wEqNc_s@_eJK zC_@WzxDkJAQSIF9nU>QwJ633?e{wolgcpw->aktXLJ7TIIuqtXxD7MoJ!(l`-iN5{ zvJSXl2<^1?Y+?nS8C`QVqPITo$a$G}aQPzHRR>IAG56jX@o2Xm`E9isID#|h71PaN zO4ECCN09Nr&V#&{j1Zc`6GrXGi+>^5L)57mha%nU4`0PK_l)@?Y`Ya=CJep~QwVW) zz37~>xW$1b*lp&pEz^VhgVoJ?u6Hs$hn#A6!*@QuHS+=LBkQ)|j<{BVTQ|(j5xz4E zt$TzKZg6vPT2gxae3$S%fJKrDKP`#mOg3b`czkl*S-n@ukxp70KTY*JP3sQq@O1D+ zlZGtm9nXIulEm+ar64M&hm@E>Qtr$l`E$1#w+F2@nAWMth#eP9Lxsbc7LOGYN1S*r zi-OV~1*&ue3rzy{h_5uI>8sJ`v-^Edw|Y$z$spqbxv-KXR5E;hr9fj*nY-G{p}Mk> zz~|PrMeUj>N_uBu%qs+H<1XeNPQWO|e3-tYW|(5wvN`Ga*yhj?*abt09^@qSqs+Y< z(_WHteyE+9ae-GR9(PY*j#-*2){P*J4k6^IrWzLTTYc<MX|V%4^%dqDoaZ5SQZm}J z48j~tG>$!<wdKu~-{jZXCY)GQEFb5;Q!UZ1Rzs`MGmSKfwZlI5g15P;VTB#3!w*~B zrcDjBY*)!8ciN^RDjA7(r5acdfBV!a9row}4%|ASnbHvlD^VcjUBtcX$@KIPEF-ng zNcFW%b?U4$C7X6VRNy^b1A9_oUu6{6!IKB-8B(e=CqPecY0HNnv@qT;_7r}gqBzgO zW)Z%oDC)AHo@7H1j~_h|l3ycmggv-<DCC{YWQ~Q!6(?sa|EVFOl}%kmO$5%~zmeLT zJNTJ?#Mns7QJ+{DG54^l!xR=Bh2+T0+4R_kT2Wl62RF;fxZRBS(zYL5$Kf!_J>c*U zh~bEY3u`YmF1FU$NR{q?vyZF7GSx?Ia3jhIUCN~%G6RYSo*?9KWHiMle42eO<xr@5 z$VI_Sjqc&7R_I1IZNm9d=yk_TP%i*EZenbQyZBSg0b92M6+NSdD{1htzYx9Qg^o6a zzWic8&f_Ay2Aqy*ml>ZsafK5+2Wzmo-bBG&Yn}>fm-o;#AAd)E<SjlB_OX)F!yI0{ z(oAS~<?VD%U1ycI)F+Fe{EU&=?QNIG^8W!tKa70_e^4=iS(=<})iG>}QPCva(B#ft zc-x(Oo=~Kv^`YEdqh(~cPDcWw%+!&Fr}QZrfB0F-f_C2Qqs{5o&cSVd7(*j_M!3Zr zP=}WMmFLkKZIL^sSdx+BKdbVq_oywd@*VAlIqcz0BdE)hne)zCbc%UJk>jepRbgM> zE{SZIyB*`66LRci#WJzBum?7HyZcXBv#+~-;;jIcG(Uv+c+<S*8}{d^xA5iBdWdHP zV$aAfRjI2fmhTR`8lLF+`qkYp^aNLCg&!uVgSt^rIG&h%JEjpK=iy!+WQeoHw6dnc zWFGBGUkr~>XX1ClkL@bh)*9dS9<CSBom`QQvUFmNFgPkvxt-}_>?${nHk@^pH2kk( z$?#=I+tW1?4BWM@$E06NXHK2syA6=Kjn02?%6ro_Ii%#$WvkCpb<@J&cZ;{FX41%e zP}L7(mp;<l$`8F=oI9B7f4+x9yJ{aV$Utok_5GM1f$izo<0g~smlZqDs!qwEut!c* zIVWsJyTi}3wV_OH=$O}h2loO3o-c3w4gtFyPHT(Nr7UDtEz6*^*S9wrcsknDJxdNi zXT~l?hTYWD)H8zXpQi5~%VHZUpIrp;G@s>bjT$Sf7nY_(1&Qa}7F*Kn?d#?1BVW2R z-_zB+(!Awx6tU)r+$K2%LV77{Gpl9(7$~2RFh-%#*@fBaU@MpC?`j>V1bm+3afs73 z&t2S)B5`sxG=EsJZmu{MZ_EgfH3FsIrHkg7e8VTD)Yzp=ynNp>0%KBLyi~o9JtMhL zIt<<x$y&eq>h3LTO1Jc}*CKbV&{k|J?sLz>&yH>&aUYlVeW7`~5+=1!fj2u@d(cH} z!-?oW<j00jl4IjB@S7N;$wDh}`r8Qi`VJZ4l#g{W-COh2JfExIWD^KizYV*GKs=F4 zmz-XGY>E1HHsP?s>gN{M0}DMpjT+SajE!T&`L-|6+y>g+KK3(-KZ*Lfd7ft`^k!#N zl=wz&?lmn|Vzbgeh2#(lDzHLb*rh8YCY-K6s8v{-iLSmvU|d#M3W?MhV3Qs%Pg#dP znMFd$8r(FkgA#VPP}Fvf3~9Rkv)!BL=`82IEP5U(4pD|qVI$PkA0=dUg^a?fgQ6pF zdCou`m)N7IncXKl(wri`L=AEu!SN7WaWYjrjAgX4h~y_bYRf~ZJ?6(m<xTUS&ZE25 z)kp5FjS8jePuGEyeRdCvn0XX^qiZ)wh{moSt6cNf*N>8TZ?x8A>?6-{Lx>0RX!`*J z`B+DKPZIIyge~9R3n_~C+;VHt`AD%zV#r9e@%@#Tj|&ATqi5V{73kjkk#9SCmTcT{ z%jhV=1t?1-*iXe(s_?G-nM1DK!YLiBJMzm01$hJ1@YwjT;!>uVZA1;Qk5f{fr)n7E zN`DO6htRieUg34{XHVlAh8)*(-PuD1vEmRSYm>(_Zhcq8_%*Kjr2kw^DIFQl16%v1 zkgtYnZiLK5LrExBWh~}dmXSF^2CDVtpDplG@E3%9GWgGfzhnPD$oV&<`M&}=qS5VN zCq7my@J|cFIZG5~YkA1I`HCnJ=l6FGE_o*sxj+&xh}`!cjPTy&e`+L?w4;C6WT~Rk ztNBmSHIW`fBWAvQu$P1+*QrFWO*{mI#(Ey$-0z%*mFsEmezz9}zf+e#+6(K-NPX=4 z_r?B{x|ikWh4>|hU?Yb=bX%zz;>>EuF!`S>_V>oge-`@>8vm{J`X2)MUqm4D1}*V> zQyri2gXmJ+d@CYE6h%?|1w{Zh$rjL<wrs&+x_%&j!TgtIgom$O7|{l#UoCgVvgex< zmN~5x+0vV@rpyz}4@T*K9{m3WauUMX6Kl8_gn*A-3%r&0jx23PxXAhHvj?434#!7N zO<VOv91zi#M@MtbHwR1jR|m^&6@-$xati}B)qp`ntUGt;CDACHX&y~fNfGJ9;*)S` z<ZLljZs~({WmxK5p_T+!<bqQ^@nKqu#tRA=wD~C19ej_2l&-eQF!*U2%$T)RkePoX zp2SHprY3^Yl9>{qOH0knnjb0%Ka-dJs%-Lwj+(3YS^mu-5d764_~HzlNLn*Sg6nbD zYVEcr+D#I2e5^?eaju2&c<~0Xv`glUdZSA0dK$BtD8v;?2xUak%G+$xU;VnS#kB#w z)giE!<wJZbC&{=J*b1{Fg_xhGE5E%dYg(P#&^$M`<^9#`$ho;_>gH)(@UmnwkLrF% zG&OGiQ)(3TYpLHNPhC+nn6ZL|+Uw5)$|eTx1mjZab%^PaJ0aG<Z?(4#m$px6cyZ!- zXbn8tQ)A2Pc~?rI_b;wgum(guwomFZtts6YYB6>@s=05jb<3Bl*T=`0AFTwRnT{bS z&ZcEZb+is1aDM0JaOn;iF%rkoL;h;|x-vH;kvD*kV2_~%3=i4q-6Ggy*>(Q~?B$>f z<xGpj$8ZeyeURtZNgZhP^$m8^Nr6;Dsta&^3!fU?jF?MQs9w8TK!zct*Q?|a2l=XZ zfulAw8S(5dj+*FPGUkbI>sHN2T<XnDS~V`fIJ>Yc<H-7E6+j%cI^L)M0&8J=RbPHq z|1`Bn74(?ZKH2f9^7ARFencqhzd%gd@)K)6l2E7?mGzh9S*LWAy#Z18#rRmi^d15a z)r4ZeLtjlw`nY@r0-VdWawl-QDd3KsEGoYdTE(64(<jBwpIPlK;MfBK2m_rs4ILk= zLzW{qtmowkSYJM9GekGNU3vw!(mSOte(9%Bs_3mN`qnHnntN5KqYyzGEAGOsTfbh0 zUl(-ls|!i6EdmBM0uH%p;jP|aT&=AU@7nw0ETtIvzA%~`!xwzL4uJOyG)!9TS!l4) zMe)HM?U&`&%3d=9ugaZ`E@Th{5u|3(dmsmo8qL;85FfPrWZ#8c3Nt_n=D7YF9uy(x zW_P>a$MAY-%TO<vg*Tj7GAz;U3<u;hj%c}nf41PfFK*OV(23FOT2!Z_AUM5l`k;x4 zfo0SJxl;PKd4chN*Od9w!RwSolc*s5vb~jovCUnJA!+p>q@T5Vk+F$M+FJ&(g2`LB zD&|T(h^0X?*AT3aDjM^(L=V;8t>Mq~bTvN&y*<~o7DFO~MP{*gNgrZ-$T)Aw`t@b8 zFr&G+=OYn}1<b_Fz6K*RpHBAIMbmGb`u!!3ecl0@;cNxtC44~iZe1W>x(@FytlQH5 z>-FpJ&5iU|ix%6ZC5l8t25UkmKf!JG=c7c!FCb2fQPN1kT;--kkjVX_)pt9Bdr=#V z?~7K?g%Je`bAlzJA423N@9c&^Hme|g4>YKNel$Sbsvz7A84<y&I6U^Y%R}y}E;xm7 z%EH%>;TIx^J=3Pc@Y^w<&kY=H_E(>Gpy4)bJ=m+){9akZFQBqZ=bz(!#LjIa0Urf? z4yuTJ0?61b?c-epXgmd<VP&{9A&?fgY02+fjb68W>()=QL_nSqAa(>kk(ToqYM*Sb zR^q%7jpu^Ad7G$WwX8X7Dm|oc@*%)OlW4zZ*L?yfn)WJI*avE4MW2?HAU6St)Z?b5 zS~Z#(fyPwu2@GWP5$Xc-Kxb3gCLpsf&!gzY{JN`&9B$!X)_q6oMc6n@rGN%l2;_dE zNW^9sCyI*#TW97U{kG+Ap}6}J^K0%Ab?|x~+V4o2hkZD6I)C6Z;C%$R1Tyalv&`oD zPgf5B%+eAT;ySg6c_{Cxes>m{W|*HXr$9F=#Nw<7vDzyxZ`r!FNpV4uL~_20PH%@T zxBU{3_@Wc7m2EfLm-ZRz1X>H<e20?5-()g#Kt$y0v}f`Mw!<!oHWt|@>ZOdPS+arO zN&FcCY5%}+QhmAlkSq`mfyWktS)NAo2lq?8=;doPMR;hD5rgn6wdW52DA3G*vs#<p zZi<X(gb5^5MO?tq_GwVD%9tGk_q+T(P@DI7|Kc#U4D{iGQLslSxz`fe3@W?h-!8Q4 zOykQUJJ3A}ppt(|e5VI4fOz`_Q6%-tGR3tfCNFijFT@XaDa<p?;T+31Qw^u*SE%cO zkqkpSWo_wzN;iO9Pt1Me>wsG>ul;69zKI1{JV@<{oQOX-rQZ&;>Fy0}l1;>e;r@I7 z{z!YzjOAWSNt+)s>kcG}&(}0vWf-{1Z3h!H-$p~{2y-J6C>;+$?Q)~95enKdssDog zOu#l5_poeVSTu=P;-Ewvr~T3the>mngFxoi2?@RCTTWm7am&+3mUuMq*bEP)MUZ~w z2d-<BB_4PONQ>Z&8@g|`Mg?OK_6cCLcB5jQ*N?P#&|_Y2&p*bZHmdFc{kz~p{`%?F z&AKN;QVIs4{q}(T|J{>-o@C}6EUoGDlEz{P^vp;5P*nRW1IdLTWV`cF6nW2&<V>7b z5s=#OG%g^|Z^9*i<V2*v6$}Py&IrRnAgN^3PPXK+&8GpYzqFqvvI&f9L|GUZp~jz0 z$H=IY&V4(uX{F68ssXTlgY5uR&gWes04xSXe^5gQ*I<8!crs^UL+*=LZUflHw~&Mi z5+y`Fi%Nvgay*D@-@eB1Y0%4Z4>^;lDX%V};I<aOJ17M;+vgX}@{15QyH~pF<fGVo zj^nmzrA`NsXFC;;r%z&M$P^{q8_Lmww3+i`m+L6#gnqr>L#oTaVb#!*|0l;v&Hq6= z35iq`xKyGKK3@^Nj;vvmRpS^ze-yo|{k}7eA?QIbV;xaDhQ_+A;;UYu_1wkn@+efD zzVEVX#rYI8%$m+0?TTP>-^0^GtEU6$l*!;Reh1p)>9=@2S90}LV7NjUt#sQL$BlI7 zbw}|uH{O1Ga~1@mGf8bAER{6feD%P^8}fbnri8E-dJs~de03=l$vzxiZmZq_R+5vE ze8Yd;EVM&MZp%0d#SjsNYLL{v18^BEV=k>TVu_~BH!*`;x$JofkgOr2tUpgu8ZIQO z!FMV`AT_Z<%L%1~UC+0?yu8Y;l)U7m4XnrZs}=0IZK)<w%?Z`!6HBORcM&59dgT?3 z!x~dLW-#+2v8xXgJh{{5c90e{+K$tyqxETMg!T>%>zHFd>cnW-Gg66j55FP11?{;7 zSYD7&^l}OL)e+mQtXU#)2f6i*2New+zXMwg@obxAeEvoOn@XPh%17ADo>*k4MJKY3 zdaQU6#jQr4m=Ry8_H1OxWiFOMkghn@x%iO0WRaYL*yf<sPhszTZYSa=i~U?Wn=x9{ z0`WDuUN-b*v~Pr^A8FAh)o)^Yfv#)4Cqc8hht(J`(}ZJ9>H5(|H?Nn4097=hGX@o1 zcrjrP_P1*dunOZ7fhkF;=Y9EObFnj0nW<CB`Y{u88gN|ZZ5UQsZ7RW2{#o080^+8i z^JUFAF4$`2$|^NFdE`F)YYn%OIpngBAp?f8<`2`J&d6N4^G!dEnD1H#OzAWQ<XQ_@ zy4P0eGVFt*p}=skcu=VHk+h@0P|{q^z=W^derwl?%dXEbL|&GR;!lK)Zz#7$PaB;Q z|2aca@(tC=p~v*CLCa%gvydRY4iNc{N4v7qer_~4;_s91?^cj2y962n(qIJdfH9ml zK38@_ibA3@>f&U3g-HRUBQC5@&<(3SVz!`SGPUMCqDw{@P*eoJVkH9&T(lT@F#TxQ z4J0r;+rFy*_Ux9Z_&|Yy{$pzb-CpPWIXTabZ7?5-V-S`rSAqtC8Ot?$)iRU1a;>Db zHcH;h&g|bwV!bo|VB1mg6!mWA@$(zqxn9t)v@yq)R_fP0l3Y0Ais3o_Tw+UsEjcgP zEy+#>SArp3S^c!Qn^qQxM0&{QR|Tc}^4|HButIRwya{?$e#RMYbsUa?!KQu*a{D~~ zx~mJ9pjF6v*Djro$uso5nJgEmk<X9t9}XJsvLPBkOYnz&V2(&;rjA)C%H(C(&<T-3 zWjqH~^_0k}HmZUC&Kv{q13q!J&J+0{ALt(?4&mtS<Da-Ht~#Wc>wNI_i<9J~=rgK2 z-b+gQbwKkCa=NvZB6IswW3gLv+Y3vbq#gza$vcnPZP}~Wq~uZ(?m<S=#t;RT9=VUZ z^@7*T?#m;Mx`w))Pw8P+eJxiqN1CxLb)mS@te${grcUTEN(;yS<wtgMl2`TyK1-sg z@<2o^1{l$aEN9&O<3!}U6UCjvD&4s=IoFo*{UeKEzb1y2o`C(-MonJ3(u}%1KOqgR zU+=uGNcwqO*-xwT8<HUo7b&!RxmRC(cFMHuGV%8?kL#chA;SWEA)30uMRPyfeQ1r~ zRuQnPv>f-0+4`FMrz&;$JgWH2-Cd9p7b3vt5jJW3$ZO>z+O~ZX>o1>Tg&hIbfQAUU z9N-9fhSeziiQIKsV&22z;u$|nwo{o92Z~JpW71^}q$NvMu`Z9;UEy;3d2E5>qL&7U zoPY@VGvb!X(?L?@t<=`C7U38o6jCG|!EDKDD(+4lYY)3QrY^>v*3sH&n{MN;D#r1p z)ls-pk1;MOMyaWG`av~>G<B9C)PchlXJ%VP@WO7D+`N(&Pta_Q2u(#m`87xV??gmQ zliu~H`O#Fw59Kr;&-FXf(?!Y+RB+qr&Nu?RVNV%X0*eO)H%bj?0SE4@#B%6TV5TuR zOANQT6IfrV9!NGWoobbNdtUfCCF_SwtmdAcNcWg+>3D>V7#kZG3c?*7g}fk>wEibL zq$?joCWKAc)m(=OK95cC1IERLs4|~-62epoH>98r=8EL^fJ~l{kA)t5xiW%@B_ckC z7;#cO<fb)54)jg-3z6nnQ9r8Hw`0d6lIpV!Vr8JD;26*y%v}lHA6iW%&SVEoWjbj@ zey+o|Cws9DwzXTS934Xp1vT3A7uoeuwi})~wMuYYBJ_x{Hc8qy$aJ85WW!VUT#H{@ zjwP|_q$?K*?jD-_2T8asep1(VvTmHC+Hjl|t~UjaSA7lCAXv2a*!hOH9*!x$67kf^ z9aTKucl42;CRShkhM89QJ0SL)bmko(k%#Kj!dbt5y^Eb^LTUe*Z_x=~Ot9^Nuk-O7 z_@S4BcAh8sjXdYORs>)ib><n~RFj<rzO4QVL&Z;*)pq4Y3?Xj=`ONG`UefyY>iT7a zQ~26$AM@<zI-|EQ<96BHl;`Hh{AO?|>A8#VVFBCb^^Va+PYj+8@kZ&&hLLvScX4Ap zWx^Xf!)d!!xk`qa4&7Q*VVT#2;<3@LIFG>WNYB<KXXT4-^=&=SX*o)Jc%>JC0##Fk zb_F{f$D5j?XX+@&82aVGZ<XWE+tPyu{0mq<jM%CB%ndPz`A_OJ1_tsBTPbcL<=@%R zfsf!H?L%&`YncRa=W+>MR0jL!9UJBi0?K7zr5P*S6W>E%f_Y<C)7DMy?ZBED*8!aw z3S1+I)4psUYq1o;ES99VLRazA`2vrms3^emAO}@9Z!+8>Qzgk!QI+EBMm5|7m95gb zSAR2*y|+ARs@7@z_UTsY;~bsrY;O+L@7J9Y&Pj)Ci{W|4;VJoQQrW@BW3Xm0SBibV z8tkxCw_ZhYmF1NaqwgE$oF~_1Ef;LH&D)99mWbkn$yaLz@aIg)7r$(l4NrTl5I<rN zN>#<RrPHet`f{96GkPx1e`}bVGg(yYMW693p57x82$8JGiw6^$#kXz!#b>4X{%;_E z1zH${$ZTK+%u5@#&RD}nrj1lsl_D`wd{PuKYBuNvF|BqPJwmgn$ikp+`MkloOB!|4 zF7%iprnA)3W)arr`s;K_wSt(X{KPz5zpcPwuf(IFAoZUl?}Z@)eEZ8|HE1s=7AvVq z)6LMMu-McB()nH8=)1)DcdRrmG_e=k(9QbHbs2x?^~HbMuQHcd+*)=)Lxmur+CJ)S z#tp6!hJAd;xjF5b|2r^sGE0F&qKvx4O4kx9L{NWK{1!t5wu3x~Aqqnxyw~}Hcw};) za$YT%;!-QU!|-Orlr}iJeITf54%-0YN1nPh!f{}G9Rl#|wC_J&ry)UYw|jO;ePsAF z#yKxf8US#=OzOZCzB+8PhKIwdAkO8l6&4-P6m|aixxfc~>RpmW%!3|wXA21NwEQ40 zPI}yn=*kTsRq&l<v^{jHqzQr@$3hEiOID5gt0g50I*Db}-&A|f^9{=BG&gw4&E$kF zwN=kR$f?@+=us1sFGHzV;c^nYHeo?%T?4TMQJ|Jb683Fr0Y`zH?yy?J&9EZ@nBFhW z_IZaz@z~4f1EOHMN`2;^p<3V<>{?5b+&E%=>EwW{gx64<m>BuFQKfwUWWSZ*hP7A% z0;#o)e><8}zT<w<G;#(2#y$a0v2E9FDYJi1>2cTAQy@kF*yK0svqH^KOL8i*D)1xY zHx`FR6d@{?)1_RQ<L$DAC^M?+2WC}*ZTIIOuApn0IoLB<MZlobsuh|v0p$hNxrh!c zn8cz(Kd}{_9iKP3)L#H0yPR(|m4Esr$9{7c?@ySljrQF-KoTGPR_!ot`tdUey+OGz zjAo~RSNJQNH>=uvX4f*34XaZh+1mC(-)11M)nzgVa9i!5@g9`9r<CQJXfJUY&J==w zF06HleP41umzbaKKD;@o*Ko7$s}aO3ai{v2cg|RkAuH50DC3ta8lgXVOpr(I2m)0k zIeoC|eyg%DZ7QD|jHrZ>9<yJl(%r9$q)4>UBHfYCK+_QTee~61xB8>si7J+=Ll|qh z)F6e`i%(n^rJ|>RRr^OYz5Rv7?>QM!^`uVB{kqL_jrOyzshP^=#^s$S?@U9iq_$HI z|0W1?#3_1sWSM$EJ><n++?9vi?A~)k;zCEo$Aa$97`L~#`!3Li9x>mAQ?v2@ae7v} z^%UIe(Y(ug_E;*<a&$w_glpH4_8c2s1|@f_XL5+hziVAx^JZRS8h*_`ZDQhTVx()( zXs$(gYHkFEim0uGQ2I9<5*74_3QY8;jRE(s>0c-!`YxS)QG9pw?%CTTFsbQU_ws#q z+YM_sh&4#vGox(0(1;!yI6}zRf>)n*M@g+VheOnTDN-(nor-;_bAi=1<jk4Tj%4P* zul1dmoZt45ZMnpqQU&mMP+Xhc#Q=A*Uim`kQ-$**N*?aHI9k<0yJtnv%(=(JN94>X z6yiV9XU5TXZsd~i^&$l(u@J#i5%CK{VorXk_muT+h|@C6YCQ{avC|+qfc>#J?fX)I zq(c^fa#k81%W26(-zm=8I=V=nUyZh2p{A8cs@o-TRJC-7-iY{^eJG0Pk=v=le^${s z3j6g|(^2`C>PaaD9$Fgya?JaD%tNqo{?DVA(*pRi_%9f2TNSvmyOkKq&kktqSueIb zzT+^yIOr(y9mREC`@HB##S#qWDMx#Ry=4cp@BU_pdW_XFIpvtBT|oMF{ZPfaVQbT) zKMO7u>&aADw0Qf|rLn#h*Zcvc*EHG4upFM+DaR}V1!mbTC*9B9NbsCKtAkqdX*Zl_ zrg);O>t<6dn4j&kHFu=$#fWW-Ei3bV87=ROJAQgdzK9NXPGH<K8cnCu@U$RB!insF zJz6T^8456%-^Z$n8aMZ0kpo#OzhWf_LjDpVvZi9U4A+-}ZAbmXZJ*sGh{zfXOqVa3 zv<d~LkVkfZ&%Kj(V%yB>6eO0aT}P7?vC_91yHfSH=LK_oQoQ3M)4F=5j`!~<-JC9g za`77Nt<uvf&`R;}h`6|MdYZK@sAx)$X?2Wt+WLe3V|14WvXcF4cTmVJTdrk56aA3o z`9yYBa>Y~lg^vQ|6Qkam&cM^4|C#_og__EWb!RWGzPYG&>+c2V^QqI_yFoPI_nNkE zHCFva3$e06#-<B?>++6BjNV%uVteR_{PsdI<`mL8K^;}-s<ErR_3S9m(a-#h>Xy9V zVPm`(EAYcqvMzK}`4uCC&}DIUUyi}*QRBV*Cs>AWYG2>nnmYoAXUTerdKfQSJARz< zXy$fDO8aDsZlT+%2%;HHt&3Xno(DXT>;UTzou&D9)#k7zN%Ach)}(XysS&=h@j6Km z!F#oR`};THUcW#}rHp>Ma$ibZl*j{u=ljap?}ZKs<JHab#OPV+WeHXdc^Njlnkk(X zI!fE`lh(h9w#N0_yI(9x$ZRxSRF}_cpPtEX)<oO&tBD^=dTuXYD^p0Nm_sEZH13t8 zJSMR;#8Wtx3Wc`A<U>Liy=6ow(G@OZ9#qL;Wd;5u_GvD(102T${dtd6dg9~dhrKB* z?EdU2XJ~iqj?ACrUbsp~D-(JbWYV9&+&&Gsc<Ddn;y0di{HWW9^lg64OxB50y+4?> z<5akQM(l6C#<{ABbq2@o$%y?Hz$`aB9OYbs+!?dP+dQ(^9_g%ub)n&|td;7g9KKA< zKwH5f*lAr4w@vQ*0Z2Z#yhqz8)}b2R5mJ`w9Vy-Yv>m>M({N<vt>?a8U775x*2>h@ z5HtF{v1GFAroN-z<Do~#^ONEu9+{Ebc<07Lc2rs3b)Wg3ZzLrzh!P`z78(CHWg5Gq zC)ulX#65--WyH#)z8uGR**`NCPW+nI+bvULu-$HzUKr%O`h8m6UR~+nHx%IJt2yP6 zDzm*tJEN@0BV@B2d`Wg7E$awBP^zR|*D>;4v)A<SzVW~+qo=(~Lh|LZn>U0%dq;1d z8maNs&xSbJxd#;>3v7AWD2L50&5p&z?zvjn@x<<XGsk<nI9-)Dx}B@`=S-)6mh6f_ zoAVw-s7#gX`MfhKG|Zp4qLI{OHCl7D4(e|n<d+h$Y7S3X@}QMY5QQF#mLH0`e*M|~ zt4mWWcdIXh-0c6>ByjaQAJm9`*SK)?I&-16^GW%9GT;`8x_IN88&UeUCkD1+p&@21 z4NEM&6pbayI*gDH-cZ6bRl5#HkqhAUGlNc#E;%<CoS_*P)f5xgK<hQ)#$i(J1^F|b zCEDX%yJ>z!?ph2=AYTuLn+ZOSebN~cKqG3_$Kmit72lKvZ6!Cwrv+$aBUz;I^aV@# z;4zSj*p!&ysm;j)vM&06AnVNSoJ?Uoddlke`(%fMOf6UHomjJyo9q6l`CV05m;`i4 zeuJ;sFzg}|9$pamUUEen7{#9K2v0pT8u}TNU979L?!(VUcInwV{L12>95{LLIbxhv z2#!(AKR#672iMJJe73CAobqacyP04vG_hv3BT>F%#ZlQhE2=bThNx{>m~B|bU&!x& zF0xmh!0_x!PyD(?LVOfvqlhW<OiDF^Jt&HKEwzk7skr*0m6qT_p|Q|L%rREQeN=ZR z{eg?WZB{Ah8xfo`E^M9bEHlH@o}g6(8`v2M5rq|5rAdNQJtf61dyv<3aBf}B1-3n^ zI6@9yTkLu3z9N5|t4eEgnO3~rIt@3bo<P+CZRBXEWT2PLFd3U2rWMkHviKI56`jQ6 z#oEDXHvDYnBR!X9tCE?0{A<}>8Ds4;Q?*%ae=Pm~6!y65*AK*s?7St4Ai?QsaVYAo zC>Ql6{-0C9e_pZB_N554|Nll@)Q!%kJh=7fi1AFpN6MB@!E-#}WPd&EIi4f{*H?q8 z2T$!E+M@n}#M`!z%#37RI<Htq;=w$s)_y!L219U^^47KR`q;#1ZirU&^&V@IBM0`J z%$V!VTs@(*{r#;UTI&ARqFJbC2YCk1Ik|7SyrS5^*rgjw!iMlhaMlyyoXW+k7hIFV z=<&o|PC{8pNt2hZhoUxY2hF6}X!mX<FgO{Jv^3It#j;1#^gqmvADS|Lw^@^4@%rY& z_4QN`Xc1*~YvBhRkDbH}%C1^wDwu_G%ILNIPy_>KoNj*n*t*LyV<9<1Z*#6LRXKL7 z)a);v0`uD9`^x>7VW5=%$qw~8CaoDOTrvP6G_D^B;USm!ZwFj^LFGqju}eS2$f#!t ze|djh#o54J%i=+yfQ!f!JUyC_c}PbT#y6gY6)12xm@2{EvMv(v?bhQ@%L@uwOaqG@ z&8(g)I|1acmWQrgVhAZ+@p~JIsMhX>Hj8f_l6huirbl3){-&!-snH^9v&3?KwT6Vu zDD8JWOIX1N31RdB8j8bR6RoDQv4Ycv<qxy=c&HTT?jfdEDs8|1Yz>oW31w_otWz_p z4ct7kDDueHZ!Hb;)yd!Y6@1f|_Y*g1xx0#H311>;47je7L`cl5Ts@KYjRh_Yye;_< zQhwSNQaaRD$8GnSUUn#EtxG2Zq+?$C-xm(IH0vKfR{ER7SJ`E7i0AtMuDX8s6PVoZ z7&6iOldGS(?q6Ze2mZ(YTCYPo*tyQ5)$M`5Eui#54kqMog6Bqse%;m6klL-kH-P-z zu}%+zTyDpfkHF{0ic%hh{>z~9Af_($Y*K&7>#NA~o_9-<SFD8RS&nHxcQ3J*kX3vA zNM{XP*AXUbf!WoWd)q#~Pbz0-RGX|3bc~AGVQ{Y^=|XXJ5TWLl(c3Q+mJZHbl-SB6 z`dd?;3e^e+v$o{KSyOV;I*2tI#Ps&KD>dLMpP-|@;m%O3tuS@w$(5Sk?#rF!PuGN9 za52yEFT5y6zv6o4isr{@)roG7R@j@W+=>v3jyol!EAlS%w`}jmumZX0=@8sD2CbWo zFeIJz4eGQz<-e0fntL#Tv+5(qb3W36Pof4wSJnTB$Ox(g!;=HGV7@YZ^6dcGdm-Y= z_*R_YX{f$lvR&m@X6bDplq{^XlBL$#T{^l*Mk}!}(qcOQ)3GFGW@_Q36T~?1{f<0s z2jy}RJ`Nq>rbZe2qNY#zaPDA)`Nkv16tiqNb|K@Tvt`Fvg&N%%UFe>GqC`Ei{928C zzhBg6?ev<&mvbj~q(|ph$Zp6V?<zSjfyG@rV_(lMu8_lZ@h(0do2D1M8gT+i7CLvD z6@Uc2bFFw=X9sPP<{W%VZ(Gn8KOW|SA(5lUffaS1>WNFfEk`Snlq>@2*&iCW{*L1a zlYZt#V50rltDBqLuO!ZveGsMn>t^*u6*Kc;0wCQCTeiC_*7D+}mYeyy;36#^FW!&i z8gAYc^V(pqRML$L`7XoWHVIgz$Q-13oq<?1BJA;)MhvHhFsgC3nCu!9o_^6!KM=?7 z*1gpfFVyfDrdtK6uiJDgye9`0T&;9(SkXLYFTuj5XJ+QsDL>dnKc_wUNS5AR_f9k# zwD5V+f|3pPZ_gusreTi|+*;@bI2TlEfPA<s?Zp2KRJy3RXKQMGeDgQCI(f|~w(`Bf zimmRCjW870^QH4FUwJr6{#t&F51bC24wGr68MLYR5pJo+{Ny69hU~FUd7WJ4?QS~W z9RDOL^1SdNvTcvVRDN1xVaASHdwAP@ykiiJPRH2iHYW82I2E{<Dz@3dx{v;ZtCT)? z3mE#vVcCH#1>K`TV+Tg=s}L~T%@^s4y{#eO5;B%+FiZ3FkA^2kd{k;<Il>2h^4+y) z6mxC`)BC&ve-`8-y%Id|5r$#u?j*cRP_7jucd)-lp|Fd>)wLo&o374e+Egi5nf-J5 zKWzeA^|`xXknHn~5ke1RUKj)sCDz06f5C~#sVQ=2^e!Bpa%3R8HMP@OUM-%`@U&e6 zN%G~<PG=#~TU{0#E_L9VJjLr^p}ywR^)GBbwjTDvZLysm{cLloso9E{QITRqE%e7B z8u4eidn;f&_4a4$d`Ws<pvGXC`ga)3URLeU%_bCV^M%>%$GxB&9f0B#;Y-}xH`UzI z(n`VFiuZAFa{kcKF5CR3JS|i5mQnsmQ<@XE>@7W>YcX9p*liRH+>RvB&}Mx2&xW?Y zll~*GMEVbuDv@#fq7*O-DG1Wwn%p<cEuXk7j%&mQWl*#P<bs@}dQ1hJ4$IsrmLBjZ z(^Km8`LH~L8RbC`H-(3#obeYvgjlO?*})3Yq6Vm+(Ri)$Xp3|EOx~L8!8_CV{-mgk zhzMsorXaS)x#Tc32e&Q-eS4;>=Oph!F?#%iJ5(ooi(6}tWXy5qP%9dJ;e0H^alHF0 zJ>M>uBv%@&c{f7VjNEnj9j?3~pZ56X)un}^e#xH~|Jg73H_bbTv?h8hwV#zt^bzQ< z_Qxwc`Vq!)`^toArDxqW70KBF<uka$dl&THVc-wlXLY_fP&{BF@zz8!H96KyV$Bin ztu9>@%Ih7n$#nl@FC8NZxg#{q0(|?htAvJW&`ygyzM0Ngw@LsrXbNtQk)0V-f3&V! z74C9oyl#6}JW{H~f?L)eebRZDcS=vb+$a1I^YfzKvMQJELD38Wj$XEBnnw?g-W?wZ z;68Ga{DNX)N>;3rpOd}U*fDm|zwP+{gy+BdnZY);(_#mdBhqF>RX)I&)=Asb9DN<% zWk6l;y|y%3<fgnx18#BV)2j~f(8l0LI_uyme*(+oQ=(7bafdziC|R!j=wz0{s-MJO zx?-!JV#p)olS8fXu@~0dtr+XF(a5B6t8f*g-AzkEM^D7Hz9^S3r?A$!NL|^f+v$$2 z-j|TivO_nar~P(fZMj?}mzg%~c=x`sE?YYvy3C?&hAYO%j__lppuj?Il>Dsv+X;W) z5&O3%JojMhmd%6Y^S|@(3st7U_0aIQv|}q*Hf<MWu6FqxYl+rwrM$_4g_s>Mho>5P zzMncdnw6ht@fc5dnt{Kp{(g{Us;G?jh7v63WizaYRq-A1x3$m_24@(t%c$hemSns8 zS{@-^PTY}yZW~4wyx%=DrGa^Vf;%~lWStzjSLHVxWai<%$0N6W(_CnK9EO3i%U#Z| zk0Mm|C6CO89or=?X*x`Q-N*ObGOw{WFKLCW`h;Y(ERX&cyJds8`L~G^zc-JytSZUh zKX*WsViK9%u#v!>xG;H>8iJ-?bEW+}t%BZu@scn|_KflqckW>isrUJ#Fb`(xjV=X4 zy9zLoq15_3P|?#vC0%sIlGn+|uY0^sSG2;dKPut0#9y6He>`)P_95CP+iQ^g$ZkUF zk!M;(k1z2eWnYmOgSXckiE49ep5cb21Q7kW5!|VHDy84^yK+H@OtqnhrrZ^uyWPPc zKf?)ZyMyk1J2&}%!kN!<y1x!Sov+cFWN$2}Jp0x~R-0ttok2^BBdk!iA;0tpXgl2T zAv|M8N7$PP_ur}na0Rq3!GGNF2<DO7FQm5K*|e@{SLED$tSA;>B485I=<*_#n0(N6 z!s&fIJzieTrO||YyXdLAyR!J|%3{`V#h1h8DX;VCs@sPK-Z~I8N9V!j>w!rd&4=m6 z+l5~)0q75#0HF7c2H}ZA7cHnIqPYWo_x#$iDpcXQR9ksjmoc_iuQLfxK{2Y#7Zgsv z;-9U_e`}dW`4y~zTF%#ZY(_z&xxROYPcyNq=iSW$r5~*FO)os=8+S;ntcCk74LvN5 zBP6$YQVinV%^vzIu^8^k<=%$4GCNbs6~t7foZR8@@M$GAxuZBL&8)3VUSGbiuLj{f zOm+^Uiz#^pRr{jGUnzH6rN2n7q_pmCuM$ipj?BpyHr@8<s_Z6>q|pit!+KnLas$(` zDqsW(9^AwmbkCwIqx%lK;fg$Ycq3od)WiH{Q88MJiR>iN=wYOt8aq=MBsR$_4t5^X zx=)TjaUSE0#e4w8pr!J|YgcmPy|I^0zD<rLKHyq6b+J=j%=sGLTMbv+IcsS>n&AfE zaeckX$aAV2M(Ig?2lHeeV&e1R>8<fcNBskGN4f<0aQ+>^htnUpr5g>e$QI5RVtbmi z*!f=VxPczl9wnDfrr*<Ge;!MU)sXC;d?BA2U#6usIzD#!V!=1r7`o!mp8dZy{rVsH zzKC!8W(jTD8{?0NtLx~srT3OLQ`+qaUW*j4S`TRFRX2Pn&uAWhTzOI`dFqV2$EcFx z#qrS4@HW?z8gI}SbUX9Y<=+?N!ygSM)5#}?aOvmP1{ak`?_(eER*)J~6Fc2;v<5+u zP4iH|>3;F*rU$jR@Aj{pnRYoTp2*d`HGE*3ADvVFfD<%8zwM^2MlBbDM0Ua(N-pxZ zlB8ov3WccszguO3OS@;hdJdynQmRg%@>nv#^N4kUS%#$~r=IsCRBG^AYyg1}T5mf_ zM5t-m*R!VFVFmhHZ2_xu$0WQ39-0;%^bm82S9;epBvwZhg;#W;mOBgmetYn4Ng`Sm z4A!9RdvKvL!@1&Ad3FSU{63f9CJ66xI}~Lf_3N%>5{sl?^7^tQmYr6v;YSFyJSwxa z$U`6s`<-9?JZP51Kum%fQZ&d#H9!1Od?6JdZF0DEYE{Z@v2=t$4KEG%O`%P4(0LZ0 zcL({&A7y$GC6x9_?B5sa2-nBp9sQX)lebS5xWwU}1h4f|w1HmhmcU3*&>XI9du2ZE z$>W*oy%L2;3dL6*Z|A@j6c}a?8Ug#!i>6&}hi#^r2BzJKP@iHUPQ9J0`pWX{x)r-G z(2wj!dnR@NUf8Me#Kz9CKtEF06HqkgY4}D?^47%&_Z+<7k~7@g?KtY$*Opt*V@Grn zlK0#ZTyg^R+ZLdm9?)jKAVHE@a|Vxv54Xc>U%&0T19JA0=X9^xvSzyE1Hzrcoz8!N z!eHaV(i6BLgu)|6?t%o_k9W0_?^ic$KU&ewpf!t3{5yY#W3FKfdT{E;4*mhXX&-Bk zI0|Sr_OTuj>#~#iL$wnltuvL7TOFAp4|pvIC#hX(Z}pFy-q(@R?3j0vUroCA&;bTS zNi)gBdux2Y3)5uueRgwVj;>V_%PR-f%yuMikBM)++(gOU(-)UV;H2HfAL;ozRW5D$ zTT|9pOT-YfsymL68+TmQZiw81e}w7^ZmmG@9Xd=&CE0OhV@Z0$zPWgh&M65al%w9! zR+^-FASJiL5yjLC<R2)k8MCCRBG2~roD>%y_I{|_bPUSoz(?5!;V!_ek2?(PS*qz! zEAc#uK+MCo&%C2s&|JxTp!MBH{hP5XlWxXMe)+>ElhZVp@Uv}?zsN1Sm5dXtJIY=2 zRzYo!Y*h8@(|b2AxWUE*oZYG&sdWDvaX9q(e7rYd#J2q1%*S(vpkN{NsqBk<?qLkW z%j*AO@6F?(-uK7xsc4y$PA8R<CFf`rijY0DkVMJOAffD;>}E!Ub6RkeeM^pg8B2C! z$cbt&mCRr;9Lr!Z7&9_sW`3`sbMCqKocsIU`}uzU{CYe*@WA{1d_DL5rD<Cf4F$%0 zk-nr!qRvsx0$};e6F-0Sj!*j8iFn5g#~AM*qAc8kZLDWj_=}dtpD+A&{jZ9~F^$Yc zV-bu+6dRty9kNH=EuSC!I|4d3+OLk`VH|2K=kaxNAZwzv0Zzrg{5JyQlH+j0bN<&h z^-pcGADf2zG%ATU&^xPxS3a9(k$-xBLBceka$n8k81!fI75k~BRh0yW!Y5K%i8%H6 z!zt(Z;rUNmSLX7!WxX6bX4qi@zokj3dO4=(?-811NJoAQdaC>LaMXZdT+^&&h}kg7 z#0^cf+nIZjqXN~L#O17$D3{TM$>ltnIb|21MX`ozALR&|SNjLnSFm!p3d$61;EDqe za!^WVTKkvW=xTFLhY2s8_kp$R?tA9VTHa}PY$e&_<)I(7&n0--mPr;Dc&S#;6VhO1 zG}rrzbOl*PO$lW#K~^|~d-@dSs=|h{@@77kcob|IT=#kLJ+BN^Cf=q>qP8{-M9l4@ ztL4hadSEWKmwmLw!g8qZc*gE?jzgZRDqPYie>K%rD;n?y%t6caK@EXFMls8~yhA<D zIfw+s;5osELRIBJegDr3TWZ)~yT(Bc`o+vKICgmJ-%FTQmDY-?GI5Mv{i)4MD`0OI z>H>@O@ug}~VvxjFa@2~2MA$6n{aP?@oAtw!$;}u{<^J(!xvQ7u4|FwN^nA-OF`Kr2 z6ys>fF@3^RY{(ou0nb7{SyNkRaBFRHB2hak?{hKpz%mC*G7<E-OYnl9OQ|5_n-IO? zcL`{^&)pZxYI8mhH-R0%!OR9jikJDieK?{Uj?78Qmrm!ne9Jk<X104d7<jwyJH0IV zTjC-8=}U2khCb&sOjsVW^ENIVwk6}I@@AZ^XFRR&_y}#2=gTjiFDtxjx{6y(DWXMm zmnKbT^CY9zELl9qV)4t~-Z(#{i`W_r{t~NsRj-UVO})&bUA~K{a6e0ATin+SLR9mr zMp!7R-#T<y)nry$68Z;3P{7}j!EpDEBCrN~-9G%;-z8zj<6aCTQgQc!*4H0kMpD%t zWyGfGf+{V-1yS~T3+m^)*Dk>iDrAv=gbjXDs~x4=%hDA-Y8V4=E|C#BuOHJD8-npY zb$*<#CrhudoUjZT%7vFTys{JRa;!(r8`6_Lp5aF;1HQ5e%c>m%^-GnjN<G2p6DQ^v zbAcg<vby^)U(<t@SmDAf3XUTNeBE{2+wpSMa*$MbPw9iE<NM}7n39sn!VKj)ohe>t z{=I<tc^9KKCXddmn22hc1l1EfSA|jc;R|U-H`^t>tT*36n#%XeV$$UUIFlcJHA1y> z(u$H!R1?K*@T5b6I(GKQM2EthVR;i$u0`B-!~P|QQ{Z;rqZ#>Ab;oQZ2x|}Uv3?5B z!fT#rZtXJa<eFz-^)_2k&8tB}&E6fKPhirQ-Cr@|CW)|RpZ%U8v#E)tJVu=4;{m^C zRIrz9=m^NaM@X)X;NF2H>tKGxK{RxPdpb<=VM=+|CrT8DOBY752OR7MGr=l~3t+(O z>jI$iA^vZzmU$d5aVzyV=6cF}Ioy5bIDe{;Uu70$)IXA-@6f7Mn3xceom&jbyn`&s zz4{9tpFp51U!tp52j{FFB~of~>NUyuJ6dNxQRU_+XIK>=bVPsCug2>w!rNc#zpQP4 z-9<{Y>kw6q8G7DR>qhX_TUSC5HptR1t6uNEeKJicpGeKD%;d+$ZPX_*8TbVD;C&-} z+XXsdb?JsEktB=h+^`OYkphe@GulNrv=ejpSd_gF+<qz1Yy0)X9p3lL$ZriD#jyPs zRQ61>ZAi%JFVAO~zRiP-w9C_V6D;H<6*GBLH?(@q@Mn5{&BYFMO=c2(Li0&kVMX)~ z)WoYNtRK0PY_Z-U>uapGY?f=GhwqgTx`I>3Yf0Z69(+o8zIvN0!x>d)6l^t3xps+t z#D<+jGcCp3@MLbgLX$DzA)c_@f1%~b)8s%ONg2f9-tTl+IJ<2n>xGb)F=)Pc3EE`8 zt}P2};GUYn-Ah|<V+(Fo^4qsQO9h%Rp44<lT>VIa<`bq@?q{_vvT{}vfSZEUxIzs& zgs7)7V!D2K$B*?&?HiXsFl*DBp2U>z_w{(``*DzSg0*ICct>V>?vT2cgjcVagI08L z{f7z;omtbp?R^C^^G1LH*~MLZ#mUSXkN#+#hLLUNarYjd_WQgX<c_uv=5QO$J`w2? zRs1e$%~N13;QG|1naO-fe~wAf6XbGFL^yXSLdjt0oV&{VJD3~c<hJVytkFbd)rkc9 zoVzxsRd<f4mA5U7a6YcKH>nZXBQvBAN36@m-9i;>d@N|yOpAG%)bmskZRotVDR22j zTGG$WO-Wq)$UE4VPKAhf=*ZjCJjfShVwuP^F`$%|J}^9hGW!joOIm91i@~HnVU^hv zbLXfkM;NkX?dgf!=<l47a!dXp>Q^`~Zn`c*3b3oR2QhxrO{<KNj_O}(JCXwURX1Xr zBc%MemJjM+i}35KfvFK}W0S45`Jf-{BN&|cx~xpY4z-^#QD^jLU6rt|t1cm8YR?KQ zT+EUw>4INmX%0~?TF-ccaEGp&%UDv7?I`&@08;hCoTZie4v_C$EBl3bgcEo{exItD zw^ORS*GI~tb$5kik1aOp2jHetK49h(nd@S3kRJh!S-Dr$zCpX%%VA<thxeoHC2V*V zRKI8ZBAKNEm>e<rcxL932^1%t6KB-i>=l|jQN70uCmcG^Oq4M_{;9SUwUTCB=!Nb{ zYJ_=DUrMF2MTc%;t{P%K27g3e@-BE5eEwrSeOto)?xy+BAnv4^{XB`58)Gw_YD~<g z84gcQsXfs}!P*&AS!j=e_{#8{(ae`5g4s~+981rwj6h#1lx;qR!F(W%_#MThr@c3t zwnxs#PdAi_gKn}jxit6B_CFvFH8+7gPuj;jphe=b@Sx<@1&|5J%p;??^JwhZ<EEw) zWY%MOTf~?<)_;MGT7CapmaMt@tD&Y{rYnNv9=61@JO5a@mRY6^u7+L4MJOnlafP{S z++K<V^0yHpmj33AwoQruoyMR;dGkA)0U<LlW*@LMy<G9NQ3PLr$MjC>STode>R+ci z(Z82v)PlT8P&PlYndn5Y_V&VQ0k<-}66T4lxQI6X>goHZ-*RV%cAFDoJj>7lkA`9P z^N5*)#;Vb>)Kw*GK80fI<+HxK!l=0*zxjQVp*?J=kI?{XmFwx$=g{dd5jMrabqkWS zsnAmiT4$_%OOU(==mdGeaND8VpGe=@%rF^zSx)r&rM@}qqfsyJ`2e-1j2JwB-|m;% zii=r9qe=D%E|jEcmX-&j=eA^EVO9h!l8ueLT7KpJaxgLTcz{~e`MjZHrWcveU7I$1 zVVd%qdX?ojljm#7CVuASjBg_9lk-2-`yp&Ls8PQ;?9Ji>b{hn_$K`d;GCTIQAFSOh zC3dZN(e+1T)RLZ&qmDg32^Y&Ft<0nOTjXXJ0tENbo#*jfdu<)dh3y#HnV-O9@YjC0 z(P>sy+Qu`IcoW9L1V3%&Pr@pZf(T7}IUb{f`^Irqn>$){ht;_G(W=5q7H8Rb&wdSE z{^liJ`MSKATkE5`2T@tqv*dqzEpN&|UrCsR8pd=z4ou5o@v}QkCLqJr**Ey{c`)3D znVoy}HZE8ezBsUAtG)7xnt;@>jv2j)5zp@%(8BsV-}m?dfiYbMsR){BN8qfghuwyX zyQV$a-1Mp|emL<CCv^4so?FeC%U=~-`VZ2{pM^^$Hn$HG5|GmF-XFg+b;L|XrCs%2 zDIPi(&)q^(*<;~H*sQZkxx|@^$Zf#P@$HKBhO5>0h5E*3=IKmXsmlI>M_NQ{=*vlV z91k;bkZHnJ-5+P#=uT`v+pGJRNe0XX$c7OIU_7DwkAt5%mU=N(r&1q(*1AAZ_jZCM zB{5jOobO@@bgqcza(#n~ySMXmH7&Jk)$r%b2scgtW+VGy5U+RHZfg1MOAE&lqP<;l z^e&6zdP=&Wb{Wh&iA)_T?g5J-KShMG(l&=Tdl@7!)GEOC!hcb2E{AsRn2|*^Uo#S9 zwa+a3fq{v`FSvPOFVHqOoH5r<{T_dQzW#fcqhfHU#@l2qS0(3WLzdVx{Nq=j$sip8 z%7a&NXXs<LMJhzn-mWwJj3nbF^TGpcVh{q1P#J1V$aeS?4exe_z8Z283mJ_r!8jB+ zBkV7C^pidodDHsp+f$2vKvyxuYRV|YJEdK)e0zLmPjaf{Ld^wxhI@bh+fdDEoGVxp z0XhU+^-o)<NO4Zfc>E&y;r<q8nyIjv>p)D+$Gt7njv$g$87fa2V1D=cscMD_2OX0Y zz2yfRoUA8O{91n&T-7ziEQz&N$oU@g)RgfHcc}R3U0w(w<?7tE4u=>r=JC(KJkVmI zH?_q73AryW$$qd14tc@lzIW9Q=1iW))C2MG<`KyfpDo-X<dY=!ekN@aIe!&IGUlQw zJk3J?pbSgGU$j?M?`A}9?Y0>jZ(b35|M%yh*A`v3F%kwwaJkU}panp1PY)K9*Dg(> zjEn4G#}d~!C4TkEz-EGVpKPJ?xsRa%%0O>lLE-d@R#(+HPP2UM!3%-{2qq?+tz%Mv zkPO>;KbDLx06HJ2D?cflVTEb-3Cz`-WfKRl^^(33$HG_^<h3KJ)3}Ic*nIk&ndGe9 zN(H6dPZN8|-IJ-t2~OlM19lI#X$O;k{Ygn0xpsEzSwK0^iSXl(hvN9idiv9*mK*<s ztP|1&aal=br%$ue7sx#PwutlD;UU-i3wDA2PyN$^o2oOw;$ajkE1Gx2N4DTc6>val zFh}3n#jjMsngT~rT07U5GVd@ZW)ZJQ=|rx9<-89*QXcYf<R-Q4`%cy!NO-z|FVj6H zq@T2ot-ayh_crm~cv^q~{v|6Tn355&hW9b7`_+ABfBEl*z(e9r>QYK)yhPuL$gnFc z1isBo_CW?e{S!3wDcE!cA8Wvu=;-6p0E_(Ki6mnKd-zQ$#rdT#pNqMd-a9pC65L+N z3Qs<bbmr`I{<x}Tj>3)hkq+BvFvVEa7S{Z&#L;}?J>5Y}B6_Uay-hMM<o#|K{?W?n zs4oQB-?nXDc*_^CT+X;>n-VxHO7F$yaM)3c2j_*MC+jE^ztpxS84B;X(rXWuxKe#a z9<A0S)E-QB+Pe$vf|y(!>maVA@>^Rp5v=8s2RiF3o*(8@@d3GUFR)T3n{+^7|NM(> zf+B--&~m=cW4SZ(g2K52$w?OCu&quQS_a$drj!duZhFOFt`5a(KJdWib*`;x?Gv!H zdLuaFnEoQ`cp7u}noi1XiI%0>4@Rr6@@GVCh;4e;9Xj-`LstV+^o93c^usk>Ju&{A zwD*=T@bu!IujgenQHO532cBhJs7M0d>$;+rb&|q3F`QD$lp<|7!H>EfKSkUD$(fGE zYVBX-n~u<1%830eriQkBE}Yg;?LXTji_|paWu)zW?x7m{bwTw7Y`}(Z`UgU<Dj(kp z**UspC3@2u*d3Zb9U*mH{{#8^KGyVbE%&#{!}soLdIM>_u74VA%_&QyOfZ-_@(~qZ zL!x>Esn$Wb;8u({7;^wNjc&JZ?{e+-8f3!?a5j)V!60cMmjl{Ywzk^J;&Ifuhz;zG zS>5#B8Oq=Nf(cpKI(Jw)Fk}(F!{`sKNp%zxoNanS)sf`wD~)!uCjX{vj$US70*#Zq z62kSX{jT`oOsR4pmBkhneAcw?qDdq##W@>7I@_`h<d1D+vw}Zd-*gHk<7`kZuQpsp zi}rT%2=e+2lQ;D(!3?-?+dm{W<j@PT6V6~I`e>1O^R?+gwvsh_*W#EMV+$IsbX*j1 zMNKWb*}l2yHu$Aw=ShCq>>q8cDp7AhbI(7*6~t+2mNYG2`!3IHmPZFp*O4CA2hXl= z|G2?KHe~3OA8}aYN7^(Z*5EAI^O1I3I=T2o%x3lA9OB@Ynr9NXU-vLvtEZkdNwR;9 z_I_T!5`tAYZmNLTe@--V4`y8(lru`hU;%So5nn)H8V{$>_E#0FfZazkpkN*0N!h!% z3G=1?;n9tyA&V<149L%$k=M5$6^)DrQ#=py6VJ!g>IEM+2<|%u6}ag5%Yc>vs>ia> zD<$df4(6+e$si4KTwoGN@>2r60fSLcJ5bL(+H`x_<q8@*aZn;B+RSEixtn*xZU%M$ zJvFe(>u<!`8-8kRUQwwZOJ#F}(`DpzuJ60}R&DQrR!d9nnR>{!XLaA)`^D)+!k#C( zmvoQrKKetX{eG{5v9iDHheBWGoNsxz=fvjNZ}&~zd-&maqHx=$!rZstMMfdM)C824 zaZB>udSS&y%;93-mCJ_Ef@iGEs%QGm=%vQmy{RWocj%;`#U&2y>?fy_(#WgX&5GqD z=%X4~Rk|l@Qu(<a8Brsy?iHj}-L6BW8BzzY(wn>(c9Y17B`G#lNc{0dk-Y-l+oW{j z$K9Yiu1y9R3AI??9z|NTct1=DOtlhPt;X{fU}>Sf3)!M;tVtP^4TqvUzNDZRUW&AK zA(zH|ztUAfkpjye?(U8hV9SXg?8H+$4}WW{LhD5?RZI4}KSO_~bvDAD96I4$%o~id zg}LaL_Zs778%1u6H>)z$D0wa?O8c)!YJ~;{fmwYF{?HBmh52~$k)!v1uD<cAzZi>l zklFJY`du$g;jH3W8LO?C)>WIC?5>uNCTt<$U6KBINThq#ZpM(e;)Gd~sqWRg^R6-< zo_ppkldR<doN*H-1tqT|h5{@ox)Ljd;2@1e=w)Y1zdIpnHk8p1HAcI5)vxdBIW`cO zlc!?)1H5Q`{#WPJv00UVUPhe8>yxTP)%&4vWt{Kvf>aV|fS`b_V!^D(iWFD8@9b|? zs8vc5?zeZHvOHB3ci2a3`;K^N3sZd`1-$z$2;uvkD~+3xKXm@u(|Jo@CN#;@cQv_} zEXHxw_i^i^^)YI^Vl;(86ln<0h;twx*&MMdnjw1ox9LfQ!{7fkrdzcDE5cC8fw^XK zjLiV5u0QxdD9$<a)8mqamt8OpLQp33nNA)w#7sXg{KAAuoY(O?pP_<gKNcX#7>VlF zHyH4(xcAxTO_=KYwz~<IoqT9In!}p4hGl2ZSGn^Plh1w*V5Z@E3Z?Rz>$611-K$eR zdAnoG$)zbGRCe*0`5e`<SM{T|x0}`0;s#n%GgZzZp~LB&&&{Z*20XR94f<0xGq=i5 zg3!E7dJ@?YHBifvwQ<Tu4NFvEGDPF4tofLWD2-)tD6iT@q?FOfZCD)Q?e=}L3$e1A zN-pzcSg>KFf$A>$-8*8hxZS<@A#y#dbEcT!ox#)NB=#%+J-RtX2gCf{zUGvLk+&-@ zo~=3<k3LjJq`RceI&})|sDXW}k;@Ze8)bBzCVsm&-Q{<*d+3+bY#+4F)+$C^)fj3L z?j-IaQiH0Dm}gv~1Z>bdGC!fBIC<SJ$<!S?t|@lFC>3CL*?2!W*<642V59&|f{`(? zVtH%t0V|n{X&4s!3e(<3truyM@4PiU&yw^3TI%Q0;(4mB2dc5l>Oo$iV-7Nn;BMJd z$YFMIJj9=QDCUw0!KKYM4~ot-mQVEs3?r^d#2#t8?xc7zM)ke~%+;l(xqg=(JM=v> zrt1+gMXFm*VrE5+<41Aj1?Y8TcaCYif@K;zjlEz_sG$zzEnB&;ZMANrGAf)Zl6Avd zRu4<-$J-i7wZ5CZatmVq!=D*4YHEh!@eJ`~H4=4z_vwCKoP(UC0uje?B8i1afUHxZ zOAO#j)6Eh67Oq=U0vm!PlcgG1s-)YJPTW~6Vl~o_0^KpWBn8I|YS(mC5QD^rgQ=xB zFkfmn3_51fS*9X;n{00CpT$Q6W0t`RpbVprli$1N-l?)c*?;5~J9U&Sbbk;EpZ0ql zEfV9_Wl2!k;^j1&QONF@!+kElj_rExfd7nSPa<nJyZ5_X!Im<jnsKa7y$3~v`qTTw z8o_j$Yvh6a%;;)Oz1BLRg<Y6o70<|>9WaCjky#_Q)0bI_IFy!BUc&i~)y!tvuMF`x zeF56c+OfVkhoFqZE*ZSiiloWl`p~yrHK&eSXCeh2&~^0Vr0Y|4!=Fuyyn&Ql(0Vw% zuFxJ!iM}irWmoc9qwDGy-O&sB3)hmJyj8Ytf>{0tn&c3TEe?Vg^WvxhE^$BQr>#8r z^WXQ`Tj(!%B|D+vPl7XM$2wx=fyVoC3X(|eE{q!)dcmC}uEvq=t)y;T{R2b&&<{6{ z10fcfEsV1h^gG#dLz_qw7F_*QhKJH_p35h@c3rYWT-&AU$GLxP$HINI>l{qVx;2Z% zW3Z9ii@3b!&1_GF!$e;<ECQ}=<5<dPHYWl#Yp37Z42t@tw?U7hyCOOiFi4KpkT8uI z6(dmm(JiI)?49Qc%Kq#gwQdfBP13C4$<pg$BL#4#wE>;qdE1eDEuMC8SAwU~^^4JJ zmm8W{W9qewYGi~s+3S#cF`gT|in%((oTu3`h#b!XClmHZh9ZMKheD>{qG!?YsZh%c zQaU4{d-h5jl;y>Z+^$sE>JU^JMo!T=n(3vr&ogqW(nwAcUP#<U;WHP3dygy@(s*V& zq(v7DfX~`g8T8KHa(d8eoBvg6J^uDTG7P%*S#KHv?(S^0-6!fp0rx}h?G4MxR@a`f z)Dt<wzvL_2FDY}%5Ec>YOxe;?;SOQey|q3uFxyKuh_}`K{{xXTzzZ%XPTKIgAESSK zJK^M_zmTuJiU{%aJ>(md;)n$jSawYMT$fmgQ|3~bBKs4q*^pr)VaS9h)dm=H{BNTW z?)&ph-*P{hzHR>0*ff_P&tYJmI#s@H_IjV);!yL|h;HGHNn#s_liqJIcjP1$^}>1M z$C^=SBQHHUdBxczTy!Vnp+J6$6S}Hk(R!@LWqa3z`)Vw>4{Crt<X7sjTzJ#+3>#x$ z?EJJmP`qj_EjGgTcIiG(<_t~RKrZ_adqo%43R!FwMlKwJ1yAR$$c~1P*~5>Uy1QEA zNgB6BP8c(uN5h>yW8=H=sU(&~pP{Gr>0b?>-a-P3HBTk+sK%N*C(QPsU*AJ59?KeX ze>SQ25m@>UWiXv|&!`g5X|xR~$up)u$~?JZqJru<#K(&+Gnb4omSSItJ;F}O!GFL0 zjRE*!{nPje+oqUi$_BTqn!ejW6Zkqgqt3np{-<FVb4x^h2_S#yM|-Y3TI1{MdDAjk zWbk@~1HjK63-sY`Wj#X8GO=_Hr`+M8C)3#nGH$@=Vz}qJ$U_fx!#$02V+@$8Nq+O6 zsRj4;9k9yct8sOMc*En;+tWekyY##2&RhMPo8{4PxXEZ+yX#P}OlPft;eL}ImIZ5b zgymxmo|@v@Uk`MN@|qWDNDs6T8Tt*AgrWtY*s!#v4sCAsfdLE?E=h7%yzOP`eiX}H zQb<Y?Wx6wZfU=$NNOSV;1YEG(NIP-ce!AkeHhkG?;~V}D>z{?UQSJ<c8?WCY_+-C* zFm(MZ*%skyKMnq;R!6F}2fQB1BsVt^^SmRsv**YwcC=x4Q{?jHW=f>LJg-xDwpvu7 zwovJrJt`;#hZAB~v+kx)Cw%#OTQYyOF>PT?lJ{kpR#qH0qGhIz(=x7hvwHpD82o33 z)vI@Wj6D7mMqc)BhVEc$M8`Q$KopZ1Dfr;XJ9^nVInt|X>ew;65Kq>%AQ2xc#eJ}R zayH}S>AuvVa&lhhgmV<<p+J&S$8d6f9AY|v%ohNscgV@9s*P{>KdgTOaua6QpSe}& z<(rk*<$aawU+Y=jfFppITtOeoB!ZWbhB;*zWkh}xw5~)HiteF(2o^c$(dE_WZ92K6 z(1z3v$E5=KsN01;)a<2S{yKD8thHKgcR$T2_#|V<C5y;tjo5~FbN0qC2opW|#`GV2 z#MarM1p7?7{>MK|DDIS{+>1B|JRG{iAE<7VR*}a)Q*3};{PI%ty$m&=zpR8fZ{CL{ z`=1I-^2C0z$?9|OseK)$r09qO_a#N)1}A|?^m%*ObP(35jy?ibB~}tpBq6~q>~S@| z-}l(%IJ$)F+*k=~v~k2DNfje@HJmr(FXOPZW|6HWDLdHIS1={pNo5NEWKJZt8kn3b zFxD9cnCTbXZ=3|%D(B+O5AHhW%4)?4qrS9ZmxvTjG1zmzO)K8dx=EkF*KGe#zRvs3 zkyUC%CUMk3vCsuj5sX?Z$t{gG%WwEB6D$*&oTXIj#8bm2?ttFfv1H5FN%pTqI>BK@ zW{M|7z=g)!l0@=wTo!><-)&-2UM;RywLP)lo3ERWN!OLAGX0(QK*Aymc+9udZd(GX z_Ga9jQwRHw5o!PZqtrSI1}=1UTku{~42Q+>&F10m3=Il;pZ>Y?|8lHJ$x!C^XuZH| zBF*2uDKAO!jDo7})_hgGS>v=yQ082_18~R>aOk|%Y0S0^)AYR!xDk7+Q7lxPaOwe? zGuV+Cxik~d?{(b7(uK56v-ixJ4Vrzm_SwAQa|F4Aft}iY@0<D3kC4>lb`E2P_T@jE zSNLLK{spZYtz<ZY)2461>jp6zJyiA`KUL+)M~sSdh|O2I3lg94t5R~m(<PD0Ne9ux z@8@ZOaWbUe9ohG%*evEoB>K{sQ|>NlIWEI@IFHv(HgAE)Q~JDwlwV&RtW~0t<3J{N z=R&TCK&MuF30GK<<(t8Woo14yxA8iP+IEO#s;X{yzQ7jtZ7kroTO>jWO3(a-_o<d7 z^F6282dh}x`=)Sy>n2POvAZ-0*RRa?d2h;&eMs9|mh*2wY)xbA%;Htx<ayCq4QuaF z+-X<j99Fx}U9WQP0E+Q#5|%kFIh=wE+9|SEEEj73^p+)|Q`tb%2lv8(g9~-I<~zGV zlp1@sy)bjL?_I$!Zj-IHD{&$xJ-YTM6NDo4&<E$!s1r*HD;M>Btl+ctyEj7;C;Q?? zY+GBjHiD*d-eRo%(9XBHA|u5q+u%cb<^|<SHV)Q`Lc1}y>b%R6aJaz(QmxKF;i+z8 z)V}1pXap`epm~MR9ls5}gLAJuc(~=COn;N8%;0hcYQ-)@P)g^RZ-e%Id5IR4frZ2q z(&w)H&RU7JWMl}fFWG1hAO~@s9AdSv!|l0fZN{u$%aHp6Dusop!IX(WO3Gna8gvg# zCD0PF+%@6M==WAU653}ipH$@;qe;|(w0yD!VsqhixiJ#PcewkYJGo1!poxBJSR-*W z5zk<IgxJYIEFCdOvdG~kuZ42fR2mL9Q$=>U@}<1`t1f`km%+d}zpxL~ln+f(FXn{w zkMLvoRT;s(yUQ(5#|(UTZ1}l82$}84qb;(L0`M#w^Do8j_Ej>IOR0!6Y|VCavdGA6 zB5^jRUV|h?b<Y|=dBRZGAWNxM^@6yO<7Qvn_O(^I)Z8sPKCG>5c5~b0_Z`FQr<bOv zbYxNwZ)60Lc5h|A!_Nb0f9Kp;L99Q^5dK~g%$Ci1?pW<dL<-!SA6G@?o`1dRpFEN> zs{Puit2~zBO~a<Xe0?o9&O!au%Giqb^6RB6?g4Twk#@$#p{hBJ71J=GC)GNTIZJJ* zaf$Eb3>OViYk6w{+L+oBbliyeO?{u|CqtDUIa(=_&MLtjczuBrrzfTV0;;uCYq0p8 zt|hZ7M5%$-Wq^ErIj0y*THOHz$kbIi_zFIUt3Uj=;2zN8IfZ1ug6cUp!j&WL$ts?* zP~7H{hD@eeDi&B&u^CCikpjk?H3Ii}J=x|}4E6Jvlyu;>KTCtLL{_!r@M!2exVy^M zLykw|>1j@Qzdexhvv&FmdEpNbODrPrw&HQgPP9DQMfa^etj3$DEVm7)`WkN>9)*N3 z0^+5UO!W7hlYA&ZRB|oDnGEbNaf%~!hDFC1Ze^YHN%1Um0t|$f)cWiagU55`H1<6r zg>xVi*?$Fy`<NB1Bz$@sUyGN2<MO>KR+VWB77ZJkN~Na>0lHG{f5#K)zPvRm-z`yR zB>i1Qz+dU<9wkt#{CFfzwA~%!uNphF!o_%#-QnVj1w`mC6aI{dM;%BcDzE8P^)8z# zo~ERlgl>3%iB#+LqJ`e$QO;c+^v~E#nxv$ZPUpoK%+NuPhbj8q-WxKL`1TJtG*@1> zG>rc(E+V{i!|het0!vc_Y(^Wjc|*pu>a<%7Gj#C$3geAE=T}%G!JmQZPT7`knbkwf zc^x&x(_Y8T2>GwLiH$p)_!oXR9A}_`2PB<zrXw@tx`UnXtlfz?pRj-OCUG`8HPEyK zZ6r@m{$c7&LiJGPlF3vFtB6=(-dD~lio4nMqXmfasyJ~R&-X8GMfNoiA+_Of!(_*% zY#+;RE)n|ewKEaG!|7CZ_Ojp%W*IFc<7v^G8)*w4koH7wKpGkBoE%!Q$+BRZbc}td zS8)lr#>C_1DXcZ9^Q0PrO>xq0vg#<pN{T|ockUp#628nZtGYA1n#2br<o+z$K@SBg z_WdjIZN5jNfaUnU{)sAc@bPA9yc6v&dee4TNa1OJaF>^rR7v`3nR9i@-u<1PMPxHb zv1FJG1zIYu&<4h;(&nOGBcf-wG2Um7WoB7uueg3&GL}GnV%1ScwvVKPQB0<G&MY!Z zs`pil>(<4nk{5R3wHmJ0=4llzcQa%4CKul^8pxC<v3EW3pE={5&>b5_a`w$5Q2_eE z|1sz(j@8gx;!ut162EpAL!4((jcB@z;`8d~qV0#+gF?AaBk|URCqeaDdR0>s{(e>c z6Et6p6uq%ztl3=uw5y{S{Qa3O_?<_3Y?W%Xtv0ckY&3)~d|=&O@+Zale}P?Slv#3N zXFo(HnJ6oxn4Tb2P^s&y^n<M+l*%Y%@D$HkxV&&HAq%06rk$=SK2c%TSK~P~)x3U} z&Ou@Y1K!wh-a*^MD*KhK_gn4?8R5liJV>HZ`^b|DEA?9WNDI=iTEP1Xh@H%j7Ho5l zKgF3ro>7k|cUHNiP%yygPeBKC)9ZNOO>@sp!&K8rH?opQuOt7g%>KvnK~2@kt+1L> zcwrtR_{qApG@A=oZ#Zqip;?~uI03isc<rmihRstxTn&>|tW&PR*4qs&&TbRQZ(gdC zbdYI0KlPndYj-yNnCs<J(XXGCrW}lBmC}m*V7n3aK~2Kqu^O@JHafqS@q}h}KmA7T zm;V)d5QUVU3O=u#Gucz@y|pq**StR<=vh^for+i@+bSF5(Nnqk0S5#?$si@~??<-S zDgxj7Hj(25)2W{K``{iaAe^Bsdh3*pV{`hudj)o6A(!15E9=k_f<mWk3m}eIPWIUA zl>a3JXM-eUA+o@uQwW_<3bF9&=&LDloTu@%OK8CHS)x_d7=x(3?>KI5MluI!uU9qi zH#r-^aTowL@&v`WdA`;!^yF1bl9f68gjA!@<kPNYqIbYl;CPURvw6pw+-}5HSMxua zT@BrE%SZh5P2YCKpVL7irz`^W6aGxZb_*%|Y}+mtGJo-b-@maMsZrrUXvB<@{HwIR zzs__Xwv~xXAl-Wgf(+5!X?vOP&)W!Yt1=>+hb@`-$ki6}5ECU2oe<9YWn?^H1U;$t zZVlSxK&D2F9tB>bz6=%}5}B@Pi9w;hst5k;8vNhEs2kp2w)t+q%iB<K51GS>9OGRU zvd&R$c>=Xdj5@v6s%{Or=z*kqyH$i)a)k{c)el|8YZyvr_&kjpc}|o{s>i~C21Dk9 zq!0ZZHcN{BW!(3FNfQ$bv-6N&JWA|vCv=JjV+y4zb2~jV_YM}Nuvs0`A7oPCldUH@ z&eqBhr}0fHXAe0wf2Zo|ZwtCxNSe3jX&<z58BnryEwi*rH@RBdS%RI->BLLA5mPAl zsJ#QS{|k`5?m#1?(K?X%<GSJ0)a<<M1@oxnsDbMAy{}Rm#NAU!=~Z*PtaM&+gY_Ic zgl3F(VJG}2?!8CaNL3HEd%)9l!%Hm`T+@)G!P>&zbFLD!Z2ycP34Y!%<bO)!xRI~! z=xACN!X6d@8NAMSr}T{UMoU3`N%U2bD6_Kb^PD9J>Gej-yM?BAl_ojXWS@3SCG}dP zc_&8+?rH2Qafx6C{R0yiW#uH|DiS_q3nNg`@ERQT+^ejhrsx5`$)I$Q!J8|jA1C=p zMG9Old~K0)Fj1+h>j&u{bKh3f9*TH+`rCJ_puz@1x4ykW9DW4Occdw>eU?c@G{0un zOfh*0?D1InO_LUcS8?I&llNVnbmt+_TaDw#2ZNh;@^e^!;I$o=ljicFNJ)nL2lUPL z@~WoxK^eroD_wK%Ed-P7zpTaXlAuC+7)0+#A>#HOuww#A)X62TOd^|5LtHFGl#CHn zV%t6-Cl@6ECy)`KXi$roS#1%$|E=nKLy$P%v-j5<<9(#3t(TI28eg02gNqA5Jbr58 z`hmpwxB~cD9y)mZfR)O{G+aj(8@uARAZ^fHN@?U5E9%Nb-M)iTdZ|A7uiRCS+)?sV zC-}Zs76#^u+aNlQn@}otV4k_{Wx=#FKVCu3+k9`Rk}DKdDk_ds(hWDok_VGOqGi~V zp3SA!O{n$fb&hF%E>pyhft2jLo*X5-;U}?NuAc`-rhf^n%(oJ26X7eU(1Wi+j(`5D zp(MaX!8I3P{Jh=;Ur`Wj9uV|;{<tLVW!D7QPP8jTDKK7|vfM>%eomBx1RRuV_3JX} zF{yKNYfLnC@l?;{9ht?qTfmOJwP3nJJNwsWM6jG@8^v9_Q?jdphoGH<o#V{8VII{r zfx65fc`#FmonRW{T9(AQ##uZ24^Tl#_Na_gV0X^t*i1n|eu;FIMMc{;)4SFC1<w6P zFeltPh!#6)AU7k4oKK?<Y#G`8!&el0I~No&QU|2qL*^r1GyY;u46_GytS8WS)mqwJ zo)XZpT`B06ITt!Eg|<7g`;$=VE>8<o&Fgu;c2R;Dw5Rl&z3IW+ab*f~+4a&X_jcTD zjx#ub{{~Y~?#W43V!QnaV?rB;fFvH1+&U)#ReF7OJwV)6^%o;>Xabam4o)r!BNurC z#cj`|Rb%ZjYh5U&%F>_Vrc87OE1BBWh$`5d*&>|N3Q6~#tOk?i&L+*VTz;HwAU$(z zkxNb<Z7X!F?*>qVx&^o70YjC8_-^M{sBiZvY{?87ej*;*z2?=<RE%9QY3v@%1RUaJ z3^?S!gqX6_h36DwtThbc^H+O85@bzfQfXU@_+@<`JH^ju_eB+(*lFK4mw*l=h`#*} zI%PBz<5Ysj?J<`rI7dL<Az;qHvx>;U3?H85xLS1QhXKpuWs=Ym6Bw_m<MmJO4+U^Q zLvP6og56G5)O#ok&Zj%A=@uF=);6MB=(4P&uUL|JnDFrRJmL+1rwBz!jL^v-|P z8g<v@D$O6=b4$#}$zz$uD>IAI1`-qt_q>6oVaT~H8){DU^-Nk-gIQRy$QNpfv*^-y z&}utZ^_Vkkbh!mp@noI(g9aq&L~B)|YJ_7{PgKmCWKruw)G<72rAPkNoEX&2zf5GW z#(i7*m+H;Is)}RT2pqi5yGGWgqddg#K28JQUX$yMyz~{*P$!7_;s1y?lG`W7h+35v z&yE=^sWq?GTgYRVQ@r203xz4_tmFDKwA*j8B+NMyBBS5ymbR(20z&kw89S(GOC)ol zIc5todZ8JnE=`9NT<tZg$%C;KaSr`in&otpbKa$K4ptplKd~hS!C7#v(oW`A3w9EM zaF0r}DCT7^*dwEd&J^MckZj6w1?3(p!~9_U&XdyTQc9j|vdsKXv`4}%HIdmm=uV^` zTFW^8HcrK!{fUSchsg`2YXTJ=JY-_pJx{RGDN22SX5mcCD4#6`ul2_`by`vK0*{*n zyv@d!I3;-I2M)zKQ0c<rB%tnsB+O2isiP`ARh8W*`qWCDhXPWi2;GB}bSs59N&H%q zUPpr>yK}-8#M19>I}5cejkmoR(#!s;m+21s!3FpKNG#wn$9;161)`3{S*2f53;mCS zoyV&vt<nZeP8%Lk<#4$COy(}G&^@0su{)H^(y|Xb;GkjEE{8Py)MM_=`c>U^pKPEG zq#`QBQmg0Vh`V-7QjAk_y`IK7aLSa@>~R%g4p!{G)H4;%+RG_AZ<;w=YUBF}M2Bv; z6KP20e*h?0?B61^eMgylRBB!n{p8wB#geCyRVN3{bQ(|S6gt(89h50g2H}aOGIi9i zD`hHAfDzRMssOVz>hlpNp1Pfaw57R;&y-k0_o}I9`{ox6(NfwgBr)~+X;QDM;=U(y z+Z^ak?~zfRTJUw3QNc=dOS#7C={qYX1z&R_8sJ{Kzp-vIV7)iw^cIlF=z*fmMv8SU zqFr`184>)6%ETIZ(W`?E(Yii<-JciHADp@dRgN4ycN>{++Dv3MotSeYye`8x-p_<z z`0wcU6&>*u*Y5qvxOa#6eJKAGz@q`)0bo@x#NdL?&2`|%SA%BV1K+(7*_(dVyd84N zSW9R?bD!JS{EINSt^Hpz0xIt>60Y!X0_hBkI~;_B?pPxHcVfc7e<yj9l+M!HzP;n6 zX8-L8zCwLM`mGJ9Wc?Rf5!?m{aoGA$;2qsmKd#pNe~9?}xWnG1zy*5u{$SX|yJ(%p zXx-m^xYko(Smd8_L?f5#LA@EgH%t`N!tr>b7Nf#dr?uPm+AwI{;6EdiKK||*j-O(t zfYMae4xfGh73)dA>)p3bxP^2JSkOddI6_eHf$H1;3asBt^ih=UJ8qr$KN%RyB(9Af z48iF24x)vSLl!evnka{=T=>Wx5qKzI5MK<E=P9S2kAgbH|Jmyz9O)QNK)ZDDuV`-B z@q264^A#+QLtgs?)=i^BISit_`r`Ner(HV3q6$ZTw~;$tuhjHG{>5h&3$cK%-(3p- z{%c#zKT?eR<9#hetea1MiQ!+oz;$@muNi4c{{x!e^UMEn5E*wE9Ov{ger*|$=N|=; z|2S*1e$7_rPjQD0e3ky|u;KrqAnE_1;C~whL^=g~=Fb&ZaIMs-|0q?zemha}xBM1V z>q({U@_*DhFYE6yyvOedU0m_t7diQ111yNg<%ktQAn8Z+R$l%&4LwE!&G1019InYp zqQI3ad$t+GSFtDE?cclaf&KD($#Gr6@TJ|gToQgZ64Vf=ByVJ}_iTca+$M1K0eB{s z^(KqF6C|pL_B_d~{zDttsYTEpzag|q_N-2oSwU6(_;iwrc76do><{r@UAU^h2fny4 z+J;%Q&|1rt9NKvnJO?^9*%zYB<p$eshLj6N&0o}JjAYRSQ}w&uBk!%pmO!hJ6K384 z=vxMD!M%)F@=kc1v}!S$)6>Kqx%fv7gs-6Z4R+x1!|Z6uf%V5*a3_RkpJJ96M0<QT z6w+d)a9^IK$s5rNc>roiti`M^t(c8`ff(v{-8YudCjIUW=aXx*6k0p*=RZyWlxRG- z6m#xr>(Wy)_cN^Er*Vzl7`M8T7H~6!jpGz-xthF249Nx$KDS)txL$4f(s*Nogoifb z%(N`FMbKQU*6#&Eex6%eSVE)kP=s^W(J$k%+EoX<z7I-ODsu~+NS2|Iu~~ZiAYoN5 zQ?OSq->yC7tr~`m9*xhp8Y^nIh*1UaV2vMa+IWX$I8H^I-Hb5-#LZ`a4YmBXv2TtR zUl+oq_D_DZ_t$t0<XWJ&kiU6cp^FAC?%Q``C8BE4o5P)-UWPzwe(l5D5lnGwuM$%I zS))5O8-D!o6v(Ay6|MLozqCnm6R;A|XynKi$uIf@<8?3Q*NL>Okm_&d4;w5A2nyui zc!NH5B`NS5z}`lgPSY~HrEPL#mbdl~r}ZTqpJqg3mveGPTPk6-pjmCdDj)@)062Tj zZ}^=;@s}-~D5k>P)jQ`t08d+{?rp=m@fWOHTXN^!*B${fpoE}T%iD2VUFDb5A{>W# z+!YiZ*tz9qK7p~bzdjz<Qv_A8E_GT{)k@rt;|(m*Zb;wB`HSz;h~SxHwzr4Yjis(F zca+k+YE}2c7D*7|RhCIIpltG*E9wWdeDXE(_60F1os!nR;$eNcVvEL2keZl~>r0dO zL1G39BE)rH$kA<UfvZleiuMY+40ds?7jt^MLA>A7z_o$PP1b>0l?v+~fviUQ5h78~ z3Ib^ckMKndJ>Fa!1$vSBZXEb<HTe22gA99c7Y->T@PaRl>K%fCeWSQ!feH#*t=8sH zwQpksf>tkMD|xXR`o3~h?@kL8WoZeI<Si}n)@W<)55M)1zfzVHa(#rrZwude_Q)x| zhXitEX)j!!gGm(baGP{|hfQ`g==e*naQ5OZ$u~;~b?y;1D03Fv8Y$@d26J2$3)0Ot zU0;1v;CWR^gA7Yd++nZyS|D5NSPuQNFj@uw*k<k9z&KpfZuB(kJ44H`RMih952U>x zGgNj|{RVrk0Rya=@f%24Yyi_@6`ycM11PN)shJ0=<*Fy2JliB5-*i_{utEohTy;j5 z35NCX+Y2}T+$Rx)WB70O3Y@;gF<PkIl$HadrnHL1H|-+K<}tpz@9@LTbLI0FF|YSS zuKnFh4#rAo1~Y53xd-}$>^T`6husFl`^j9)TCrfdEJJOo;Q(S`L`QBj<SOM~K%{_m ztN8E-8OB*!T}L6xSAl#>bOukHR*tX|Fc=o(&B)Wy2YeFmue=2G!d--fP4gqd4XDo< zf>uq)xvP;svK5~a@GbpxS1eYtUFscalE^*F3*O{<dH8WELjn@RZ({uuVjwM!4Hmls z_^)|jvFlmWPX2f2haVG)qc=e={|iYWBd3R`Luwe`gJ~nNNN4RQdhxYgEE~ae)gXD+ z5>DRrTS&{<uUrb=k$wb(IFKjL&;iut_m1hBqQ3%oPWrTeE{>#Yy$WS5-&Gcbi2Vhi za}2fG(Bn42U5C0eg+}AXJsEsX+qpG;m$0PPV56`s*iD>nv<VXZ)kg<!S|=Lh^fgSk ztmOMd7(J@$nBF<*ARCdrzl!qjyv49TRK5n)4R`oI^C-k2Ibb-*mG;1cY`r^OR`fLh zkl*g}9I%`_jayaqZ~h64wmALupRxfWwStzUEjB@({Pjir-vZ=zr>>vLOn;B-`wzuY z>YNA>NS!k1Qs+md;G;VpBrYHWV!^#W`4+p!Jl^fuEzEMy;*R3HS$W<AR@#z1lQO*3 zeEU0ey%nu;qKl#0eyU|h4XT8y(Wwl=Bb{_9a4*tuBkihv&_sgmE^tNgKMNJOZ#WvJ zG<bLG0se`tTG5=9ZFpTc+!`@SIpvHP&W4OUSJn-epRp_feMLTvfeRz8!w6R#*hD*7 z1POEeR|3B(OtD&$@-S4a8v)RELtV~!?IsGT8SQ|8`#CxssjcTaZi$1w7LuMIZz&W0 z7T{LKVjE<+fagM6TZq<PlR<H6>Ak|CK)b(7{ap;rZ)XcZ%JF^@orx|!tyb|;=^~vz zgJNZ(_fz*m(mx%IZ6ie7i+OXG`-LN{sqPgWlL252NnR=%?T!|foSXqS;jRYI-&<^i zH~_JOL0|2)5V**j|JYuMbLt8YtHMQN(%O_Fu9<EQ1D6`zJGXCMEkOM780WF}+U(8_ z`J@#viQ^?~YOiDqTIaR-fWO72eKpu1c3Lags0}EBOJRvYr)|tbMrPjvEDSSz2A}gW z8NfN$S?YAJDM)T&sTnja&W(WWuKkTfjoYPLAa|aj6w#oaY@PHitJlBwl0m|7RIFQF zk4*GE|J&PL1r<NsCvJ9Jhj5TFcDgp^&En*0B90A0rrL;gX4Im`RquwpFyeN7HOb(K zl*_;OLS+WqO;OR<<{l5B7UlYRz;-}}T`x!Zm@gb{yT`C5u-2OSy>KpDE3G%Robb#O zi{|=sS82cnTRmg0ZMaNYt>$q@3dVw`EC*i&&5tiRX);1jDSk=9bqv%nbFrEX#N-ml zYV2|S7343Q0<wbitE{)QoL*I-Z@7N|oOH=QTn`ZPCLD;tuMZwTJi@5W;H+&{>Ee8L zgi4f(ZZD2Aja+VKc=NgrXsK|fQHzLYo3F2*Ur0O}`V+X%vZfYkG1t5%iF89_&lGcF z$-ECR6aq`9HB*-!V_838{PHc?H%PXK+t_N1DXR<g&RLJPYh2=2Ekow=Qb+6OSE{}B zI_5)o!?oJ%&Jz?P*8r`W7&EdS#VOOti3!FnW_80ER*aN-FRPCFMDcGO(#?Q$?dWK7 ztNRx6;V2IVJF#)EG9-+i^CwR>?0cR$?D9wnpAB|LjwP1AzAGB2HlBUnCaP*3y-_iG zf_UFF>l)Tona@hq*lsA=D8sIN=UEp?Sz!jrJ7g;Tj-Y_MdgYK`=Ys8=4HeGstw2yU zCPT%bT9c~ElYK}+&^bSCs%>gz)??5`rZ}1m)JrpiVAC%S8S!1fE=CIQy(FL-k$-Z+ z1Wbf2=-v~4VEK&eML>O@<6<hcP7g|;>b;cRf;rY`hq+E4bD0vATq95m+Z^De*!@&+ zY`35Ql^&G{nqRQ5Sds%Z+2q*BY1UkO_fG^$jCgdROW-`e%Me4)L3WcFNxSB3RQLZ{ z%n?<Wv{AhL723s4eCgafvR@2B<*^JOUM`0nvXW$qSf0A>zYS9NhYUo0l_!3!ZTrY* zTXYCY2JKWToW6_C9OmcbNS%&R=_kLKI~IpJyG7P2+&9;DZR{3ut=^E|DI0X{+%^$+ zn;D9Ef%jCxtJ1jJT~~<E$_Zv-spq`6v6rC)d7zBdq*24bysD`uyG*4YWDR-*JQ2;L zc4x83YKa`si)G0sc-s?J5<%VMUO^T$aH68187>(eW98~~S&o**?nju{Y6cD0c0aAy zGxriSDhdUHezalCgkR|DL{gO@q-7&80TfWa@Vgj-*bBn<wb(X6;_iqM4?cqcZFzi; z|05CqhHK@8N@)XNcFw5!^?kl7ni5MBK`T9m71Js=yL0@jVKvifB#r&eI8otZ&nTRl zO-Bs_I>cSdhJW7%cghN}Q`E4KNEI&^+2;|tS`_P{!nCFmRx}7&?CuGK4LfM+++_Wk z{ZY0z0T*vl`qp9C21W$t?^vT?Sl2Uv<v(cfhaO4;Zd0FfK3_b6DpFooMPe%ZwyVDp zsxVr8Vw(?BN%MR!bhTPHm=Eyrj_{xi(J(P6&zbBbRaL2btNvPzsGG|E(=zlxfNN|M z12M4zKi=nWXERz)a7XqC$<=hHonU88s+GbE6l>L!Ry^k@CWmXP!efa;+>85w{q}he z)D_?44{dIle1}c=tLj~66^}Haq6eoF+Rh!?Q0nf2uO}-*jJ)0bYtPoY-?@Z6F<N@U zbBX@myrsmsOhuVwC#$}1Gqn?9RZJdeN2Z;qdf&TZf=KFZ_S0Y?CRHo$pD*u3psfU| zjBsmZa1uMyS1Sexr^VzFn7?!4NU%lulBeEXi2iOq6!>)hyC7-DfaL$ZQYiPlhWKjI z*UA2+@U^8~*z7?YbeU%RU|IIU&8XMb4{FyTJTY2PSyPG~{=8>IDt|f6GjHo5;eJ{b zYF6>NB?b3+?7lc08>Rd5Zu6Bux4MP6Af!-6tTWaw1;-!A>h4NheLqF0uc*@vuN&)R z#F`C8Zw9TnwM&1ei606)m{TBrH3&LDC7f<E&R5%?>DJ#hndoxRvw(bpZxF0Iq)0+3 z9D9`lVwYU<WHH=Er#!fKHe0!)k5;A5WIP(@(qsw<sY5=yV|U>bIE?qU$hT_}WpJ%T z;_BKU{sj7_SE&kd1qoVO?{%b5C5xYP)%QbS(!6W`4qtY_kAAhHvb3*Qd!+jrF#jHr zW7qDT(&$CNx|A*Fe3f6FiDq?$!DOx~=Nd^wL#Le60J6wd*>9ogz3xawfxx%sdpxeB zInAzEb!cq~pSZE9RYA{(xC1un4@g)o%s})~oJV{+v+0WsSy^mV23y@9V+CABahG{i z$r32&QZ7SU^4?*e{XNV5NNNGW9({H&WbCO*rsl|)8j`ny;tiAEnwD^v6zkG~8-`T& za$9q_w84=!Gk$F%F~t_`E@5s(y#Y+CYdfZl_%6~E-P(nx*YQGYOikz2D-i*lG91II zCA)gUWj^$)#lWpPcpsXDQ%L=4NBy)2E6+Xi)%=mSNw&i(D{!|31!80OVhSi&_xs2< zG;6Ou)2<JY4|=5iB|^JyMby?DghO*j|1F-q9vw5z)Ck$b2L3zkscAg1w*7`?$(x_} zvTnSi!>C#-M?^u!T^Ymbgzhn)DpA}LsYgoKgfaK=ypm=bqsO!-WrF(5T~3%Y@8Inh z<pBETlWU2dpeiX|S#1Olys}-c_Dn<o11>@8W+4#%UCV);t9`6BKJ<!7!YNcc%9l<s zY-Uh*;0UD`gvyO?&jkOWJiBB6oCHZz>zF5Fu`0?2-A+%@P|Xp2K4$YDj^zzNZ<|dO zDYe;9l6Y;#MgwI+Q|ksH-B)ZkgB)z;E&hy!VD<)FS!apPIa!%0tI_(kA=}npoAYNa zyaMF#&A0({1$|MCa+#R{B&lYcKjKCe%j(~^R|C58a=fwubQ^3_RsJ`6(tVNN-3@%t z`KJc)I7<}SeQ-3Dk45yiNVRtJ^Ilh8oQMbjxSLaWp;FD~*hRpx!VLSWBg`Zs-Zb2= z9tIjCgaS&}g^Dx@%8fPZweC}ms&9l^C(ei~)?%<FOsOQ)C|$dTcsZ5Q(Wm9gt8en2 zcjZrqrcpz%iMrv$*O=q=WgwPLJJK<pu$~eJHt3x8pOaO)W=7*I>VKx%jCHs5yYpS^ zrfXlO4=SElQ3_3>f_8!$_sDm7uilNBxTHxb79?d?&8qMzRUPioYk}>`?%$C=K&xu5 zLKakn`>(R52xl|_h(>eRnkp>J(-jqzxpU%qi;Dy$n_kh$g$I!b@q~auFyQ6FuNv`F zA|C;r;p2gS=d1D)lux58$Gfb@go%gs&dDT?)>SO@oSCHFl`%>3w4RiC6kQ{uXJ2}{ z|G9FI*n4Q3b7NwCDW}CeitxUIIN*OR<H4EQ#+ZPbyH;d9@t*bCO~((lg=AN1Uu_K< z_D#lIjFO&@Qix;C-JmI@-XOWFny`A!es8jdTzzN>8Gs6~x%Ym$om4BIDZTj;)D`En z`1ks=;;gAXmAMp)TwY?>^LE4&7ZOmbTk!4IV)B=tstmu){z+<G{1p!E)E0M&5|DYG z==sSU;(4c0VYwH&5hKminAgb>soQE|cI|LkC>MjqF<cqA6uNZ{Q%TQ-<E+=A<!Vxb z&#d$|?c8GvE(T+Dv#8?%$&}a6Ke$l?pGNKnBP_rxQC$|#uZPLmUj&d}#T?*d$D)>( z4j|{ANux(QP`_=<dYU0-TnKug5x!#oM(-w$^TU%v5eOP;Rua~{NXlvmrO06K2d(lD zG2=fsb0+h%1h>MU_zi<gQ2PUa6<9x9*mux};_gwEyD?sMjFTka<7<hW+OG~-jZz!@ zZqz61?5e>mJ@lb&t&W<;PL8{$9$6^4x|yzr!>K8Yx`^$>+jR`KpJ1Djo^_~2M2em- zWjc}XmD;nPlWu^4m$~+E6fnD<{28s{<I;g~ASX=b@j}O!_^Hn%EQjEpgr)Pi3h#d4 zFnEh8t`AzpocLmpGxPtk_a;D1p8MNyT%d|59u*N~k(OGOO++?9P^s0{1ue4gMS>C_ zNXkwKVu7|Opd$O?Qb7d5W&)8VAeB9dR!D#V0cBA_h!T<jA<1_?0X$kg=kGZ)-#haz zndvao@p+zmU#{PEUBB?`62Ql*tl`(PYeOjE1Dvb&=8L!ONmCszIOK<A$~4}TV27>U z9a?gj*GF}7MXg@^$EVa2E1RqnaAY8z9$9}RUUsO+E%s&Xa`)zp5{y=y%W=nLSnc61 z892^GoVko(ljQe9rs=wDoN_TmOV8in;Z?;~56m2wRm?eKMxU5UTQE*)DB04f-n8CP zoAVdj#!_&^9#`@}@RiVx{c+LJIAmyz!=)!weqXP+ocz3ubMciad2qP<ZMzUGLOM+g z7`3ZttRx~*XX4wb5QNi+9Yop!*q|ST4T2wpx-_C7inxa!kag{9JgGi*@%(oUG94bv zp(C{H$*WQs31#|pC;QulGG5l_i=$nI2F|}~13I)VJ@P){2be!p39h~K!6H%R<8Yh{ zu24)$vZ*}9bHqnq_hcQ{LI5v-H2PVaYVTb4KdBR!foxgt0lJPysEEv^JC1rm`)hFz z`*f7wgHY2swc0r`ju<%kkeq7)g!l7Zb3zUvh=pGb@Usez&^wn_;;xP8CyRnp(K4%Z z<UIIy<j=MFZ)^=!$~Xhzu@dIPLggTwYa!kV%tfx&eqk|Ei<wc=TXlVnLkXLFk@i|2 zxlwYwTF*MYePRnouW5oP!pTQ4C+2b_ut8;+S@^52X015*7ZXY{uW=Lb_CmMbr(=~o zH9jZFqr_^mGBe=Ruqd$Mk&j`QY(p3SxeQ}%PsaCU75ajqB%G@AedNRpPk<y*yDL>! z`mntGknK)7sU>#+Wf>ZhK(TvH`cS|jCb!>s5)knX|AxLmkY>sYfEhEwcb=^iG0n>q zyuUOq(GkBo%5QS9zqDFuan0LxsT8o0E|7RBx!HQdYo!_=jVShIY{)_2T3<fJy(R_( zwaB!rjbZIag!&zI$h|=~*mSldC|hW2AlHK?cN_5Omd4*Wl$e@r%2Lv^@OKW;Zv6Fv zZ=>D8(uz3*X=it3iYq^2LTQbR%xb^Ox5J;g&fqL3V?gwSDTnR@t=|4ICJM-d1e{6p z!2Gn!t>1=gpv9qgTe#=d<B~GlsbOVy1j2WE`)Qb3FT+iC6$)uHa5}j+FYF!y$8>3= zYZdm4&(RqfwUSvPG{6<%tKkY0`08-dL|~#V*HLhW8H~0jcF|7X!U9)}l&T`%3oHvp zZ`^o=*w~83q7o;KuQd~tdh;#dtFUw9m{Y9d*(UwQKX(M50VV={=BI>|<4n{eP6$Xc z$i&e4-?CM2c%wSrws8YycLN;)6oX-kU+OKyV0w`vrX&U<VPAbPf2M}gleUk32|`j~ z^4`)=t31_+bB&xGQF;K*92S0^Wq?oc+u`WDvi$%nJ+1c?pyzQe#uF}^s&qdD$Qn_L z?^0bA2nV!FK(nM@J#^)Uno_`BnbiZ!K%5v7?8HihO^{ILYQTpeB?P43`x`Z2;#;T? zXMyJH78rLQ#L_>m3$zyejh&EvEY`o}>TV!)0&(uIKTwXm<B7b7!KZxrHy+qV080|i z0Go{{&;*~RPJju!gr$RV^C7z_=t}^#8a@Czu+xB${t^H5eJu2oe1uc#7DmJ00S}86 zpii%Vz*v`%6&DtZcgX+oYjT%^-VqQemwUkJHm>hJU;rHiQl?MPtp2{({Eb{;a%~aA z{cp=Ifu{f4cK?3BW^v3ch~YjtW;m~&-Gvw{;BbF#<KWK!vH~A7=%AMWoYH?xD}{#4 z=M?=Tc8+AMU$3<OpSlWIRs8>6P39Djj$}u0!7+HGmW^gV0jG%^WXA`n^+#Mj$!G_W zkMI5|%v%z~O?4HeF9zg-b0i3O!3*2jHjNUepyhUPf9K8Ul2wImk?Z5sk(t~|nGfgP zf~@T$JQAk)?FYKNhTm$0;R1IM&+?7r22YhJM1L^*ijF7LYJ3D8Dn`Y8NP0KARw8^I zxdkc+h9{V%tvU1oC;Ojl959jIT|H#$4}Zu#&uVcMjx4-IJcb!_)miz$LT2|*F2n~o z`<iQzx63;v=9{fRIoe7uvR!NW2??BpdjLxD4t$n&Q~Quh<Z_ufAK%_?K6g_RtCJ>u zY4jl@{xjPo<YXN$UjRi4FEEiCcYH!eBVj6lvb?*~t|(pqp%yOj;As%=DuWx!udmts zUp2<Daa;}*cC26bJ$8gn<a24aUOIfRK!5H)*;r6Im<#O0S6tS=Ya1IPiT<9DT7ej? zC(gBl!0&JW)mx?R4{raSybJ!-2YZf0t@u$_;X~h#6GYwpa14Gzc=3)t^xk;+OwGps zA&=frYk<OQnXscDP7)wj{*X>ws%^}}egg0vdv^Dzj#4ho6?5Rfe@Xt~OTw4n)I#__ zug8G42ZRB|0#Vp}EnCROhz-4ji!t~rPK~;{(dy6OSAi2=pw+?|W6<Y5Mp+<XOiqZI z_BFpk0_OJ*TR^)H=t;{yI;2S0B+&Bu?nbn<`TglR{{o4AF%>6z`$Z6y()@6!e_wOX zd%BK`#+bK;W}&lu^SOp>E=Sil#1JT%z{wTTAbfBNe*fgydk3-*s-16Ri>Hfha)p!` z$bxXh5dGYcRg{Y|$@}O#zFp$!n-o{kcxq&j4RD|Gnjc1(Pd!G%g;{jndu!Qw5~cFL zKW{odJd758aR+6G=9zmyH`8qcCqEESJhv%6?K*Hg$D7+Dz9PY@KeIN$y6Sc%Ut&i* zd_%&4aO*u1up?JHb|usLPh3RNT4dtCD&%13Te1n;$qYTW4}<<{>41A79t-q0F9Huo zGiH_9y|{MI8uV!Qdwl-9;z!>d^Z%W;dUYhdI+qbYx_7|rosj8o<eUc19f6suX^8*q z%-fgS*2<19o}w?r16!+bqJW+&oZKXSVQE&)H;&8Nx}m5}zzQ%%<{I<fqrP7#1pgS# z;7)oGO`bp<!bX$T@?%#PvLmaV$aJrSPo5vaLIR*n_{Ki9DS;+*?`enZJbN@RBd*iS zB!Vp8XEI!_>d&I5=AwF+OnvbA`6-OBK%sXe0ICtIX*&$IZM$Td?|mRY&ScPq!o*1b z64KUFYgbF$Gh%`WS(Oxt&h(>G`&cnp$8Nh}0C~eF+T>de_3mI&y;~$tBBFVCgza!_ z5B*ag3f59jTWka9^1Z})U%Ja&lac1d^W>HO>>oaTtQ09FT`0Wqjb(YJeGBA{`P|0& zD|NfvGedP`$V_jhM0LYNi8!0_7F!#Hu3*$bCMepFflvq)%i{o#F-8RSs1PZf@1>w8 zdQQX;T-y8c$Ezb<ODrbm*d2kDaOmu{66UsIVXt%y1cD3ncQJzfOknRfM2tLEN)%Pr z3Z8^NizHe!JSK~0noC?nFOU&a_SL-<Vql<-gV@oWIIm~^#wYKwQ;FxK71x<qLQ@MC z29G{+)PwvC-f`8zVj#J2>HG*RSrk|rrm@wnU>u?WlMgcst^@Wf7ECOA9{}l6vjm(g z*R5cH<i0_->n%hx&F0gZ6RGGczDVU}b8&92U*vb#YW=_W<fopMm|59(>j<_0fqFus z3MOw>kuXV>$RR&kRN{v<;@V}_y_mcREFO^xTOtIk$_fGEnlK2SbfGXO95_KmnhRUm zbVP6qTR4eikk)kre=PX+qQLUpy|n%e=*R@ji`FJd?tQ*pD7n=(-R#`?KlyB&$*`b~ z*<anW%yHD9OZFGsDW288Y$rj@RGUWl6M@X{Lw>qPL0M{lf3*B=b4~Pq0Z=qQx2avD z)*!>|3L3?y^s`MW7&Zq>;etX@y9|G=)ck|~)2H`T8k#pVV{?H|800m$$Z7wN9(*V4 zS=bFEi#d8AbS&HJQmN)3tuW3q`^-9~ZZI&z3<SPvk-*mj?=8(vS%vewHTg)bp-yPv zsm*p=M)YxQAckrA<F$y8)fqdm!GeB`PA|4?M&1_}XFvr+riB%U#ypjJ28==$uFSq& zvIAQG@NPfY9|%km1-@-g1C`4od+MSwT!%p)N61Ms^rJU@s>BBJ(jChqy<(&^qHLCx zc?bV4f6VkIpbJfM!>y<mvBoWtCMAv6)f%ROp;syt72`QSY~ca7irxofN*g+QgiXt% zv*k4>zK|kvOa^eYybcztm&&|a!J?8Nv8~}jXOr<mKDFc;dO}9SW?0m{1#1!fxN+P{ zR1guUm%OCb2o5zM!}-;`-ef|pGd-;KBEJ8TG0$(#<)Nq{GyIHC+K1%FJtyQTj(RfW z3sOC&?mb2U+c9HWW6T79i@E4+B3tOU*6~t<&-3&{dBtH^wFatYwAG3Tf|l!iv89ra zUT1<KSQ}KD);U{^&e2!H4#w9^{wx{4h-0Pt@q?W(SHE^0QPVeaApYH)xpp-blS;+p zQZe};h?Mg;T3u5?y6Ayh&XH-trVTzfgeosj0>@tJTL5qx@*;uvVqS5ES_3o{FuZCH zUs7l4rAIapM!Q!vc65r|S8`i%nWIfwDRD9pGj9Q$4#S)Ao=ga3YLGJCBs*b3W1ND) z9EJlSQ$|O9i)gk08*DD<FT=yf+YbWUBBOgJ)A<-ejUFF_$?H?(cTAKBo~h2})EW?K zJ?Q0usEI)BBVD@VM2mdt-oE&sJU-+d#$M<o)|Aj>y>x|VV^-475u0a_A`-Dj(BBIL zn|h5u=l5p(wr53PF~o-QjC9dv7ugHvSPqAXx&o$+|K8Lw9-f!!Y6yA%kdI+fQs%QP z$lS_}u&B+U{lMb$xeaMge*qr;Wyexsb5q+*5H`ffYMZCX4$&j|td0>vZLx5a202JU z;Y>or^s8rUO8$}a0QCEn5BuH!z_{zM?W#W>qpgpbca1a8j=LY^LF4o{FizW<vHLJX z3X$&YV8pAyMjGGABD%+wq;5wy=(5PYAtXxd0vk&tQ{Hko+0BhHx9tm9{f~>S%l-Sm z2TOM;Ef!SHWvsx-Yjj+<|D*Pk&eD({?ub)8-UBwL*2QT7HuLlq6|7xwu~lU%Cw66t zU=F`44CqGfuD>_Z;Ojc@`DsGSeiyRBhAgJ){XZs;l>H5VTtUA1wMDePl?AE$w4UJ! zHOS%(G-;BX#&GZhtL1@Ky~(rn`%;7P@D-*{CNSf?sp%v)1=*U(D|uF2NQbDt)C=!F z5}KISZsJ(6EaxYld0=>`!?8DFFSf|#n^v}9fHAC114)SF9(6YSvi^$h@RvXSlG=Zu zEvDEb`TQ@>E_pOtys+Crj(YlCSr#EF``7a?g=(^>fBfU++Omx%=^Iyl_fq}O0c-7! z4MXRQSMi!;Lx#eqRnX?eb9Hl8{ayXm!tB>!rbDfhCotnyUDn*m2f|K-Ic^$#{<X^{ zjhDNdB_cPl;m4~VZHhYXIhizgyP;mK#}<r!M4*;h!yJkmTGWFP_9wGOE&_Y0o8iI) z&Dn$a*U%C0`>99ILHjs3gG{j^C$tKJ=ZXQwaG@c`N+&HCU}0to_-v$B#V?abF1`j9 zSueL4OEgqq{dU@<8u6wfaAVcE(N(Ld7U^L|DDT>HM3U^@;w~K6apz2^1@!jUY7Hp! zB;Td3+^5&_(1x~QgA$W%m+z;;k6X#qf6O(%+a*r65u+UNj=eOOa8?h-JoiO`nB>%1 zm|VfAo<8fnxn%;C8n{ljc_M?$*lBdtbPW0KHaBu+g==<0sgm9b?ccYaqo5*x*V<w< zb}#~a7px_?umW&L6!lR0z4`ZaBH@LkU8nk8CfBW|UaqWOCB3dVY48n&d+_b1axvuW z_)sMv_b@LNc!J*4Ca@?RU9EDWf;+t;ciOJ})Bk~orji-)6B-9%CJ?!b=Nt-Cw{D1d z!!*Yf_A5lb_QLmOz360(fCfE$v4jXL3K_X{oyfQGj+o&4s9(mXa%QjxZqBi<OmH?K z#GQrtmbHeCUlDXZT|EmByxafG+uUo9|1m7m>T>p;=UZz#lLjlr!c-(qFky-LX`zA% zaVyz-w;1hH{Wa$YUM^G}@K;hT@HZG+mpUniye$7!rJ(;0D)#l9i7n%TbUJnXUhGQo z2s?7braBcfe#6oC)J2e>%v61*W~X{L|LpFS8ECR6Y5y)wqwGG6A&6?VaDzp&iB3cT z&(ev=A8l9ve#&?%kx}!2`LhdP`NK*#cn=opZAtNf*B>d<QMYTTM+seL2$Lt*20cE3 znEufs8Y41yWq1o}#c#RAK_1kp*<cGWV45wWpKy$B3>KUdLyn`bfg$PA-?8!V*COE7 zdVI;b^;3U=Wvjy-B8r<0Hm_nA?_+Nm!q?jzeHo$rZH}y_q7e=|77P#2dL;Fb%$jfN z5SE^dv_A`|g#}~XY{Yhc|JB3MN8em_8+6RAj${{`KN6Ao;_yuM@6Cu*?wsX-D8z*V zs^>q{S#RsLN3*#jD*le)b`05VFsUzLv2sV`R#Sod%P6gGYSz)0N$JV+a-94|Jep`8 z?t4+k$tW<KV4`mH2Yh2~e7)O&^-@4fH0AIdn$Fy%M;YNPFC?zm+$%H3mp~M?s$zJ( zFI=kpC7sfJ-!_dX76~ZuxWYl(_omcO!bHRTA8)r#@U;)z?ER)^ERD<9&Sk7Tz|Y>u zWoVD7ybO8Pf)2V1d`Fzb1LiZwDUUBHnr;97I)zAiEQD_cv|YQk`fs`vW!rW22JhUz z#M@6nz+=O@Ps_PX{$5nOR6pS6g}9vWheR4QltpIFX1pjjEF1)LU^;0qNk*2+J{A*? zA`k%27^eBXOK=7>XLKBnmAU}=c6fNjYQQ?{plwR6wU=60A#R|=i5infaxuM_%qvpT zW2-K>>?IKWfS|OcB5E)kDa<UcU-td=h->qDhYo>-@aE}%t_O4Guc3LMj#!*5=x2wc z^($P>jEaecJWsM_TE*%V*SQy7SRyAE<GWE*n7&*?sT<*l9y3rXuQk;M{SayCz4@<u z%P&J_94YE0CXxkCK}+3EPcS0<N$8BzLBSicPL|F;))97ak;44A%r~y2O?E@|F7$De z%Aq9I$dFot+Hpo)%H$DOCGRVD6@VAsOwARHX(CurL2IvE3G}EnJH<s<R%q=PKz$YS zZbd6yZy%nyb>f$F>Z|z88zxB8l?VcHhJiF&BL%PCMGe43_%*80+n-xRTl8M;4O^co zR4#;|kFQ&qrxr%cn$<k`939t(nVB@qjagpNzX?HbFq0JP0w|XJ@<RJ51b%*kxv4+C zE@WEd_y;`a+uS}|FpEzO&ea;aPOg+ac6~U0JRB*$Q8TatV2zLOUn(KH*(Ibev-|b^ zUbi`4V)&9#H?rGl#skATTG^XuY;f5P1qS5+=y`%l=Lfrh>%%0s7+#p3h~<cV9D^B& z3V&9nxH1#k|6^9(;4?7dIRMT%R<tQU2!E!KIEPFcxOg<T&k_`Da;Qf$n&NVDh8I*H zrT%?gmFStFD8Pje&AdZPGY^CUr~`S@ChmFq4kf+CZf+=7qsc8Aw+x+*3TM*{EHsjB zS-qE^$-t<&U=QlS+m|R{N4#U}DWd6b@G4EzDQ;-^+PB(r{R8@$xXM2R-z3-HdcSD_ zPr$LEezeZNPuJ!kvMON?AWN|}J4<?Tf|ie7Dp@u8cW@Et0T+v!c8khj5}c(!Hryy@ zUK0s2{^XE$q%wQ*K}B!u9}U%f_6`3zSZTtmBKfS`ZW;L`^;QK2+w~>qksKJ{Ywk$b z0a2DQ9Dw`fJOlmi)p{4vr&h0=+E>9NUsN_d2Umau@!;B%4VO-@I+ee2d1!`xIO$%< z-1KWM00179{K0zdGX3Q1#INUuBOjeP<}XKM*9x0#?nA9PQ^%(gTbX8HkJtWn>&&=p z>_{3E66vLPx_tvVdVsH>mHQO7OfMdnE=cTjKz%3;{&)ocPU=yW<I}CepTW>p*?q87 zem7V`Qis^W%fw__By@6EwX}%9)?C9?7BrlK?0Mj<{*k~^ka=Ra(PHU|p!C50k~-kb zh^LBcUbsx}4N)N%C|p_?1nwFq4qjK+X*Q4K>@E?4_g1Z6+Eiehzg(E{kZVzIxC{2O zYUIpNMZMc<=<LcZSun7~0T2-NwkF$(qCv<7>q%duO&VwbmjuidubI;;Y3kb_H8t=K zE0UO`W`J86m}9_a5dm(AqmPaCuYOoC{y>EBzbzQ#M^pr&FZvRHp9)}P_>*>))$^8x zro%s83AW7u$vLOW>C;vFe{YZ&Q}zBYXb00wqCdOKada%=?L3(%#F5@cFbmCtg>+Z( z6DC{KT;hwX{+1{#)?w_=1)F-TQrP!nC%Oc0sw3EP#Ja9g@_*BjGlQ4q4C>xCZ4);G z_2tjp0!#Gx(SZ3)l|3>WO}`M$V$eeS(s8-ESq;iz8`nqTbT6^brNXh@5f`IX1>Mbl z>A9RIW#m>F=@DHmW(|p#3P+2Ff<+UIxj${T?Je%DN_Hq<QWJ$;s{IkEb9~~=4a3Ub zen9j9(<!yR5D>`-mg4O@XPO%0<VrGOWQ%ouNa;5w5^DpTB;U@-52w>m*8blMl?;#M z7Vl|L?uFlMOR}cq&Tlgp`Hbetz|WcM{63pKg{m`}+>dk}9(=w@W7W-}`w9f@P&1uf z>UPu<cN*E8zF#~gF<I)N1jy7zsfr;x7%UN9BPbU8VD%qOyueMi#x}p3Il4!ils>-H znDp#6^~7Zg>B`A3(imTvkHx!_{L-7tA0KM$(LtpjFRx?1sdk;?k3_hDQ%47T8i$#l zEjIA7h4jO2%EK*?W5y)`2^P7y+L?mP!yix6qZe;_+<R3azr*OR{6rz_h;&h`O-L#H zLSgy}PQ`8cvtiF%EV3AC(C1^$#x<9bX+9_b>y2+X>@r+1+HogXB?~$N!R)>;hSE+1 z3`XijF;+v?ghi56JO~ip1!oOlO;`jV>$G@)PY-OkQPo!V<22~}tX7#-qq65bW0q^2 zKU?BEQCt2{gmuyD%M5*UB%E}?4--jNXp~*qMacBkel~!Jb^TNEKr4aNbA)YbqzQ%1 zOV<UQTkzo_S=?S+fFYwMyN#hj#NvVu{Kh7e3zak^y_w$8Lu->+dur{h^_E?Y_MY5K z-&|Y0S7~5`?V&Wz&;O_&zunkRq#Yq}bRF<U8NA<xz?Ud=XXjGoz~1y><?6EomfG?% z<_*cUM~2@u5!K`Odo4|#F<YqoO@`X_`7E7SglU5OsB+|3m~seSjpjq%5t6U}<$qO~ z$MCBArX$u@X4{?z{4iL$TuGl}Pcls_Ie^`BG)68leYT*^m03Bb7}-207&`G=^Dph* zvoiM7;G&T@Ms3|uUw&ChwQxW+QfSZcrPB>)j4^{whGG6MdznUEq9R@w+n>=%r0r*N zc9j(N<3tX1#U542aSASBUc0%s?zr*1_J}_{K=9Xon=bf-Y0=9X#O&?Hz;%><hdNf` z+*tBlw^uAmuK1=g`K3*xMFlC%m&oVY8k^QNM$$t~`$qAud}XFC!u4!!En0V8E91v) z1j|^WA<LT8$jHXCa0vw!RqJTh?OjtBl<B*0DRh%QOe2xP-uq<2B=POjd32kHe+&Eh zQ!D1V#g1wycGrb@si{f>%eBD8$#&WP?~~0xA22X?&H2Qv-i$YlAJlEgn5c+8!lj@O zU6%<x=kJj!zb<{NY{B3e>CNHzH_!JJS-~+Eq)y~byV@|UDyw@v4{8;2@lnTxsF|KU z-*#0zJ}j71#d^6`?eu!(&7=@jE66t{IHx(?tnx~XcP<|gl}D4^7^P@mSD|yUiNHHI z`^BCu_RT)y1RHXo3XQ>Ep3$RS%+a9TARY>hpwH!HHFE{mnM$Z}<afp+ffJ!MbmU2@ z52ZH&-Oy1Ua|$(^qxC8Sp#JPPGoI*)@%*vQ5vE``bRS&#GMZ><fxxUw&ArP@kO=+V zlx3x#z_Nb(?BQ$IE$eYM=_|UXXN*aJ(5lFjCuNcMi+_gJs?<Z76BVK7OUsNstMjFE zGkk-vLkyP8X71tJR(<*}ZDN;c%v&)~-re4=Vyh01%63_-@&?&$k92PB^8q<amfaij zB21{kTl~CtPIhxbXnZp2cAvr@?VMT#mG+m8%+`@0<FB2Zn4}RNLFd~{To3$ddt?v+ zA@+RZ_rPQ97&4sh6)K{#?X}I}EJ0uFKx8}D@}hTIc{;8RA=b`=$Qt$gXNJ<YM4<qE z%s?(?|KcJK!9*V}@>N#YfTB%ka6Va+F?6R~q0g?(<zQ*_e)*aEgWvVISY$LL%%OO+ zqLFfjgDv+td}kpBuPEn;fk@{S3Ple_3dQ3Ub1Iqrg}p49d(+XDzPV?g<|SAb=ldeK zsfTW7#T56f@yChib-S7$|LAcjYofBw!o-D3!3&GUF6-~7v_{I6@s2ai`R)S2sK1zI z#>#fYT@6*?@!GMs&f>SJ<j<k9RXue!1^-HL_GsS4%5KK79g<kx9HT@wcKx|nn`ZLm zzStpMtT1B?nX_WVZV$G@S36uk8%Hgolv7-YxCj$QEb;s5IaT_kOLr;Hiay|NB`yja z7HNbx^z{wQao%u>ZWhqu`=|pK$p)V6Ez@Lrz{Hk62oXqgP3%Hz{(1?_C}1h~xijja zLR*W$@pLA3PYX8bC=&jt%>QU?UXWBq5UDTKx7`I+HmAC#ecEMY|7Mx?^ha%F>?%Zc zGw)^Rt<(05Zv?7q{_L77XOyR?(%HwQ&vL>?$1L<V;h1(Qp#v!%0u#DEB3l5*nfz&b z%0m!wo^6VaY{C~Fb_iARit;({XT`qx7jA9OSa%)8qgTlU6V5%uYzr#(^=rm*ZW(fh z?UHmTC8OB%wR%SOyxvi>9{DCsJtf+j@e$srF4?Rpvi&bva~?TP_5ypqv^A<HlEFv9 z*Cr}gL%IE+ZO&Y2;|hOm5Vjw^eR^qU1(szW?ySp$+pH6A-l5o}xgFA5zjyPUbh^t< zFU|A%`x?)C+{R6WYo7mWqE@D&BgixM-0t&w_r&hE?%_)m=349YZr$MVJbLJ<1a@0l zRM%5k{s~t2gvmNm((^jDN!Ns-Ig$r5(|Fj!e?@3!B@51x+1(wb47-zmgko7Ts|>=& zF5)l1?^lWrG{%Q!g*B9w;RV1hx7HJx@??NLSjV|FNA_)F)0ov+0avDKCc2#Q>E$FX zqaJv9hcB5$;^;TlTGbtA^Ytk>*-28U>|Y)TIN&aVTPgL>wtuYfH~SCk9+W|swO8Ko zSBu*T>HfVwdvR>p#+yBP3PI@`e%OVo4yX=Cv)8mHEAn=SU%E3NsVGv^TcO`&iGLll zGQWd-bBvMfnU~gZd%h-}$tD$1QR8K827%geqw%vShGoH!Y2Jv_*xRQ?oFV*3QMv$c zl6b+`(-n@MCNt(wzLr`YK3d1_I-BD$f<;q1dQOL5YU!KYC7jf&tmJ3!4e2rA$Bi04 zH<9icD{FQeN4EH<?=^RIf>=#afqRtXSNN;^C)NFRufE&G2a0{^b8T${<%&PGCbf66 zlMki1_^YqjRSo}<Y3yWRai*D<-F~w+$qnD5Frk>Sta!era;?=IWh5Y1iCM=|M$p#$ zbT$2U`#?FtGA}1SF6~fFQCtpzkuQBAv(b~UT+wG~?PYw~1DVpWFVR}(JXNppSH+bq zsT~!G&XeQS9gMpL!m)r_kU<k~s=w37v$hA$U<dno>A=z$9*o&NKhHS4E+A%xnsFob z6o91xbX=4@3U-LN&)*@QRs(o+!_$_hma)m@?1J0r5m({qz5)GKmwzA)l`D3%4cbJO zIFAPLd0zNdN^rVh--;*#3*?^~SvO0r?qHE9nZ_F(k;JR%hYDO+!)BnPEcGkzDvbZ2 zx87tdG>f<VlHzpsk9R{)<VR4-4Vf?BaI7c|?gzC5+MJmaEb~-mwyWZLUyxPWV>B#W zV0`&zHThDXxOTm(h&MT2;+3<_3ZB+6_(Bb59xu5DFb8>6U{gr?4>pBMNd?PNtdy~w zr%_FHr!A$wTld<eQp21gw02K%V4Mx+=A!wB4|%{(?;@fmD`dup@vObBQjbc+16RBc zpa>4Oc;lu%+H5GQJ1=X4w)g1Uao=!WuE2Ml-YXMNWdaM8(|mwqB$vJQbO%>uLsduM zI`j7W;oG=thn{CG)8TVF*_rKKkIGMGm)Hv$*=Lv!gvqxDoI!Qf8urpm@CkjyTKg|J z^G4h)N>(fKS860K?)HDyZ;8_TpDHSGR&xAl!_t*QiUe!Et!Jdn^v@o#tN+7G`iH$q zHn*=?toVx5hn+X|I%v}6&QNqvTr)P^!Okk-bSSR3)aPGNx#4oFV63d2rN=)SIncqr znad7n!vldEV{{LZnH;DYs*;%Q-c<hB+3Iq7a|8vB4W6;5_7{u>d5W!=`=M_11RR26 zST7;o*3F+GB6AabkuD}-L&Uva6HFmBk@RawQK)R#DPax}?n+Mlv3Gw0S2*CDXu4@! z;+#m8@TBwsd2<nJ0R)SV7}WUAe{S%H^7qm;hjuahDoMv<ehWkAj+e5n4r>sc2|hgZ z1A-+d?TS>!UmmG+ugS#unhx8%1l%@M8$S0h@mWswXxZ!Z_AGB6L(js~6PbH8oo+f3 z4%D8Fo~hWY6Xpn1hm#+p?!k6{>8Ylor1LsOG)HzfZxr#1UeN*ENo00@M)NH%ENUWr z>435&;rC@ma>o8d#d`K)4dTwjqW)bCGMB}dy=loB?XZn2(rdIg77S&%-<dZRV9jl- zhVA0ewA?uK*cg~uZJww+Y_AtVD=w{BBeAva;PZmT-ciE$bYZ>7<Ze38YA)A1lD2Cs zTw8A}w6EgvPxc&ALCIROZ_d<#Oaghx3kMuZcekW4*y}lgjiEVbGyCdty8Nn~*z@D| zf=9g9b@gt_iQv=_7VL|}ekEvkd(~Y=4$PSYy02$?G<yu$Ab$5kZ}tAVC5m%x1wV7y z{L=mGSLwgq&z?9L<tMxVLBMBT8YfCi73&J@^U6uOHotml7F}bSs=ukwu?zZv(KymJ zZ|eGdKks*vN5|7PHMDK%g(@Fumw;T)(y6l*V$&TSD%E{v9HkaYrlM0jSazBVD_OYQ zL>A^j=&4i243v0yc5kKPj&1ul+Z;z_YDA^@HbK)aGeF5wm-4niu)h*99&jOM6SGpC z!YwzfV5l)kuye66PVj{Z>t+I(ka&pVG=^nEiOc}Dz;rX(Ej?&DTrY&fh#R4tF7}d4 zpH$#>4FUont%p5uJkwPm7u;RC@oM^z_u8IX&6+5Gyj?!&f!hx;I(wC_-@d)8ArDSI zb@S47nLlOCbGPnj+hdvTyVveI)9i_CpVD8!e!<c%c@dRX9in_c0W{xMc+Sq+Ra=l7 zO%=k1dMC`ya~t~-Bt1%tWcL)(OljfnxP5%joR`%~+P1{+8!1zK<Rtpq*i0vj?6EJ9 zpo=oen59R&3X2<R9J<&+3#B0=flmWRE%Dig(ZCl;y}3sgwn2Y{oou~*%m0ieJP(<l zdD{d+tr(F&Fg^l#WWF?p|4x_OvkrgRSo&?x8dK$;r1u1jK9aqov=`w*&Pr*AGpvZm zV`JKUZn&q+yX2>-<rJXD0z7rPZ}^^C8@-O?o3{1g{vZMxeoNCFIZD{~wHvMD7J_`o z=2U-zRR^awc;dNfV*W4F31#Fm_g{UC5N?x1G(B4o;O)#dX`bQ`R(SV_>)1V}qg9g? z{B5>lk4s6nnWU}W&4#p_OvQ{#kM}mlqcX>CNSNN6-HncYUYA|~nDg7KN_tOx!4<-5 z&t(<CVdd(M^u{(>S74)kscNrlrm-zUaqF&%lR6y%$Et?IF*Dp^B7*lgqq^5Hycw%7 z(aKI8YIB~XC&(W5T#x90mY+x(IOrV8D`4h^73KLg?}85^e+cnw<%FD%6uT#t^W3RZ z*vN{K0{1%ES}xDh@2DtmB7kO)8k)fOuJH2XcL#Kd)tdcWjZouZnOtEUi>|9u=N-~k z)@F<1AFlLJnG$9a>WnR(-&h3lAa_wp*P!hzaN1+;h~hj|ydr9N2O1gZ1LrlFe*+Qw zOQV`fM%50L?_Ci#V)tgZ#a63m6~(I8QZTu-sy7vXz5uV@YY|%Qa<|73i!T<{^3i;$ zkiU8-=dm8`BXxx%c^y=J+1p(_?((xe{BB*#L}$NT#hvYx%;qc&&zvz2dNyF4vMmk- zsg8#Bt)Xyz+O1h##6)zyY$g{7iZkRjx0!r{a!s*{?r1D^=qar3qZa6C;75YGrpubY zsoaIw&p(RnKfnn#<PX(xc}hHcfwskcD{^sB!2mLxf_GBBJcn*$>y7$5dq&m)_XP+t zJiay}P`WYF$U)le%wlNbv+#2N9jNch-CucVuIem``XScF!QaCtHcvL@EZv#;TIxmE zCEVT>r>pUIX-GYKO6|dsy7ea_Y2_dz8mp=9C6=YKhBvI+E8n)&@n{KK_VCHI_9FoT z-phA)AU1$lt_jX*-nvQGrB0rWV|Me=nTo@^`7H6kY)@gKZ2#l~V&74ZhG8GtTz_EE zRIr<72z5<7wY=K<E<e1-q%JgpBEO-763lv8=Giu%(6fU^*Gp}sk=Xmcv(T%z6YQ=; zp3m_b>1K~SHQix_DFM=Na6B9gD3FB8wrQ*=-Ln|U0|}$3lDuReUaC<;SsC9==5LEx z$uEU4MdJ!v2lwA)C}(&6QIwt;Hk2LU8kOI1`k0oL5MO#y>f3P!qI&P2dYfl<>8<>$ zx3W&-gq2#ZOvS@UCdcV0kKS?mguUSQ^4(hZtr)d<6R$5co1;(Hmi}muKI@7D_|Sgo zVrS(WSz5lDJD<~ax-6O1y`8+uz~Q%w^s(B=ijx#4Ki6ZvVDSf+$Z$c#glWecD}u)K zP5QGn*GTRFvFX<jXXKMMhqi@gjlIO$HuDZw)(VpuWyN=ciyVB%c#|?UT<=HJGlDxT z3SK9cOH;}`vybxDd8{4FYV~E;PCX(`8kt72Dzya~;X_qEsZ3VSnyht|yXc8MYi?5w zsAkA!w;4fl^I$jHBuGUE_>=*JV&q3jY`4K*5rjB<{~3^i@_~JbDW<I%>)aP$-oI15 zDh^=L=`1Pzb%6}oK$1-4xm)Ldw}KmX*#8;Sv;3;!w{2#b-2O1NKX%!qou}2kCJh+E z=@deLZIX3`LRHpJG3X{<sPyfyp}RSRzRo0y845Ez_F7F~S{Y26@Fgf$C$#T^zq(+% z6L~({B(AgdX&J|e`4dRJ<R6fY=m|!;s!Y*!zmCcfTzmjH0|^J3QI28mwK8I56u6et zO+9N5G$%aiM3tqFJwA&NR(oGOpL5*G^^9rZ`%}s<F>RUMmBnZ)upmW9a<c6JJo$xG z9K9%=-wFPwPJXozIP8|$*mp@DZ?wbo>`+db{X|pItsjsn<+>WbbgrqK&e|H{qjpU; zabiuN^47o7_j`0DKkePKGxSxE>Q-`*D450ZxO}=<yOPqUIvYxH=?+M>dTiChsCyYu zWPF3(nFL2+0g~dy4fXNY@EUEWAe^nn|H#^~-p7V$e0fy6nfIiW=q@M#8)oUiZCoGT zeWL1wGjc!A(FC94Oby;Yyk$aw-kO2?am*5Lyl~4{Tsc0gnk}et5I61EQ8h3ii3Wo} zpnn|R*2KmwU!nItu`uep82?k*R|{%4RLpI|7K&=AuDQw^%O7|;5Ea)IuvEwiwiR~P z)uY`Vp%xlhf4Lm$bv!C`Mw{vXx%VNJ;ZkfbyuyleVkfh-3)wQpT~ms9@8-ZCTJlMc zyZaQf_6HKscy=dP3+x4dWa~7de>_x#7w1nhKw6u2We@2mc}kVHyXxxbyT&d?GAx!i z;zUE`BX3NG4RBk})3X`GUcR^I@5Om^{%aP9s<)Y$@svu?!7X(~+-a8vx=+cM1HtQ5 z9u2_P^r�!$);n*j;nE0kW~9Ct~7O9`kyCVd?g-k*$wD$MW?qT|7iVNsaz?8a|R^ z1zeamZkk9Ey#7wD;Q%;Pv>dQ;gKzn%jS@OLdcrSwbEHGdkg<!(Mv>zU`5Z)r)~FZ4 zpm8!TR@Mu6oc(hddq3j;uf`x`pGOTH>FeFWr$8ZWFcOo(mB@EeN-m6qb@WV6YQk_p zMQhqM2H)HuxpyrslArN*<Iw3>*H8#K0%j>(TX=D%ZA2#G<m$4scHUbe3bvoX5AE+j z-6r4l;oRA+4!mh-3S%1a?0tY!5AHzzgpcR}P=-UCBq!+yk&h>RMwLM~1Y?UWR<%__ z-AJ}<B9rx?*Xzv_<`Ej}UhHdaZunbMo0+f>QD<%i37F{spzA*f)%s2Kg-s2AZ(y(g zUWN8Hp4QrFl*N4jfAWAP<5gW*H-~Z*UUarO_H)Ro&MV11qeu2J4<AdiD$dvvTfk&6 z4foNrfm0j6UE1$`cC>m2rFaUsjnP0{+}*(WYXb6JgYLC(v8qkpv&E86XUt6%qdaIn z$vVG-9&3_WK}EiN(-*1@2Tqg!VNF#`LrMc9^+C?fpM}6U#_62(uo(d&9yq!$$C70Z z3WpgpZ;Q7Nrga4wJ>PEMP(Nh+V?%=v=SyIq?-Ak0Joq1u!>^$eHPzVQp_wbQBbuP| zu6nrTTGfxExot!2dAcjzAtA!g>$J&_YKK(DRU<gIe%X4It}7Yuue<>rK*{;<r<mrE z7#LH(F%+BKey^&X{d07@X$a=^y;SUerb9>Z!DY5cshZtey7r;A^X(5c3>kC2j9bZ> zFJY(nw%l&yj1bn`l$gOgY3FX<uyQi_->>rbLizo{B|zms3?yr@eq6O$v>upNt1qpi z?fJla(&X#+=no&q#$PC%J-vrQxaTaR)}RU*(5ri$S!mzchrm`c|KHa(>m50V5*>7$ zqFuG=aJ2ci$C5zqq&o|n^wb5Av;Ne--=FJ4TYIa)Ex`PVTou>k=B4G=)k75*o8i;+ z5XcAr!~5YS&UIWc#4`Ess1nP^zDBSY=Qp4@jexx3AFdF8Uj9N3KVA|s*U#vafxNgM zR20=h)5iV*V6mtJ=MKQLb996PB@Tfp5S_$G2QniK5sQQr3{l!Oa^$A1S<cJ+D$*je zWMN<}^wB*Pn5XcjWu&;!@e+8$Ttm1k<3a6TP|(d`OSJ-;$rru0fs7|aWQ@4UhP4=t zmBiKo->pVT0xfH&0!o0D`4rI9Ip17h<kp8m=6qC(+fxlX7<UcGt3g#vMmXYZshS&L zu%b(4kp0CA1^ja0mBMwM=h@=DA)Nz=?Nv!)d+!=!F-UBRp8J`3`!q1cYaN+?1+wp= z+3W~-;bfc42@pt@4(g8wPOpS4@!M)fjVPVSa@iC@Ub?me>QcL8a4kgEIbeIZTnvm% z8Zq-6#|BXEnOBCG(8AFS`7{8hYF+7%k9><GR?GW*3DPd1g{ppCSwQqdCM?FilZMlU zyvn)=;Tpb6za2nJ)DzD_LwD)zK%?e!V2ihn#0Vy_6287qem!8vpGnyXgDH@(;m>6D zdRF~5{FW*kUA;US80MIaETSg|uY2(-iK2KKM4T|7=2on2{pz0q`a#pq6aRqB48sm~ z7AZ=Zhp?HB?%!O0UQ7!%pB0w4FvJ96g|&wTsc$*VZku582;#d2#w?rBEn{*m%dfUh zP|OhI)650p2*1K6Al0I*UBnNUUpK8d96bz@Jpo~S(Pbq?5GTkn4K3$J?L@z6cMaLR zC1T#B<eF*ygY2UG9Jt>ySkcX0VBo!ZqGP&7Rum5s3bn2%>TL}h2Ym1<)Dm!~9tw|H z*(&v31!|xy)67HvXyw>(`Ol83dfEe_<<`FV>%sY;feC}}nn7LX=mLC^^pyvUeEh(s zDgP8xqDtFx*q}>tjRr8*ey{NAp~ml(JaT;KDPW3Jswe^4LRfKTTsaIj598k%4h__i zF|}Qa9sg~7^}hwlMI=P?1K)yPw{``?_)v-p*h+(Re$Xz4N;mxJ(oN<BGte{yqb0q_ z;Q{!vYGm3yX&7uj<gNC+cITTu)FQ87yXCZrm`B561WaOx@SY%-AZjJPeYw!rqQ|wi zCE~$MVt{Z=%;gr3#8Rg$hyqn2zAMj*o7ujOb~9H_P>d1xn$OL1y~NaS4Cszx7Yv_; z1d1Vp)W$l5`9osK2)*t;`ZR{@nF*AmNHamr+<kl_8c-p*N55e==5UpnRcBQ%6tg90 zh9}RSzMtylJC<yQsBTJVPs8Ot5JOu%g=#H;hhV$P+FzQFVmWuH&l8^bD|5%@2CV8_ zv4!M<K3h+8>MlmM>9K`gTED^rmZO&ggqc40-@@z!GjBc2yhP(AqH!wY?PT#6W!8q) z=HLt$(id;W0CwDFpet%lFLg!BSH*9vq_SfV73*sF;52sH4wd-0x)=MT+MGvJIr&oF zaAe%pnwPw>MWPQNw(+DMhcU^%Td{`4qBylD&Q#uswaM1i76~U~y2=x$lgu<;8G#VZ zoKH7M-in`;y_G;<J0XNzYA(hco^9Ua)%*Ao0R-bm_apPNIKwUr!LbJav$C)~*<r72 zD=vr300L}O_258lxQpu=SpE{pW_w~OoYbD)msh1Bo3iX7x&hZ4>n?C9-DBa}<t5b; zP~B8y7N@r|uMzLNqjuu%1e)EO;K3WJgY|W~IXu$%Bc&Ps0H@Vs8+r^VDNo_32YBeP zu_ICja>gpv35{iRr%TVue0O{!IV6^@`^gOy=vu~^e89K21h~dvRt-d`o;(6cBIZ|t z!Y;C6mWAHZw{VEfY?E=o*;b~=r!ix&O^-4132i(Ny)kYt?{S2p^fvN$wqh6hkL}jg zGHB~kO@UUki@S-B)uBfd#9AAJ6`llVQg2Q#(pNPhhSYR8ur?AQWL2F7CS6_o^T;9H zYs{#T-t-=-V>=IiWI{Q0-ldicKoGC+;x}&u4{f)5(aR!H=GM8aDw;Wh&^tLEK4H}P zmJ~_noudyK&<NGcEmx#8T3Z(<D*fe8u&9C>(HxVa(m^!`;mIh}Uz10G@HR<x87%MJ zl#zRAXyqI`m{DuHfe)WYj7~(!|2d8;-qWA#W*ys*Q4RNDN<D9K(J}GSlTE+MN`z0| z2=GalN%j<!MLEQhn|Q8ze2)@_tY}0d`*~lii$J#jYRv4Ed}DQjWBS-=HzBAa^w}CQ z*X3w7!`Ek9hT`KsT25c#LhQWdnl7w9(roV4T`xJ$Os*Hy2};AYuO!)rU~XHNR?jI| zc*UxO7<w$+-&^x%TliBAFO`lc8g{!AzMoc?-cT%)=$W}!pv`9`4j8Z;^EwW?BGY9z z;(K^OJ%^)wM;m49)(Q$?^zNeHNxhX?eYR-TZ5ujDN0VRLz>k&Dg4tV6tR5Q?vAEg3 zVS2mrd^4L9mW3uswsGblwR8*2Z3D}?@nne-Sa#|O{yH+Sig!a+&fhf(*%5KTAKCQC zot|~bZ%YuJYu~%(^2wf2mo0!d(PmJ4&?!R=zl&})YBo`<;U#2Bt(Ma#zElEq-qoYQ zJ@T5$85L)!X6**GoS%FQ+HM0>!gH^BH^M@D0T0=~&`!O~{tOpmS*Rzk@$Udj{<a;k zs0;7r_=*l#r`n~mtkp={Djio=FMO&!iCP!stRqWZBj9!<IUvI;*e=~#2h@ktMn#l# z_TE43RRXSrjqY#XRq>LwH_-!09NEt&xpfzz%aJ`Elz1nsLT3ltMn-0;=>1WJ9)4Gl zd$0TBfoGmhCKyuhwTRITXPpRKyR0Q^O@^mHw*G4E@{sefkJ^0n;IUitSN7zs6a}Qu zN2C%?66&&!lo$tYuFC<ghx_FM`s32jqjjX0c*JOHj-pS8FG&P)J$h-=)~tqSj_zuX zT3QgHZ7Zr@QW6g6m8g56&}j&Les(BM6>bGa)zTh0ybS)P;P8)DB*p2yTc~<ooNZwp z9mu#}NIa>Yw!XZ1Q&w%05&yY1N_AB8l79RE7Hxy><foOhW6^M<hxQoP5=$(sPYpdu zuMF+;t_DG58-c}K?gk^4mLcAmAacsXe{(s0*k`eY8g7BnvizOfKN!&|qd+0*$iF@E zvN^G07tGCkX;cT?nUxu|A`RM5WV*P^y{)`FGq;Y96LeZ^MuSGiF8EOjc=!vhr%UZG zH9ADL`6LBSgg2`dn=!kG5lPR{0{r8|Mv9!VpQn@ph_yDmdyB17Gs9%PbEy+aRtd|D z?f)5`fR(0(WjmTBYt@df1tmH@1ukdB;ZNZ)08kUOGAP>^ExJ~Ha6q*_wQt3b`56tx zx+q>vz<5z0f#h);o1=KHkf!XF2}L?dEaNU`K`GUmZu)hf?d?aq^ypp8D|%6e@M5qa zjnt9Y*K$<@`oh}1#Gy)gyxewHW_o#@xYOe_y_7BAH?01z&a<%6Odtm^S5@yTHv_u^ zEX8fYx{;J<Hn&q@8EoY&J#_I3F#N6Fc59oNal8Cq{(9hyttj%R*7b~s(M|PbZ9qOi zO3w}ugtkP5?>0>ljI7U}XzaDr*K>^#1jTfXSCpN0bwE?ET&i{{q|Z$#=lAm5t09(8 zpJkj<<=)FzP+ZJJz(}&k=~Kpe!>1ODDq+X>tLUzP`RMW<4CUFb1NsQJT}>e0z_fW0 z7;00MoF&NRH~7H^9R{dgN5<2zlL+k_aMie99xaVD5-YvDqmOFP8T=X)hFvb)gpjV; zuTpCUuvbPMN!ZEsNL6&nDinL99@n^HPw&k98l`FX{VALGuf?+I%V5{bY}XnBw|l5B zOPOG05;w}gCceR#0703A-wv$SQ};gI7dT3=C=T?Dda&FfSLm4AJy-1%v^WoA24dU0 zPWbnXKCMGwlpl5cNX1VdM&~O%mstbH)UvC9Tx*><gYUrDXAwQSm0_xr7b{F~_UkYg zmy`r29`e&!Tpt%9y`NgZUV4sE7@G<A2YXF@Ukykb>&VZ3Efnxky0C*u#B4~?0kQd2 z+@$#_GJ2i|fonpVS~0o;>nSLYw(r4?B_I5Wmz~S+@I1A8B->j`6%{@XG#iua^<drf z;2yZ^vC^!tzA$&HlK~~Zz?}%<LEwPJ<9)F>)E9vf-E-2gE#+*`Dtc9*f!zFd!_xTK z2h8W+G&*g9?b(hoTxuoL`V!$?imLMU2YdRk2VM2N?iRYe@F&Qwk9;li!q1)OEw{5e z>~9qf_XH%3a4K(IY5_XKw<)V!j%mV&A5hZcdr5bCqr1GMg+b{IUH)ux(qNrgQwJxX zFEavKET`EjwJ<1d<#}fH$=9h_f4mQF10f?!Hf_$0Dyt0&nRY`)G;&}p4aj_3pDcr| zm<oP3qE-~$>f6(CmiHL-`b`&J-@;XAla0-GEmva3(8GbkUz-nKR2&DS%AgD6Qp3&P zcGPh=IkmpM3iItFyT46g7+VC+&Zf<J54dm9;d7Gh<?@vpcY!p(mWrE|_4tj+b@{PV zJgrI{f8){QrIXPjeCA{10vgi4<#Nwo3uTPoRkGF3EMjJl!WS^v76xfCim_d8pV!8s zgWcFSb~>Gg2H`HT_j(br&t(x^2Gfs!@9|^IwY%RcGu~X)9?9Y8>ZRE>JQ?5_GAsJ> zWUqUcUSwR{g_5ogs5%ilz^fe~RGvup%>!9z2O@iMq5(n=B6sXab}mrQX{Qc71Jr}4 za~XQ7r|e_<3f^x`#}#l`lxhGdYk4o2oG{d$!0(tlU~XCSq?$$iY3d+Qo!M+DjCRrL zJKInsV{YxHxgy%5p;(r{$~4w)Ja1g+n0~uhCLM<<iLx!Oh<P!dE1!}1sCJ~(IXySg zp4sM!IDix3<q+mHp7k58$_qY>b4jJ5zB;cpYE*M(cjwzSo^kt6Y&+T`L}o0S%D>QT zLgy#6cP6>%9JCd7MQ&B8^iEy&JYYipNvKP*q)LFQl4xxr-RO%I+Mu}!4Oq_A4~Gf; zD^Rhuab{`UHg{03js^i6Ktd%1ii0%l={Fd?cSHHHjFD&6h2L=}kSPxbPK4G5S%m9t zg74I{aF%at+w(%3PijhKaX8L4rCLPJreI$y<mNvM5`a4;v)im0F3Q}xK{LEU`P4<9 z&24)r@pF+-@n~xfQq7cN&&XYz*pl>xgkG{ZHYTW1im<^il;)O}!@;wVvw2`H!4gW5 zVKkfhZn?f@GgD|7PEV76wQbG85rk*3R81`We4!HdD|4j#!w4;3%C1YrPCLvQUNK*B zs@dheyu-m3Ope8zrbZ{8Sxxm)Z0Q@r`W{eM@1~mT-#6O)t#ZcL5uFJdYCND>>9RBT zkje)W3q*}zw=TW{E0GErT($?#PWD$<eyU)<;caO)J9l3`#|@Tmq09FI#;S8gbezXf zd5-)}C2e!rEYJIec+VO8V#F3RX6+Bbsg2wOsir(TCcRQlZf%maV1Stu$H3Kzu^u;U za&cN7mum519;<q;u2E<MK#L368$tSTlL12FIGf2S7$f@&tt<+-LOFXZ&qA42=5bo^ z_Nm9g$@j5-NN)$j%3-?vLGBRH;aQJ$p-Bh)c&b;xoYt&XX$Kfnjhmi&$PY>EBbAR) zIP-ODw@sFF2|AlvRQL0Bw^#mMfl?YaKIAR3QMnUW0bXd7;sN%p2mnY!87ek!Fik1m zvn;ehX7zQa?7sn`FdNGQP9#Q~1<0%co@{;t@C4s}N0k>K>q?IEau_ydWuhlo&|OMF za!Ff|ixjN;i=jKPx<z$yAP@*VUNi#TJ}e+avvN{P?k%|W^?VH*1oC=20O<`&X{a9y z2$US3oV4G8f_C~)h6a%O04j5G#_Bz64LR;*IkOhOWp3PI<j7LgL7^yVdu!MGV-x6c zVQyVn{~Vg@p(F(y-~io<{qLTAj@}9Lxk0?Sw1fSt3PU*uG8azh1sz=T!sJPdWrGH@ zChz9Lzo@Df3Z%tasA}5dK5@klq#stnX4Kw)P@sUB8m+kH#*yF(2I`?_L~k=e>N|;$ z^qmwosta=XM+(2o`H2gip(Z>g#r7jWU`ddH+3ElDeV<aCO{PI=%pQwcAVG0qR0ksq z;otLWF=A20qk_3mfU{RPfaw%B1d@@R;@_#~oTN@Hh!@!`L;P|n@X9Yc{w^mpx#jYO zO$9l0{c$qlKoR%R0T?XkrAxBYci;u6cI$-9I}~m;Qv9v73;>Ic%y@<NL-mn*J6>24 zIr-dtl}z0kad7y?`hLiU-YYOo?go^{P{j-grZf-1U^Z7l8aY$Qe8dDB(6<SV6~Ees zA#8+{h&nY$p|EBW60#m{1-Hzxe0Xw8+)?$^t`lItm7rC#5L#sjU7wl=K;@08P}YPV ze2AI8O3b1~faY`B2}C=CH6c5X4`f4&2^C6PHwn_#nJO$iCVFXDf{%vk{)F3KX_iV0 zho0I4Jr&W3$eV(2IENBjV3u?hfK_Zs5;y4P-F-4<HGa7?^jrmem8cf;CUwd|qD0(r zv<a4#2+|^g1#fE@0ANSwo*_k3T!DCmR#GA+R(`KC_yrs2!C0pA&E`aFM+;T6h_rh^ z@wmG?SZ7WTBhaHkfT|0INf*!0iG{AjBU3QU7!mW*o)rOMuGWqK4;=&YEsZCUKoCu3 zFl4qu%G0CEK#N8NUI+IoawFEZ+S=$%D;pE2G5zh7k;Wdoz?(Sg^aDd!eQ!4~a2l$w zjAZBf#pE1Q@dAER*m5d_c1l4vn+pqm0GKhBIbB8>ym_q097M!Pshmgazz1$h@Ww`P zEUZ*ncT@h{+#5?`nS21q(P(Y!ai{_E=sONFG{(axTpmWBAPUSQQ5wVFPVFniHbuWf za&@yaF1@n1D}{g6yG{-IL?iSG7A8RW4HDBgaS8N-#y6lPH7U?`$#uJLua^S>!lqQ& zY<rYvt=~UjQMghj$N_2E4ta8eq>UA$pa#KYKudoSk`p5>o*`*mSD&3vqanb{iqZgL zTWSiS{$YF0nn5iu4;fs0fD?t=auAE%6QncugeID_0ZA-t5=c8hn#U?V=mrWN2EVFf z5*e0h*z|ek?WYxASb*NFl;HAE=i0DtTBj}|_DHlK7YU{7WZJrl;~b%A!-5dw!xu%& z_yef^wRPk0l+6}HL`l&jFvivbSUs$351~fBi3FGeGJU&1e=J(xIjKwZXHcRE?&<p- zKu_t}(W}AkuTH%O(s&-(8>Eu~0vrIK{PYKhSKMBb>E;1fD7$4J%f}YY=e4g(nhIDm zb57r1;`{+o{oi6b5<nho3*aa5G+D^?{qGp7gQuYAz(O+&ehJO#_8XtYR$h2yu?v8u z-T$nF`@gm-oGIYRca_g(i0AoC_7X#JX$WmHKD}T^{BNZlP`IthFrZ^`=!Xo20%Kuw z+sm#aOX^auJ#$~YQyx_oKnwZ7_X(kk0l)1F9jBf6X?6O?WFSefQ4)oMo=iO|VX1va zV0W<Q91AdrKv)p`qT26%Zg}0_rJM&M&LBn(e)az}sCj4S@GnSUMmqN4;3xrhQ}7#{ z`q*dT)2s`btkj}faOYZ>HU6QW%W;q*AEyq0ty=OqSe5DLYRne{r=WYFpOebFquB>? z{(l*--7a|y_PO0Rdc}eSS|9fPw};Mx(*YDW{(oJ$>Tg8w|2?|$|Ay{sPE<wKfWz?^ z(G{aZ1fGkkl($Rnw=rYSK}yW*>+hLfd}5YBJ0M8-3jLzuhpckvIumQA|2l3va#R(a zeAxrGdB<mH3agefi(Xq0w8tIFj60mY;FciqXZ?UX_^A`YXUVq~mMWOS_Y$f8XUU)z zW`F(vN8WotHJNSkqgZED6vvLV(Ge^(0g(>ZacqDxg7nag5CuXIgwO(x4V9*}5UORs zK}vu~LI@!!3PONbfB=C6q(liJ5(oi8;GKZZj8pF1d*56Cx7K@Ev$ACJedp}6&px~S z&e{7Mk@%CTyZ?isIQ+jklrenN7zG?_7>yEjspHlXX&^iQw*>Z!jQNi}2;fpDME0k4 z-spz4OG=df8-Ju4l=q#Vu`i}ff>Ocr1FYFGP6BEK>cYWkqQ-RStT~}u<d$7crZ4Y< zgoxjqqjxg1aZ!Ta86J5OM~@OG^$Q;eIwj@BiWkaqIo{bX`Zn6@^AVBH7fSrW<8s1n zo?>j4mLIR5>q>B)pVW&w?QrLz7x#B*{};G+y$JjL{!J9V7#tEoanRRR7<m7~pz_L{ za7%}l(GQD30kZFZyyJpfD9HOMvGa+Sk6ld_!MIgY1Lw+>k@UZFBtQ9I7vib~v`9bB z4xi?A4p{jG#uwow<-npQkpjo^hCy$YXWmJi-pb~#-wE<tCln~UMhUWV#4T%Ajiv~W zQ7#s6DWbP}+2Cxa9Xx=TeygNwz_7wkQfFl`&GfH@x2TyXPQW5eASUOI)~sGL#1{<r z`PK)4LxR7hLg(quw2)XGm1yHXKJ%Ae`Dfqq0^IH%V(Q0oucevW*ZU4xUem30$M^kV zAlr-n#rkMk_NQn5+He27N}n90OuV3VhyCL86U7%tTfKRmn+fDlZN$kjQ-)7$M)b<f z;V2Z_Y`zP({@gS-YJM<Dbg}?DJT5aQz;-(w!-KnzrQCZKig|#+U3Z(oID?Ccrl<!+ zg+KOvY^&XT02Jlo;mCCrq33@rrPC#au4v!i8KlhiU^BZBPAJ<P${1GhQ&S%^C3}8< zz0!Tgx6bkG5hiIx_nh&Vvt<0#tILq<)5Ae8YA^S_ni&D72sWkNug?}_fy*FqYIx6K z(xfZS_{o*4ovidP=X|@EQvETCSsDHKe`@NVII1%;*_ku<+%AJyBs-;jnLzET8~Lu8 zr}Y9Q9y*A}eC?C>e58O~Jx4$HMIv4M`5Qg1bNN<&<rqf#Pnyd@NQLkw(>Jq<Hr;fD zA2j1uiI;g(b=W8;>f}Ez*$mx4ches;wW-X#@hfJ$Z%|65n8h_N<edf#wC-V#W;F-u zQ0%Sdbku&4kJs^%F1wonQ%C{Zg+ZREjyT=)#R}P81WJ?4U1(Gbf26<VeSz#;-O>=! zQY5&|($}qB`noR8T|Glb<)F{x(-ACY$5U_T9gTggwj3l}_(sG(vQXt{vHrv2y!wA> zS>ARPV#fF$XRmWo!(9AUF86VfZ7rZr+kMn@I}C|Jiv@Vf_u*~F@I`bv*h_5tY1BD} z_Tek=@ZlUXU7KzgnBn7G*0cT7{2l<e!jtFTKTP0bdDj*SBt1Fm<5g9KF7WTM#!{Ov zyBDcm4HFo$z~+e0O%AV&!kn}}kDYmm7lidoPq!I@`(T{`Kj2nkoX}~=3Fa3Fx4=6; zlw_4H9{KugUqRK(-$j`1?>p4O1;WiY3jAF(0%LUxm?iV8TLsS|nx#LafE_W+de!x5 zhpI~_$pRi@Xf~b<cLi-Ol!hA`>#wI3>q=@o3P`-_@JByAT_oJD-m9~Kg53|f&I!Ok z3mpYF#P;`!w+?~%!Zsa@Zn@_0Q;jq~t{ebOtb=Ui2z(rH;U7fvsH#3%XCzk;+B{&d zBOQNzc+wF&r!YxG;GenP(}XD$e#`o3OCh;hbT@TM*W({<-B3uO24AQSo9Yk;o%p2k z)3<Dp53~R2X?Hw`;M=`dumf(8pO|7Xv)(tJmj2jd%4wt2nhKlkI@U}K7GB*vAXhz% zr2E{;Fv#fZ2%Z2(W00L1&Gs@%B#K6Zq(;qR8JE9V^yN!Kp*pMM9l5Fp^puy|<GvZ< z+j#9MlybX1<83mS>^X_SiQ`tR0V%qr_8VoI#U0_gV_l!*_W=1$@2jt56d`6{E5Fm{ z7Fvbg1FDv$7yXOb5qkw{CtBTna@Er|-)qF#WZBiKHTL(bovaKzF{mNmDCpB?uE+gH z+MXW{G@EoiS#=OMnGTX*LEM8FNc-eie0(+z?x2a7eVa2kLG?5#^j->*i=Vz%S^yo| z{mABeu&#`=EBb>$$>$M0UpJkNUKxR4g>+5&+ISuJFLZIz(O8ED$4q*2L8&-N*t+Gc z+j0N*Z+?0Tk(3CVErMa+G-D?79G+IVcsyM#TFam}UF#fz013sLFp_0A$ort%V`FMT zQkQAS2T)0ENoMpvM09hnR=>5az`AbC#J*Ap94?HoS%$24(wQsgB5>;u>MS?FtAibS zmU{ExwdIn?kj=gWn5ew;Dp4}qA&md#?Vm_0l7q7UCf*|{to&;HzAsJ~)+620<{REQ zbjF9Y@1t;`4j}#l+UD>To7%{F4P*n*c<VqR#i0&B4RPt!P6XgjH*ddwn($IHlM|$) zZ;#`8X4n*QAJ?$l3_iME@;>hG-NEY^)aWcEgDkr{k`uq*E(+OSM3B!vwELoDZ~C<R z#6-h!J(*8IHu(y1fLz|bT@_W1;ga--T+Da7^e2*Q+$Y;W?u-o1i$nbv1n25(?3=+I z0@PRTdyi)92lp=xHh1{|N%)75+_*d4Wj~Bp;3&QUPhH*8R(O5-UU?z?JNqfDms(*^ zN*liUn!c~<@45%oJ)f*txv=XDM?<PXF*-#UxMmn9Y%7ecJ!r9l?cM&&gVG#k5^#Oq zyY88jy(64qd9xP({S|6PIeodm;y-{3A(?KXMmEAEwvhgfjM;ok)Lgl94sEQmFTG)Z z?JWU&^<W#UvpN*iqZe);%qC{uxd#~9JF5o7Fbter=73AGVKDdoXh~hadXdmo)afjA z6BxehV^ah<ip-gac=}qp<qg?Wg=Y49u%^_>X6WH7WcnUq(}Jx1v$WJ>bY^TF5L2P# z4Ixw04*PlQbw=#xrspG21up43s!QPw!M{q&-Cd!#;#aK1xd1saW@f&|J`1a*aZ+k( zY2fzl=rj)sf1!jc_l@*e98mEY+b4~Ji53Tau&a8F{{g%yY7P_oUGMN#shJ&FUyu`V zJRN=`r;T%YeEe`x=hq1zbkG|&K$wLlpvYj4hx>jrP!{V#TssG;`rJ-u$q+2sK`hOq zfRjtJg|1&^<|uhL9}Th00qpGM|0gRruiQarbghzNs=&M#+`X9DhTYmBso?w04&2Q> z*Cu_v9I|4%vCro=hgLh6SM{tyc>C;vOn!HQ|7P=sZg_KrUX|NEz=}@4#|8y%nk7QM z6^j2)Z~q4p?W27-cynFBdKY}g*_Be^Pf5}nz;Wo4P9C+Tp4Ad3bH>;F`2J-PP)3)J zk?shZnzf<2Y7=hqcw*HLWWTBXI_K;?s#d@DB<=N*D%n~^{nkTl!1#Id!P)sv(pB_{ zH?yW6P2V466Djkx^2ffue|3FelE;s4du&Zt5s;6bmu>CTuxoF0@LQj#e$@G^(APDs zkMo?j-5$Fw@LS_JvU=;9m^8&TzP1T6c{N|peEUnxdH+N3`AR06`yOF>R~93C7g2pT zM<xO@+VT>7LbO>Ce4}rm!fB;>Hf27Ak(az2SEGk7B6~hc74swOlQrpy^5N<=7X~^= z@retXg=4ePEvum<Q)k(hL@3WoI=Imioy=6eQZS-7C>^D4yRWItHP>FM2{+YdHs2gI z$E*ub58Cn+!t-a8#oBYILDDv!WaF4#yl8MnMtxO#WQ(VUy1``Fl2pNPTS3N{-MqAq zaHJC#M@v`Mwv(1`6g@e2_rsevq#yiK#Ce&0*j)4Lj_m~#t1pLcq0F!ovZJD)#8kr= z_hMuP+_A1eFlcxqzsxoJh(ALdAdXI5?qkN*MyLJ<qewMg=ALUFNSW7+^7Q!$yE@AD z9aTqcTi4QLpN|s{LOu6h^z9)s_kn*VY@WW6;7n8J(Zh=!C*_=xD!fO#^j$YBKgE6# zvsDj@os(_%Ye?(``zZ4>{Rw7FC&@qvU?3ZkU{Zx=E{sOaWX*g0FTMP->uZ8~Ju@hI z0nv2GNuL_FyfK(5TXIJ}Tn}1e<5l^5!aPvj_FX}MtGdk*(hR%Ga%{it2=xtXe~@$P ze3y7wD-5RW%Xm%N#CK>Y#g~I}UwFF%P4M%yz>tP_YNEZX6tm^SO~2}!00*bTbS8&$ zJh(#|tMF%gVL7N&nR1)r0_7M>bh_(>;#~6x+m-^)$LH<mGtqk$^KPRYXXNtysC+k) zo5@jL;WfBX82WofRfBdso~yuYzkP$amK(ZG(=pT2O!;YT5)`{aKTbZJetH28c1L-? zE6|Zm4fLw{K47FL_US3i2kRu*l+mv8c-Om{Hq?nm6f}P+%i(o#i&PplAy*LxXW`k5 z_N-5>(b(T+GC^L7U!AL1r?5v}`6=}FPlt)jR5fjv0KRlo^v~_DRr#LY7*`@f&9+xg zwX025DsZ~A6^iFFCL<V?xRFP@LP7#Y%B3FNh4Ku|7tr5AtCJByDzyPE(hkSt$r)N5 zdv#m_hSj2M{i(PSxCQCJSoM*LeH^&zaCvJba<fdqo-~+r(?q*Y@X8m#;i=+v>Z?wX zS?Nho33=@wt!vda44S$WoHk?@Y|~Fpc%678&%!Ee5WCthG?fg$fL37=zP9%y!;g?t zorJ~jJT|WwkNpY546AFo*WpJiW8GJNnlE$?{AVf~GR}B>+&$0Yp-%f3oVj1@5?ldt zZ{`zN-K1^$sr#nI@8{>Zn&wyG;_^{AT75Kk)P|u8u_{6`!XE28&~PR>ePsfgfaQcQ zf~6hi5rzgj#tpuHq+8YMA7il&7c5F%j^49!-C%$Dxw|K-EjO>67tD*^$6RsFT^+<N zs}=C(IrDRK%ojDKXTYAQ2szJ>dE@RJRTR$fj0w*dwFa-H+i-gD+56VDkNeVK6~5v? zF(DPsksW#^Dl#ybbh`ZZWhqiYdBL}8f22}vnB-vqTdB6Ip4q%@BpLI=xw$`3L^8i( zv2|HAztp+x<*uF7UG=VO{I`SK%^ognQ`F3~KDUQytFfCdUk3_J_G*@P(#@uv1zEoo zhTiUXvB9Ig(0)S%*tX}eX*rZg!LY4H^)KD=K!9$&)NUxHFU>p}Pv-1d*o7!D)g5cY zNK*}T+;@P>Me5uiHSO9tzPmn~c1S-9oWMK}9Vu^xm)I5U3UPi>b026R$8#I*dM5s( z(4Y8W$SU3>=c>P<jFM^hws7?zxk6i)qf-ZCemV@e^4|{afNOOGQEODYT)L!C9{FDr zv7Moz+x;%FwxsIIm%%mIJp*JRQ`E=q=QV(+PL!3oaF_OFBdiJ~K)dVOfy`5%*2}Gz zvkj%}!1bq~jNBW}kbrHe=>B4$M0S>0O2nA(ZXbIA)OFXDTzG?!T&u=y#KCB;1h5dz zR`$&9Y2B-{*hsXa12frTEumsyznW@2#9|8&1I$2&&l0~Ewk;VF)T(CRVAHmes#&e~ zL&xr)!Yp#rtmzp)58_?%LOI&Q(O~FzOjIonC|=lgv)c=+yF56Ua7fQ=bx@<_PVK0G zk!g=DslquC4-rc&Z7r(_2xyYX`VXlm;>lZj?I44Yg(#`ex+QiAm8x{rFLKlHXSu9Z zQ{&)hri^U}lC^M2>fQdLF9*DEH-&pjfS0s`>)f_W;}3uQEI&$K9*#;;vtxB3jFd{5 zl301fZ<AKB+1|i@VQaA<SqECf=hxrMP`YmDsqziR%wTeM{~Br+dqKlUQya$G6OX8T z<ESE8NzNBj0v9l+5_1Ecm1>$%(v{Pvf3fk}aCaIgCK+5w=HXA%94qhh-*jg%@Tt3( zrT-_o(;r&wZk&4Yzv(U(EN4V44rRlU8cOLe{4ZugR&sn{8%&hfzNAUD-tkL$Tezg{ z@8k`nqJ))e)vY^Ov?k3ezVtU8(tW!ia{-3FY}<&@au1z`zG_bR{<AWQOuIhad@;Xo z3tIk0=-@V=hMh?_)siIsOHoB5abYXpAvaTYSt|OE3nPsszo^90?EkCoLYqSuyPF=E z^8PQnqi25V&MF@JUyLr)1f)ztBNQ^6cJEJ+cl$4dZ9EKQNY+gm9xOlQu4qvGUo7IG zasD8c4w65*Fs!LK)|5Ga($%}CW{f|M{VWU9&YS$&Y+L^Alzxzn@(&M~{j38_a<2Oq z{Db;9MVq?HNb3Ks8?|e1w`b?i{6TZ-NKQ{&wr#kj{TK4^R%fnva$xFLQ7~Q8_atD% zg?*xB;l|*aEc<z0NBV4K{HhE3r7h<s7JQUnb?#pf(D)yjRoLuPZo?S;vl5Be^6bLk zzl#GFHKzQvzajV736P5_by<*q3)^zep#`&KO+zVCXQk06-4(I!*<Y~N=|8gG6R~C$ zCI)k#1tOdFy?)WZb1Q$fuwivd-P!U{pJk8++mUJa9~pR)ei=Wjc*<w7XXDnNQvG*c z`J$c$4q*1!SA^a5ZlCqYxZ~H4QvRK*`Rg=t2B~N_*>K<zG#$CTVvzk?6NEkgA<;!v zX5tA|C{J><@#~U%v$MZS)ZCWqZ~rr)$OZ9-s@eRntfm}Uki~zV+zWeGJ>xdgeq$Np z^nW->PUctqNnfti|B%_KiQj_koBw5)f=*ZtIdXEO^JKPJRCbKin*W7zDM;e~(s_+? z!~ejh|37s$Gx6360W#N;aw0iJgyL#37ZGK`wvy~y^tO<&Kt%WbAx8KkPb+Ng4G{m% zkNT?@{;lKn|M&}%vLItuUv2%(8?{vgr&bSdma@Arf_VKYBw-^?OUCkh(j++rV-J5E zd=WKeFDKEidO4ANa$Ko>{>4>NXm{|g+1zU-Ux}m2oOPQY6UjkotOtLzWD+agf9!ta zTeDeEc}voYZC!Jz@WqrU0<yG5d7H?2f|M}ddGtO|rYBP_29ErN0o|(U>N~sVOz20} zJb&TA=7&*ZGJ18cgPk~-!I>rQDRpyO#G?5gaHm(ggqk|u?8CUyKac5xBtgGF`hH{m zJPoO5esZU_Rh@<|B>&lRDah?o$`P|E+`Q+*RuF%k8?mkh+RQoV_x^5X9mpcoi`@3O zwR5@K9GZtn&iG-VqP61jSPHb54{FrOKd89^+%>_>TPrhi0~M_q_>H><G%BXObNPHv z)Ld;`K+;X%T-obZ2Q*2>+4Flx!GAj~$BZ>~@t54&mO*tmHG^9(J@R!f#cy7&s;l_Z z+x)Y={?evl6CfjR3!aO3y-&MMmR2^fYiD$B%PRxB^%I0W-Ic%h2zxI*-gMV&{o3*O zZnL#?&AD2N^|nn0-|7Zu&-I!0IISye9o+KMTk8$i#!EPe*Q}|msEL_`sEMyuU)}lD z&vh+GrFmS@V9jlyncH1{AE7)N2TBb-dwkc<M<-KKE-$pui#UCkk?`k7%emHwon^j| zuC;#AiQi_*M8(|hcAZsU4eQ&-{w;FRH!*9uooO1Ge4PP8Z1xM*gPo5gjro2N84(_? z-YBDJ+%YwFTp}uWhxxZ_xTgX6{h;%=&w=O#f8j(iY+=bEQy=^45>D(42zfm55v1+I ze1Fop54xa&F2?J@EA3yrGXC-1ix@gB+az34K}WN3V`SlGO!mwUeBH^oob47hp_1-^ zSlz7!eM*i7{}%E|ft&sS&@&z~FBZB~Zt2zk{bB+oJa#D`y4-JTP{KUy@@z}*p1=CF zZSWq)WCHAQhHjL4`vuFTY7zlZ7cT@NrpY`kNh8-=QgTGB-t{NtWhqf!?zRCBK&Eo) z&%3_%&qQm28Y!qJO#3BiUQ=@hFOxDCokSxZNXnQV0%yTl#jVP>oxp1szyRu}!Ip3} z-*i}HTyrVw%Dqo;?U~PTt<>ety{neWKXUP|7!(Uxv?Eg6?vCI86Z9q}wciMgxOOo7 zJhi|c8*p06kS`wp6BH&T<%M3c6+HL>fP|G}F4g?m>2b)W%kdHORewT?KXp9{GJ54$ zgQ~WQ%iRh8=D8t&p~|uPogW*2UE?=B7y2gzNJ<K4vthY@QwIPO0JwocF8OvKoc$+A z6eNC~I<dZYKNN&1(^LHk;~G-eB6xpC-Ux4coGr)d(_vaI38o5L{)F;?e8a57Z)@fh z%_6JD6spnoSl2<9CEH#KoB!_7$j4J?7OS2QWV6ahYT0!4-wmaMCV@0SDVYPjbv6I0 zM!5dTosy~<0fIECBe1Sb0jVZKJ^!}jOF5{?&N`&fC0``<3eCg(-7J4Fotx^9<BduX zihG(Y9qb>0G9X*QK(^kJL5T-BRu3{)!%F&JZq=~ABQS3P4v?A{6b=5(bhxKWfCdRm zAPQ30lm!&K;@i(i+9w_#kN{s(OjBJY)WqrHb(iUjNG#;eXS929YTXH7jrc{tSi1qv z9$CEM@1HZ==pG4U=>fJl7H{c&HyHl6<Po3}@+88nUh|@WV&cpU=aH2n=OU*EdXOyf zsg|4tj48`L$!)s-5Qy4R?O^yYiKCe8AXPowA(_X>S$+GpHTEupRg3UYoe{5Uycv|R zrMSq>{c8<a=-tKOPGrALbL$!aTpiFR6#*j8-@q&IfPeVoZuE7^VH<SFSMII9w>FX= zja;ijN>5K1sT6OQD)}O4UHEmVvelmo_8xL%meioX)OAvq8Cq<~vLT11$SkXo`%;}r z)qF6Cnu~bYt9Q(sEOqHiRVF3%NU|{_iK5d@z=S;fW6R?QjcvY_`ik^>lmgJxk^xqY z+MgcI+x7MSnBSW{@Mv!pP*F_)4gAf-fd8wH0KMV906+fEHtT*OCvxV4eWSXhdc+Z_ zjzcHbyKniDbBku;FQ{M8<;Fl!I(}gjKN*Hce`bP1B4F1TSnTU;qQdc2LT0$x>?~^H z6T<olC&*X2;QU$dFQ~$g;=nt*4PXl%Ez<S#h6T{lpa*|r*5HzLNq?{G`=>Qz{L?aJ z$bYA~X~@vawF43F|7K11hE!9R$*Zsr|DFD*A9qpg-EW{2_IIj0o5z=mZkT@6NDddy z?fShZFIA%sc8ByoKalzvW9C&$3>@JHpx{6E?i?_;0OXEVi6r`>dxOMse;OLRVcG@= zja0?>s4773{VgPBnN2dT@dz+|qWo_<<jm^-`m34shKOmtXA7v?QBx>sKrcdRg?sv} z(Ivq9s72m`G)-Y-k&f)-jKsx7f+8EZMkN@~bCZh?FKi5$KCuqW{rwx&DM=m@y<RQZ za%jl`?fXjF2a{KllRh3HfxRNm4@&I5*JlNU<T1(E!E+84Vl(e0ps)fMa=l!2!qO6k z#Md1<<m_F1V3Bh7%^;aqNHW;bF41e5$CgQ5b_81(@`x$Z)iuH3G+9sDyzj+)4+($l zxJ3kJW#frb4{v@`ZuAnVG|B26yTN-XHPHs{R?_*}F7wC&j?-2!|H$`L>{>~_s>JgW zy^Dqb^?mj8;}N!E@-)bOMQIT*<5_nJfz^vBI~x`?35yyAe2RL5`PRtLRs-_A_-1B; z?y=OFi-u9lb95{k>M^SExAeTt2B6I;%4(s~nzsLP9Zqu?-uCI6pXP8I`tGFsMz==r zsD;8t3x(Rn?}f*WRG9vj5wcP9Ri&+3;J>N?t&6|7aQrQ+8k7|_k_P&lQgY*O<;M3d zem8DNHNL@X)H)sbKvyq){1Z$bbRrID?D?Qm$qE@H?HwIut-Hn&lZrnRH|zh=bN2XL zopt?l>-vw0;9Gtp;$7O5wP_2)@prTqt9dj)JG91bK;KcPMC`c5E|n8*9p*YkgE44) zNb68mf)%R;S*is|Yc71-r<M$8MQn{<wPIoGOT&8e3byIPUYE4>@?QBT-YM^PK_@2Y z*n<wI`g*BTcV~ZVHCJDS`CMx(HzJc*wm9~MZ;0cw#QGBGyS4St(PKiI+$WYPZjA0K zUswM;1M)_z)Z(uwpXyHdOoh|Qc(Q1e#A>*OJ3I~?TDMT~jG(PMlt&gurh)16CQmyA zf|z`_SyqCsJ->9w95i^*vg!mkI--!6D4Ag68$LbLA%W-ep=a}<A&cJ)(49&?#fxNn z@-GfGE^>6?+h1A9Cmzf{m{2e%&sp)f%CD49JJ{fS6o8lze}<m5|C<+crnScKSH6`< zcEl)yH8)8I$$LVdtP@XA<H5z@>vFKd6YSEVmq`(Y233M~=a`ev+0bS%A9eaFb1hpL zqF{EZ*9p^h`Ol<RP`Vc+3UKwMM6T{8=%PjOx3<(jFOSHS#VxY@rPjqLA~fVK^lSrk zsv7!kjZk5uo`mh?%ob?#-6QdRVE!Wt^Mi}}tDz4D&&J<!;a~J>S{(S!`sd=o_`W0Y zx0Dn5M6eLcRtU`Hgfe}VWDr|<GaH_>DmA3rcXcNOUg6GAkf34+FG7cI5Y53U_!%=x z^94iD=KJPoFd2Y*8q@lLFSN#vex8G9(5MsAlqsKTsFG@U1HJfdtOBO%U0TRSARb?E zvKTrwPSKoi7Gvkmi`)V*rC>!a%5z;Y^j+HGU(Z-(h)?1hYB>Ns85H%=G@Y>d@5E~H zr71}Kt@O8r@YCxSGr^<)a*grXGCd~e-VC%&sGMZDl27+`i5dr%b@5Lr|I|;K@zdWm zj@A<Uuo&)0{vYgw#3o2Az~Uz}7bFIwxN~9=f*`4{!fNuA>HvYk=ijC5%>O_8$7S3^ zyEDRNEmb{-eL`bdjl`M%%ogSUyL|mW(*V;&V^uY{#ydhR7?+_4W|EYu)OYR-fLZ6b z%zyaG9vd|)#ty<hiOc?bz}=}L*=>lhk)G`}!kSI@)?=gieLmlObJr-#$~40I`qSAe zNv#v`nFH=ljr^$jiI0LD@r)Jf^tz^i>o;${+-~jplxD`^kAvKD$wwnyV$(*yKk+f< zO7Cu^*@mO{zyINfb0_m@hDR0@w3O7;g45PQ-u273dv@;Jnqp--kp)b@<g*~YS*Q~q z0paA+_tt;&jbmVT)Kt-|@uJ{gj*&$BRe>6-A}?-vniDZamm6FK$kk;h$wX%6+*)kK z|E@&pk?qP)*-DW<5A}Bd7JM^(70YJtTzmi`FPYA)AkdFYj&G{U4?p~%@h#14>Qto0 z;>xLcA@QS_q3GpiQ%P}eu7QCn4~)}lW+9*H0KuPs6;5N7mj@W$z?H<Dm`{CQTwHNr zWC3~8V?s)5PTZ$oPXGQFE3=A5+R?>dD)vbH5t|L0zWK)B?c#X}5Vw5z-Q$aAet^&a zMgmJ%o%j*-X}K7Gf<63PiC3GQcn?sCIr?dNUba|pW|a?pWAJG9eL|E-KoK*|hz+c{ z!TvbUA2d`x3pd1YQM7NrG#FGe3lR$Rs>k9oX6aG$B01_!h$Wz6fy#<Yi$qq5@;9`V zoYB>#OvF$=9@$Y~4fD0BOANf3c<<%6e{7T~mUr4PGRTe^g;np7o+D6Zp4Rne8kVmW zbnnwJ2sb#rVUsNIN*6l#Oz~4vj)BD$^`O+Jl<62A+_-)3I6G=0Lyuwe;zgB9yR7n| zyNCWvR70Zs=Gn(*^iub+%A@9rqA69U*UJ~P57PEbiRNeRlb37mH@~v;>L0lqB1}?# zy3cV?n_+#VX5YHGs}vzp@AehjolS#QDSeZBR=J)^UVdcdZ~X5eA$dL^eL0G>=wUAE z38Yzl>hkD@D|X6g1KS&nd%Hpm8lXR(_)+5%76b-ZQ<}V-7dfQ1s@-t~Zr7Av)EGH# zu#QiNnx2Z`UO;OXKe3Rva<u9pSW{&P_cctq{R&$92cVOFjMTlt0<jj}(r6i!bSbdM z6hB(!8_12Rq<zH_w`sa!@d-CL4uGd-GvLzn&bge~54@9Madp}eT#n!|+m?ncGD0g+ zlM~(HY`oYgwaSg@+9~L2qtM4OVG#O^H?hyhMaWY?VETdw`dfpa(6M!n+;LQUk43mE zdCUk;3G2k?a99lJ8GiMK&zz8wiqG8@BDBfPAevuEsT%Q5jR?#4jGSXfdOdVco6dEc zy&hNgc0X~};l|*D8FncH@7X90VBtL<XQt4QjDu}_0gFR~t<ta|z_j2kXZlnLCQ4C_ zK1^*vIo2~Tc}3v-1b9W8JRs&znpX$Id&;>62c3qAwkHr3*<ou7&Y`L6-p0q2N~0Cx zIbC&dQN(a{acgk<jJNMln09$nF}!$UGSKAIoG>`U_tvi#&tFX$2$))#g7b!^omMH0 z0T_>A=?jjtovH8@d5~bW+V5SngG%ftIm%RbvS_fn$dgYIjIx}ODL?0JD&1F}Fr=~C z?$E{Ie*3bC)O;>OSdMK);Y|p}b73oL_EIMjg;a*>bXQv?C#0GEn)-3uHq}jD_&6*C zPe3us;5{i}3VD9){kD-!L2Hv34)gHB{TN}&%|FstB%$xkEBsQs)Z%@%{sr@{7tm_W zp8Lp4ZAfmZox>~!Jf*L*i9aoDj3l4Cn~$VGkek^7j4&ZB$X?5@!p%p4%P%R93ge8x zY4`K5xbvC<%QZO{`MQWsyG`=p+-USK3gLc?_}35wZz`kAQZtlNMNY)Jh(-Raa@4_K zwbtrlMi+&ElWs>1S5F)Is$K)7?<$n$!WpT-d;1}gjLLA)!=6V)1lQV9Hjx{9g^WrQ zA{s>IpVzab5Kv-okJXk={N|G2+|@@M-^r@V4))tUc5fwR`e)rzr=AAK5py5FuM@lD z;nY#lHsYvSjF{_YI(L>h!<wX#gJgIt)h7atlca)PUET8dZXucP29H9J!aZBMHK80| z+0-GYFwy<Q7v;<Vo74}=JT$>ROmxp$*i3(jRH^Q(9^L#Q?0PdZAf@|8YALEm4o0S9 zt+B{rg5Jt~(0WjRDp6FDBF!<atxub&bh-3duSi{n?p?(cpY!0PCH9&gxu*uP{pJ_@ z^}hKJPN%-8Ge6;|z1_Jukru@$6PX?@4ri3rPH0#e+1>N~u=xOc&*?!^CDpX@0gjV$ zg$s4W38qYdj_5ZB2`8T>dxkZL2DvH58v7U;SZC}74L4a;wk>}{CseeP?$}w59pCT@ z%4uN}3zEB3Gvc{Ih%i||J^n|+?!pT<Jy8!*!KwXdTX*}^beiq@Oed-Bu9Z*b3=r6J z4d<QrYOb+or1l84_@Spz#bgn8AJVq$xE{D+BF}zgd-hlfnu(X;y(OXObH7ZO-yy#_ zrx;D%+4UQNd4*=0^`s(*a|3QFF0QjXzOfjU#Zi=l2Kl!2+j(P{V>u1|fg{p4$tZnK z@tZ#_F?bt>HB32qA}dJE;vA{*NQ&B$lY498dI&9*1Ro51#X}UJE|Q3-Po++b5(+Zy z8{m)WyV0Gy9q)}~Z7xAqy5%bDu08AI*N?}Ic=9b%$SBN_YS%-E&q~3R0`GVCCEK_X z`uI>rBGpm>283ArYM}Z9Ig&g$)O{l+<`3)2bqBZO45kp7N=Q`;=LXi1x|y!8_gEYy zd#-Z~PBUE7RV*6B+vRO~!;d7jsL>N?o+HwMHc5l!%)o9vVoNEmL69h_vftk%5Kam| zvnpb37$dDgFnJq``9w$$J(Q9!g4vf{G7L5zG|hB6l>*Gjq8<G#T7Z&P$5+8FZw^?T zuEn)*&9}sb!H!WAUpvTNV0lJ2h}%hSybVO6r6zGS-;)xdVZ+l)jmQe4U1Sa91_}4x zJK3p8Y$gk=9P`i;mEJ&iZ#1O}PIYE45CNOcEMqb&A2G4<!h=USp@cK1%d(*<;u{<V zh8|JoQKsYRqYpk6q>z%5w>a@}ba#w>^v}25Y+54Q69;plZ&Yu4Hk>bveZniN9f*`8 zx%nZOOpD@h?-ToSED)@bq{EFqq(l)kCuk`0LzahM<H6sP(kaR-)YE}Pj+jOl5(uMk z(WBcx*|wUTvrLG?bVoy}4c9)KVo>(WdP9|{;aywuS;x&waUXF-$CDdGib>4LQ5)?p zSf@t-C9Vfi(bP&^Y2|`cS?pX5!yVtA;8{`KSd2ThE=iON<{u<N9YbJT1A0QkmEErH z<z$yJEls0=L<lB9oo4yRQlZBUN<2p@T2T7<Ov^v%T-TH8scp(XL<<XWt4^s$csl;l z^0j?J`0+4UC*#q%Cp5qY&xHC>J&4zEbu;~(Zx04ciV}$C=KE&q8qbQ(0b&;2tGdL0 zq^Bo*o$hD=(cHv~dzGpDoD+Ms*0VmI(;aQoQn%;31uNV8rs}gRUp+pPBr|ueKc?&^ z_+(mu)WF?JrlDfs^)@HSO8M*Uk!Y(iZ|~B=MwienrKNAIRAP>hH@wkmNPLaw0wx~* zJ>A`T`3ZWdYFC<1w>Q4)GR2}s(T6qlAlKVnfC1x-rkowuO)DFPfcd8SPnU}1C6Y8h zq**OYVfc)&wFM5OY@gAL(tblUqujM~1FHatcEu)a4)E0JIT0WG<237cf=|F){<C{Z zx>CJ`3(9oNJLjxp_DO8w^#`tH5Pjq-u<)RsKcmnJDr+u5g>o7?L98JkZTzUEDcUg5 zSp#SFcvHyBU|)+Z^vO_eAZ22boxC9KeW<!oH1=uoSx-MJAd-F-%*gj)j|z=)pt<&G zO#|Y(_(u@GM$_cGQ>LI6ZA}+>4(B$XuLryTAwp}*Hy~UfrBG%*W=snerTi5oURmBI zEM7V0I2lLs<eS<aZ1Rjhks4dqcwD4m1O%v5v=)u2M2zdoIwOZ(v9(jX<DnMODcyA_ zSCLxoB!0IW+Vw8bN*HG6c$kpN)*IA)$;DV#P~i=Xa85I$zRU-tLR3zKXuP#>eGuWx zk}s3-+_m5;7j|Y`xkp_c117a_-|Z_4@j5TE?CXQ&FWaqQ>zb@z7h7AUUY<lXxikn6 zjBJT-$0dV=h4h)sXsYGjAMN<M7^i1l>`<#_e^k)QmsMo6+TAu&KZ56L#IR|8YZ)oq z%`_PMQqL1(nQeP<Br>{cWwb91%ql8PLRunC2SF2@GI;BTJ!gPNtm}-{21qb)SKFIp zmmdLfksXE-DemV*sn1Vdg-TE#``Q31J2a6hRg$$}-QVOT4`@3M=lhaI{IUC=z^6Ti zM=qAf*&dW5S@~JHY}ZF$Xh`hW6u;B-RZD6R*w@{*Btn2--N&BrqjwzSCUvpEh4pKq z$OCx$VSyjP{;9U6o=Fu-EwriC5U_KpGu9vZGV*c%&Kx=fEa_&6lC7@?*Fpx_^CuxZ z{8(FJ{b|p_*|3*j#ut=IiJc%9*&kCPSu2I#fhdtQEmV>}yohul%^ZiftIESIHdD%e z&<zg8rK+uM<C8RS>9G3+4;*-=3%1>}!q?Lo^BAPh1Qf(>#sm-nV-LF}MKGa^Csui> z<4*j<GIIa7gh0BF8^KcxK0+J}0O`jxMPk7=VD<a>z_X#jeT|~Q&CXvC91$NqN-5Q` z8Jc-Ca#(iJX!;DQU6HI8RhlHU9%Nni-?JE#Ny4Y&)Pb<zl<=|$yZkZ-?8`GRs?^rm zK0c5}J=F=4a6GnbDX2=Oo;h8lh~2y#x5;1<5@-a@YX*oORjUva>hKp>=Q<08hPGoG zLTD~>=qYnB-6xC8_fXG3%QEd|K=o^`FQ@4NtVfgX$k2&6HJvVON4@mf@0p=P<c5mC zHH{KBGjq`eU#DT+Oqd_bsVE)%Nn&?0>XVvsj~nI(%DG*UNXt2w=>i!FChDta_kq0w zkbH<R@cfP<v~f&#F|I|>c$6Diu25dB7I?C!1-Ou?@ar$k8S!fQZNY<)a;n;chU0+r z9wHg)_!y!aJoy|ChZ1a`YS7!txaC-}GpE_w)uKm_PS-Bqh88`}b>%PvAJ;c+IEMfW zY%lg{D_qg5Vzy)K;>)nWo)qBELeEyHPgkVil>#Vs!<4m}juF2J$x#hmHHng10x}uA zB(Q?9Br-U><^nl3>%KZ#E^qaDsAoy3uWC;Dx%)n8%!8g<7u2|KDu^r~!lCm(ARtIY zcz6SQKmKI&&&Z<`cs{&EXWVl*bZD$hJt%WzU!~_ExbHM7+e)<rRDK6h7>2*yvS-NS z1~oxJDY{UP?$W?eW!!#<=?hU%jb4{iyF>KjqEW8dPL7=GBOI(=E>R6HyZH2J<hSww zdXr#vNVDiIZ|dHtE`b}3gY*$++vi=jU60FNw?42@rXD3_LtVBb3#ZP^lNBkOS-^Gz z@ihqz2W~_uw|QV(fsuFEhDM%zil^rz_r8PY2eo6C5jG#;;QOjssi@l*xzhvvYn6~8 zBSW8Q9>F!M*A8+zniw)Jh+<MAfiOavcN(S)a*B3U{J7xQI6S21%=U8QG@nWYxvX-4 z_0wk|ZrqTsZL|5-WBD`l5RCPDISXeuF%M5wBD6Zb@$L+p4W>X9ql)syd`Xm#<30?& zQP*b}h*5r{NKgp0-D1)P#VT`y<l;LaDmKuIjR65aEcW;vz&3y`Uvo>zFb%9D$u=Mm z!IlHnMgX!ZISQhE@Y%Y-?9%>?pAZgUIWI&=cA4Dc7M}e<V^`#RcL@^m8GM-<1t&^s zw(nha8?YU6eD>8-3H|}(cmd8C>yZU#a&@gGK+QMAJ5%?sVt~H+49E17bb#LU`C0Pp z&VU_bhriMRX8bcQQ-_2}wCeE_Kb}g75|N|$Gjgn{x|4Gp%Dgzvvpv0Ue6C)G*;C<? zO6$fRQltrZ#MqlJJ735sG%ak~Eer>?os|Vlt;54|^z!*hO{J*QOibdwB5|`@1Olp{ zZB=$Ach{arO$!Z}UkR{J%t4VCz#p$^Ci}dY0O8$l2$C_A3rjGe_tX0a!lG&6<$q1+ zc`m?3mPp4K+kna*Ocn|pJe#$5ogc7@LO`<u93EfR>aGV}G&YI14~#+Pv#B>Tq5Sb; zwETAqxKIix4Ve%b2qj1VYziGOjrJ6m(mu*miwfj8U3wJ3N3>ph1}!}0ulqiJ!XGI? z5{6=)^+t&blza>4@z%70M(Gq%z|h047JPGxg`rRX$B@dK8yaQ3M(Mr9=;hJ<V~KSS z<n1;z^^?u75}AjsbBbyOc88)uL>;iH(gZiy&ZeNrY~jBC=yhJe^99Qt$j@D!I9>~w zn8Me&smlx8U#-@*UE0J5&mSBz41ZH9yXD}hlfj1#I>zxD53U8=z=(8TzG-!li!PVx z!n?h3G#M;ObjO}Sj$E{PA%ehCnf0c%38>i_HJZSdABxrfYw2=vI^l1><X$S4|1X~Q zHTBTEhm4mcy@B+`6X-Nl{2@n`K*~xad6$0oB(%}4EPepy$qQ+ry)azk4qT$LU+j;} zLR1Jo#r0U)7{tvpUU&nS@OMj~KVN6J=Qro$1D?st`dMI`cCb?pCmh0Pf*z)s{^&Xn z7uS#qt~r1&Q^@@W=zMFpthM{m8R=@nsE<8vJq{{LfevRzQdU3Ol`!Aed5s-{&=>uk z*jE?@P1p7rI)z=ok)O|fyy5JeaBstv!`LP*UE7bpzzy==?;z<2wn5&N>km0N?+cQ) z=TnrDg4Nhj2+YiO%jVWZcdm$-<CEJS`ks@{yTF)a5WPMKi)~^vlnGHRz=o8u08FFA zlZfynRvCY69bh};5-5Qreuv4xSit~m)QtuU@nRY<<yuVtn)Q2EowC@n(J60}ahL2W z97IcP@0lG?`_qr`<V2X8w{mnUJoi>u?7rjDw9EsRMI=97if;T#INjd}x%rVdt3A>O zU~|Omz?1_~{6`eRMT&uqsK7O&n^&$5&@6+Ko%XdTW8;-RO44gor_)Efj$w%L<*9uf zU9}hO<BFzr9}PT*g9@p)+sB)r1Zz>5Uq<E0+ScIK@dl8XX%GRwK|nB=`Le33@cjn~ z`6hlncpj&#-H}#Kkl~aDb8EwzCY^@xZHCcUaNE{LLN%W!m>HbW7If5#EPLvMAZvL# zli5xT7E>lb#0dO_Q9nvihj&lJ=i;M$j2^`b`XY;^SqTL4e0!M%B?nGzr8QRl@UbI+ zWGxD83=1H2U2LC26*g6<*NR#36c<M6<G-71eG~G7-eSJxmL0*lctqjNL~494@+fX! z%S7W5MnDs0LwJLAo5TJ5%@4wkq?b04iyVLGOz9#p8c)=EGPF93vco!qV$0<N3gL_) za}BroB71LnIeXi*E_MY+fdu^kKb9AN-|nT{oUoMJvpdw7aHT7mZPZ=tl0O*sqP#R# zL37%*LVbR09#jpQ4L5lNamz!_#>I^R+e#mAVyHS=GEz~K@Z8&>HY^`+qRWx$ZW=c& z+mOkQ)XYfl9xc_ft`G4W?|`-A2LM<@U<bER*wx(v(}R{wLW&`Sl3>EK{cGj>o$^z$ zHB5bbF`}^58e@HS*wa@okXU-|wYT@hd^Civ)nILl{nr^r`rus;e2<;Th>koS=J{xE zr`@UT8unNO&pLylZFlbiM&uyJ7~OY#(6FxEhEc^8Y`2Ijb;->^t<K~I$<7f#_?uyz z&0z^c6}(AdP<IB@@+3lSAiNLXTIOikEl8hWLW9Xs17pc6h+$0;w5)U&XkNg32Nt+i z!;D@wZH)A6#M836N8!+23FC0lL$um3TqNJjeH=xj-6}ont63|IpR54+0^9z|J_;>T z<@S-!I$$@wAsKT!2ED>_qr)^(Ak{8jPn2oHt*XzSSP0g|m<f2~2Ufgkg+w)D!D{R2 z?tMWNJL!DS;puW}(@+^XXf(;#b)P`L9MKh7#Qh}-%Wd4y-I1#qO+BNP+k>4%nxOFs z94+r2$5&m!Dg@>Da%M37piDx-c8hYEVTZ;5dlNs?Q{jcdneCIU^Fx6}vD|udgfyOh zJ|tE;fgO<*k5c+&V9Yoo7C+D6IMHy(su7!b?*^29uu7HM>h|gjLh40R2u#i=w9r@1 z8HyDai!*sAr?(LkRLi-wa5#E(%U~EsE)OYRuCrnt-9?l~h^ntUS?yO&X4HjaElJ!8 zhke8zhAX((g2inHWs{X_BMK+e*Co*4yoB0;i(O1Y^71~gyYOc(Vie8H-4cEzS*_#D zs&foXMVT6|=+fQmY48;>=BXuk?$xe3V+-dkHS13LArX|dh#Y=*nQvJU<e)MUl<~on z{m55bQxB*TEpy}VmIkpgW&5C>&CrB$_=*g0ykyP=W=MrUj_rAb*A<565|mU!BmBaf zeR>>k`ZRb7Pa-l@9b4p7SPA=}Xlygvu@f&d9@G;oKTsJC)k@&Pp%)!0sNyW@Ivofe zj|_1}NVoE#wEVhbrjvMKWrSN{{dNHMsSTORg(r-X>r&wTCbUbAAIK^eDui%Gb-}=# zoKd`vV{1WdoWR`Ao}oF-A{0GTu0I}5PWhD~RpeGyl0e~+XCo_#uOblBsu^b7`QCQz zNwMp$gcKW^%*&u(eKMidmeXMy!yWxP?Mm_|8l#Dp7O6@2BGDcGiEIy_c$8fOw}tx> zwcqYpgQzt?klq^X$JlC<m|MH|1$tk0E5{|l?Z`a<=($~0`$3R+J++jL=asgrei+bJ zCwAZG@Z#=T#<#f|=q9k?M=Da8lV<{msHFNpPiGGh8I!F&Mq?V2Q4TPI91}L;i*yX3 zmO3<yRd7-64y%jUdJzdDMI^#pH<Flx-wbwp*zKaH`&j)8|K|Jf#P@|=Cqp%<T*Yv1 zs=1#KPY38&2~u8|Sbx5H6w_VYH+Dd6-y~C0yLK>o@VxRN{n$mX6`?tef{uj8_^MDy z9muzdEV#xyQnY-sv0ZR;ZNvA!dY#>720OYk3iHpJn*`^Ye-oYTT&Lm`s7(@1)jpwC z;w5N<Y;lzNeYkl5X?$AH2(h_+I#9*p3Td^PO%lzA?qk;gMHXJusV_Zfcc+n?Z5`Yg z(BBa()b1{JUGni&5UiAtNF4z>N0t1>K_v#%q-yB*H(&DD)8X3oa57&&E7iw?$UVHt ziT*KYyHWeN<#mI0+^k`6tXDwy#AFWy_LM_58|b&Mn|!)I)L4KIZx{$pP%u&_>zE9M zaFvDPj#7Fe$eNF=pasJbiild;_g`>9C@iK}3{nS#KZyLC_@w6T)!_sBk1eO?R2fP2 zw^3mhU$DZEe(q#3()WSx?!KY0S&thW>;N}<7l9<44f8BklTpVQQ*AxHzkxdkWZA5? zvXCdK+a#2CkM0FHJDJ8IaJCs->LJs0(Vcq4qLjVL`R1twd|#x&x&SkPIPDFucN(oP z6|f^Y>gX558AR>(?Hw007zW*65nJ%UMl-SYiqs(X94oaO85$nXXiV<kNo&%>vr^3w zq(A_+I9{ZOG}XT+AH-DAK5lF3{}~y9AHc731Hs7eC*F_2N@S<?&C4D@I2%r9y+4TW z(@>!>3j0XM=5lKG*;KO|CI?E(o>yTlZ!UVOcN2T(8dklA?)K?=KH7Tr$5UQPqA7LT z^z#gDNR%?#WEZ&Gi;aG9KXsqN$-Q0j8B9kePypX#r$gKTSXR9*P@W`G^da_;%>R`F z5t@qB6Yd(1d!};g@J30ZGKmCqx1(HhB4iracNvwm(i}z#r+6f6HdS7REKK(x7ICju zCHpfUX&axqfsQO1;FQy<iw4H!n5i$yP_*Q9PD{U$WG_!Gl!D3`FKwvLDeDQ17~rHe zJaCQYxdxE<4hq$ZP~{<G#c}C7znLGVYhDDxkS$HI8iZTyrflpfM<JbMTZp!N(twLa zb`dHqAI3pP^27ZR8b?p92a(X!d}sJodmsFSDdK@+f-c>x_ipskw;}G9G~e?PuerRi zpMN@ZQ1;04rHtig4_~`_J@Y5sl9J3Pelfeg`|0avTF=wY?QD9n-l8aWX`arTR~u#g zX6QW@p}cXjPut@zQ6`Rp%Kl1m6lqoDRPNpO*pOr=J_VPAoUrG2dP5C%gl)QXTu%Fb zlKWD+i$TWu!E=+N=T58a+W7v&^pl687#60GbK4h5h`P}K*ah>o)t!&ZyX9ri<>$d2 zvm=eiC$&bGp^>h#qz`DV_HFIXoeD7(&VJ>my2z@Grqbq0WWo}Wc3G37K{hg;c8C5` z7gqW}TqSaFy4KXV%d|+K1=-B8|BeoCgEq_%6K1_#s-NyZWID<=zj1<7yEXp0GiRu} zbgs|_O{=P8k_RTgqh8`?W|_RNoDn?t-HH|K{%p7=g0CJ|_#Mh5l^jf=+%kQC$H?RO zk3S3qORuWVs><nS&m1G*v#73JGt&m6Of)CZk-anoOK(<qUtJ|LIutaqd4lN(P#^Hg z{^sx%w&&NdgP6=))~B+wFWPDD#vxT=thX{yN86D{@ZtEVRhW?{x=D9dZ6!ayHZc-+ z<G{uo>C8**LBaLk*882q^jXB|Dn>gJyVkX%V&oZ+7B0>BEt(WyOk<vHZ-Clft#ffk z+(mW3qk<ql=kaSi+M<_~XNRpTT^rFaMW9ua+nmgnK?#lc@hKM*7LravchC8yVZ;qJ z?85HR>1V+g)wKM($lNC@Wl37T$A28yeggmoe)V^H84cm<B8#KWP0M1aZca?A0HP@+ zQ`8_<n7=%tJT%OVF|H3)D#ZRQY>_|;J9qM9w{0-qz5AKhMz>YskD3<>R=MS>+otB6 zv<}n`*L6&mdU3i}ePgKa8fpnswX6pxmQztnyLEKZ0@h_5eznc^Ez%%9^ZiKE7)0SK zd<xabAox4#bF{pV{2It9(m~cvQ8JY%#<-9ih6?3rz8v%Hq@bSSB0@)bOnp%B?03sh z0^@#qb4z=F8dk)*j_)84oP<1OPP}nsw-93w25wKF=`9bg*9mCix?($v{fz0~s<IrO zvN~OEmg3<^irEF!s~yN^ckbPM$=$Xg!YI_Oh9*c*W3&uvs+%zaCBuLFQfxNAzmSA{ zYl0lTD)_@p576CJF2B(39@>BCMjKr<;6c#A6V0po+RlALt-GclU%eB<@t@w@nc6uQ z<W;66uGeE+q}YkWV^%%+S$yoLr=#k2rKz0?SLKiOWRM-aLa?QE0buBp12B6}DRfYv zNQV?$x*+<j*e#t{aE<IFRkky_XVNC#^G1Y?GdZ}Gv!h;@UO*iVZ57Cg4nEZ!rOR@N zHI;9-q?&z4!_5}9(;_jULFDjI>g4ohZUkxH)v%s&Ulm51xEgw=JU(=;#ne!#Wkk5U zK*n(3)HVR=5ETwzLG0$O;S&SZ<f8VcX%n|LuzP0bN0lls4F$gcMQ2Gt#Ap)*igY`K zo2&Th!zlYPgrFnvoICquM@~Lcn83hNy-);Pf)38L-O9v;RXST-Zgni<m>UGnX)IN$ zb&SF`{v3a?`witoLXXxZtLf-Rod?-cmD6uhI_Z_Xps3+UnB|LEGZ8zMkezfazTafn zI8gCt2$xz&8}voifr%se%xDL-2I|e9GUGj&-YfWG`JFpnxTfNq_VtZWqm8<m=T9;m zRlE|tbREwOK6mND>kw_|w;+)VEAg1uJMYO!<LV>B%9it<?_%qHIutmS5lHFL*dDZ^ zk6Qg;R(VAC1zNH6t<h6o<1~>%Pv!bRWQ13ln@`1ra+f(#+&RW;qz4wpQ$`KXB+xrn zE`z$1Or}<wBqH?mH##ux)*}d9i6|0$bLx}n9@U-?jD~c8C}zGeZ1=uXKG%}XwNYT< zw5@Xg!2jHO@oF<taQ3t@Dc$H7ugLFmf?(siRmYqff(mfNWRI>!vzhzdIbC(OJ=0Cq zAjLZ{=cz$Oi|=@o)(NF?nJhzG&IfF#mVqh{?j=;jX&-}|GAQ<gCPCzRs9ZvwbxOZ? zW>P298<J%*Cp6p*^Xe`hX62)yp50%urxdd`_AtinM_rwR*-1I(yetW*4bEE0YVE4f z+)X;WwYYDfZFKqCWk0&~+})^;d$n9>GHUrMh*R3MxpTC0j@w8$)`KfD@(L)-$mFQ( z=J0#8UiQB#LIxFHBzKnfK372_+xZ2R*q4(d?ae1fdQgrHW8Dflm;!OH8>>EirFy@& zC`$nM6~LvA1-1GNRh$}gxRlhS7MO=f7zo|$IJCua-j9JT)(fmEcdshCOx;^QJ|b+i zjL$dp;_&b|M1&`$smoSrD86@hZA?!&C?>_~;^kbE==L#i;Wz>{_F^mL!0hZeE$Xf( z0PN#-JS8OMxQ!E&ggaDWITi)5VSt`!j*rin7t0}^9cdetIU7AfnvF$#bS3I&rf6pp z^aEg~1AAU}U?Ms34UVA*9g^^U<E8PtI>;H}me5(wWJSX4va*yy;fMioJFZqouolKX z$3FG4qTOgsP??gBUtko;C>RDUNJpK`QNjdzv=2RM4Tk~C3O=DnrB{u1o_yI-ZFqGs zSmk|3i~gG65_Md|g?3ECuGA0b#CJW5UxvX%$fNDDZh_DCG2ZQR(bIB7^WqOFQvArO zqL#4!5eNTgcc@0h7PYj%BkeA%ZTLjQt&SdaihMXsCBAdZ4Jp}eOyZ5qyJCH#x!PD& zJi7lT=egk$mPvOS`9%UFhJ+%0$MKp6syNpOVT3KFH<TK1*lmU|nzN~|EL)|v+el{g z@d&z;XS8x@y&SB59@|0-GwpuE%ZW984~{K{Zd<USyWg?<B5t&hElepb&w_BYyt_FG zKIHtsh~^9S$gPFus0%s_g?d^vgpfhl#wnV_W-xC09`2$0?xx}p{RKZd@_aSm`fKFd zQJL3=Dy!xA+v>ZRX#!Gp7x9r{qVQl(>1{QetFUtNhpVF<&BNsAD&BCsGFvB@M=-F{ zUuVtJ-%X6`8;(p8=GA5e20XGNX&H3MlP>D6SshUL(O7~0NXv^f=g{+(&8+aT8dhLC z40C7B{&7AOjWW8kRv;bxVBg0kd0gDm-S>RR_iQ?siD}I|xv8`ph`!%EA7|OrJcCa{ z(c4HqDwSp2CqY~G+|N8eQrGdC-;!18jIIw)&`s^hP79LUC0Lrxwn8*N*5dDtTM z0$#aT#Ut9d*d+v=MC~cf!OYmBwuM4q^%SSwS-6~_c^J2U+46RYS%7q}y;k=Gp=(E+ z$3?qk9Sa(tYMHs=>Vn0saa2_JzleL!fF`r8ZFp415k*lM2aqP}fC^HT79c7r>L6H# zCQS?wr9(_Y3pf_Ak-#8L2+=`BS^`7}5C|edsEJAqA)tn$sk9_Q2zhU2&KaHaobUPG zfA9X`FM+++UVF9cTI+^5FY9Cq%cdAsLhCGd)L=)VXKHS}a_sGMEzPA6Z)(ea*>cCH z8=HVAE~WWT&wt(D{yYM<ie;%-CV<C+iTk=te)VQ~RT*ZxlMzT<>YR0Bh@)g7egNsy zXo~wB>jIb)M*gqv_7XcEullNV5q?s|icTy+x{oru1$M!?!;=D!*MeG~Ivf)~g}KA8 z>wd>OL6L@RUVeAZMC?EhK`B;yWG+8o!86T>eV6}fxV!QkbEH3S`9q^5cvB8_4~;g4 z=Y{g@v7?|B_y+)h%rBjY4A=9l??~$(KRN!IeYW<NKjlN{&eX7nd=-fuBIBg1^S+Hj z)8=*r(OJwq>jhCww$?bJ9zEtxt?j(iEvW6ED7{m^595c-XFw&|AtRRBL|XW{Bxqxt zAu(}Ku7qtZjC$2EtlNdnWLqPsw`a?fea8G4bPt_?fY}(a`#f+t8kg<Bv%(A1U{vx3 zq~uQEc^ApWYXgW$Z|-WUEy-*Y!NEiJ@;}oo2O7=w>P2p-`dvC%SEqW@n7(#~o*h3N zFGG@yOUfYq#_;Y)xjUpBJz)^GG(LYg1vp~M#S_!~al5nmx6+gAT-c4_$fmbF=gU?f zPoLXTN^dO7>ndHkpEw_z&Hp6WSWoe)ww$FT44?(}(&s`1XVSDI%Jw1Xl9$`#S6Z<V zz18N?atKoBIbmvTqwUU{jH|aFPdILwY*VK^^C`bODlo5p=9NWHr)#WH_oTar#Bhhz zxYg|j^wmK0<k5O|Z*`#o>7nuM;*<|<ctKI`x7I1K*GUlY$L);6{e>s@Ifs@X@rS=l z&p1G=iC%qxrU&&1w<yk(74w6<1i*DDx~-)SO@W%V#3I2sL|;FZ{q~9O4X5;s`1HWq z1{jnV4agzY?=0+Bq&pbHaMI?6M)YSx_4Znu);-+A2O|GKdMC;@7|9#%JzYW-BzTCD zEUoQ95#)L($|~TyN_<nWJIl?-m>J;N$=CFWCof}tG^}f&N$KG61vN4ZI}tSX(?lE* zLNd@<+h*HW5=I+hd)zLC!2>0c;{$Q@`Cr=BN`BUs`96pEOJY#(79TkP4$t3Y_ZP%$ z;tm8(xL3G_ENx0iJ}onqm^k7bsW#Q4{Zr>^RxZSeQa^QP$!V;-g~F`EjYCCvm-^V> zSn=yd2dC|&PI5P@rB9h47Rd^8*l*4?=(N<KaVW<|BM3(_me}oB*gbYI(r#3W;CHGe z7Q>>a9v-SAc&9MYrSTth={NB5p;3I34MVTy+fXN6^3&Gto9rC560?{C<{A2q891NY z#wurWZ+E=@Rqu+~9amCr$F`b-&%E_MaulGdL0$GwmOXA<%g?#T2<JH2?b__Ae`(I# zxhhp$#CMc}GgQvz2k`X@A>#e0hvH{W!)f*P5Q`0m1)X>5D}}{KUi>gtTru|1@bzn( zx`qt)<Gi#6n*poeG>Mz2+od<Cp+S~);2Jx0V0V-GaZiLO?KhQPQ)zxZ(J-u=(pLlf z<3QCYG1_*}UV92Me2#8j<9|?{%xQ^Q9zX>$`kx>NQ?7K?CSFy(^yN%ZXO$!KE=Z-5 zRWG`5K!e@VFS<76?xcg3LFHAUNPC2c)_E46&9CZKhc(pO-}&4kKstD1<3R}2!J3XL zPys(XyouHThy3S7!niaY(_@MDjf>VK7|AzrbYfOlSmXweZj1tJ#3sE2ZjW@&ksmZB z(9URI*j;m{i)?1RUlM^3HY2oI^s8{8HrIT1xoX3sjn-l}n2rOww7qh$1BEe!^<S6@ zwS7hisdH^oG9NRUeBah^)@PL8>+Ba<P(n@$0wd?Z=GJR#mp*?@7FVvYAz1H5eMe?) z{#8%w(dC-;A0nAb3a;FvY{Sb3U{3bYM+gt?Czb{69VPgJC7x4!sU+w9=_IT*qW?vn zO1wkX#H=%##ZZ{d6{BtH6=(gmLdwXXAamkI=N*Dt4%J_^HqZKc-jX%f)_@9IZrTt= z%m4mV>E{#!R1Vw?Rco`HKA5LVb;wWcDh%U0A1xZm40%Npz2}I6;5-D~t5f8*it0l? zw9tu#3EPdXX+j~K6XLG3b}aG2Pg^cfMS_{dC!%jGS7VR!?JMo{JnzJQRvfZx1_KsG zVYV=kWC<eFa!S=o89td)u`GbuwRlHOQD%H+N1ug20yG#dSQg5%Yh<TzKk~@!hCzFn z-%oT+dG@fRFfP`KY~~)&2pRp-9$s)aaA>rUeQx4lAT%`bU7T=*y8tZuZQbVT+$X1| zp+Bw5&7Eawq+aF_;QH@h-g(h}2MdIz6UVXDp^F#wFTYk~dv4}(c5l`%4H5;6Sy|W4 zhF*hF=UdB?K)s``wa>CxxlXB?WLtboDI`^8x+VUWLW-@x06E$h`)pLR<*bg*x(W(0 zTFP#2=QR}1dXza?JHN(vi;*1PTf(5J^QzbLGs~-lY-f!W=;|UU;Ro^*{dwNt$@=)w zccomdTnamOB=$!`I|#CYZX5Hw$`64K(;5URX9degO2>3A=v)O{ovl)`!kBa7@zI5R z>v5=$5iXZr!`_eTe{qZN*-!RD!wVSH%rYdh;2M82u7OHx4C~}w80@iIMFNuvw8k?v zJ>n6snkYbV#r#na%+9Fk7!Ag||CGp$^w!EGM2{hb;0QbYe~A_(sZO6dW0}+}p0&Nu z0jZxGc<plGYt$brPP?d=Y{h^D=vkXBRD!mU%tQg_f}D&3|1gu&6kM_@6P)s>gB57P zcl^$0D9G%-$M=_z3ETE|wasj++ZCE2N5g|uE;9r}$2%v+h*gg0`eaAZeTb!);CwW- z2I?n_$`5+Bthr|GgSIGdkx#Wi!uxUt%_<c6UMh>Kx(f9Jsn~CaKRvzcwYi=zVE`LP zt<{DESXk5=mvPHVXYvtGPec5ul_%;-GoBYJQ%&)rw)qx@y%V;O61A~IPH6ju1r<n~ z#Im;@1=foR^I>SoWn`PMKhBW_*bYrjTd$M&ZFn9YPvU-gD2^(#)<cTJ1}lN@tZ$G8 zT3D3ArJxD!mxib1{|IZ@=5iC<_ntv|gFdK9r^bBxP$qmMva1pc^78`XW>#bNMKU9T zoYYWlK+;i<2C`T=vW;g<YB5Xaf0zX-6d*I%OR&jE?hjT{@nLjnwv9eadxV=mV)qOh z*hogDN0*8#M!i^JgZiakYc96&C(gJ6E9sfC4Um&*uU$#k9gaa?8F`a7?EZO2E7-!; zfU7CUVqTfr=4qTw4<IwN2jWXd5}n@fU-c3T$++l!<d!6kZL{WrQshR)CL<hBAlTLG z$IE_!a;&)`a?CK4Z=Ko=L_lzXb}zANltnSw9RqDdu5bqbD*HVuDJkKA-u|EkygcO_ zE2)$JWAg|GyIyJc&+A%EPIoiAI2we#T`+PBD>}h^Y~LgJaRK<xtwfX|8xAg)jM(=u zr)TvRu%u3`x{i75eQ4Oo-rf!i*n%r^5jxmEI2q&@!Z2TT0N+(9ItT0~-`dx}QyA`- zzfrM3F#%yO%lq=b?#O&qwt;SbJL!#S+Uql}*R34=w<jJ90m^IP)@ze#JoYKE*1z^6 z@#V8jWw;$XZe*5<3o=)g^NhP*W6iLk#@T-9VU!diVIO7U3(QBmRFH4GKJghg6x21| z^8mi^)_tF_GjWW6uMk!sLP(z+`}$%`cRvS2wCMazOc(I{w*SLSB}N~ikblI=aaA)T z&d{7`pRK$geqk=H{T)S}FlU4(x2nA%yT{WXzE{2RC`$BGgBBo~RWC2-`CAy_I>gZn z+veb1)ii?$WWb>agts+5Xu`%_N=w8@Z1wAsfVxI_Jjqx|an$pF-^-4@{(G`MkLR_h zRr#A32WbsBs9g96>}O;@m83GdJzAqmSer0;maHE6@YW~a76mnxH>G@Ql_74xp^%y+ zN7;cnPMkTp(sxZIc{V?%F$}}zbjJjOxMtztQ6Quzt0y+@q^@sCG>_c`=SJZKZ%2#z zzYQJ%;gnERfa)7iMKoC{y{YpU8?Tv0zJ96VU_;!YHo<J)*`<W+I<}To+d!neB}hvG z#H{d0-gB1H@m8$K;4h47DAEt)ITjikndfa!#AjIQMlX2I!5btc8EliOEnSh`tynfa z5ygl7F494zMg`<C^o)eB60xDbkhI5sXJo#`eQqTFnOEGl-Ic7c@}|as;_u&CtIv+& z4MuN`ljhu;DxH^`jHZ}WH4o0XjYF13)d=VG#Zci}`)5W2^oyEOqu=xfdOPCkY|V12 zRch~kZn=r8Y6^7`?*+S%Ros{;2cOl!N+kZxfgrOltrv$Pi*Pnbjy<~{^0p&tNNWuc zMO+Kk=OzMMo$7YZ`CZy+nHyckTf2Zba(#q^UB1l@sfF>*Q91Q<2fp=qhk-5kBmel; z4%qA(A+G=JXG7{6wHszilB3B3C5rq?TA|3J;PLZ)Yn(i@A2$WRI#r#Zy5rJV&+th= zXSCHX+5|_85AH*TUBdAdMuAUoFMhnQsX^SKBe~TD74!j>`0egfZRwgR=~J85gwJ$Z zcg(!wckMHbpU5ANCEDndf$y<b^pL3;sG(a5yVg8X+KZTN7z^vwFaK(`)VBT#>=vc} zO<Zj@ZFc0a;?Jf5Kax$}8SXycbw653HI>BdHw*1Gy!>m^rkEFvYm3z=m#*zyrCXz~ z`*YSkpBywlz{lLvGMmn!FT?rm_Q>Dxf?AVlRh35^G(5n=RG?7oCRp243%Ja>^Afx! zgd~U`j|#k&x{CC{aL+I{gGl;PIYP6}tsk0X0NrUC=0V)LoTMqJ(kFOW5NhQR?e^LT zHi+7=A>i45P320Eo=eexJ%`{mFx*_c0T}%PX`-6}bicNhOnTbo73koJ2T4UvuS}KH zHmaL#H+ydGWVJ^G*L{2wR=>l`2^73TKP)hJ-;Kkb*~~IHT}+Uk8o~>MWQRP#*~;YU z!AQ*v#&UpN@@`0CR$|aXr8CEAbGX~&r&I)J<!E7l9u3X%k|QuPr`Gm<2|8Y238i;< z=9tiEx$)SH$_)&3Sa(~0Sq~X<t4cX`|8qoh)7w!)b}Yq9c>cd?{kM>Z(#zcsdQlrM zPZ<uK$KJYrc_Xg;Rn);JnVSd8ym_l_8ExTtmlmNv-Q_N#18wRy8WDn+BeZnK!xY23 z?+e`onnEq)4ZjilZ2CxXyyMy)wMlaDB92{?*keQj(N5xf>tOd0RCy*Wx_!rZXJ6iI z*)9QkTc<q_|Gn#4Q)GViw^<H=r>xit;ZdEd?N}3lQy2|e35?dhIsRV%@l)5F=kYg& zx?R812fJKLd0f-S3d8;O#Pw6ePfFV(j^~xDs;nz*E~j~pX?27}#X=X;n1u|SLiSL$ z`FVuCVGhOWyfB^IY+Q4r3pxOQr|(TM7%Q>VaB&}JiaE@E>XA%q{oaATg>_ToCj6dC zY?%I}?Kq-D(}3(J>9!xSMlhNfO!XAcEDL1B=Y-(@>W`xJl>ipy2wJz9dayZH0lu~B zWX0v5)(xA&f3|v1KReb}y?$M*p-Wxi8Rx7b?^mHvX3i{wRGH)$78eo%M$7CtxXeNf zQC<wQLT~Bfc6DCUz#O*Ihb=`|i;ZKQ-OeLI^|^v%^|8)4=lk5%m=^N8|7vLehB<z9 zl@$X>Lcjaq|Ns9>RuFyony&oUy8r@_65YcntN;D*hh6(|!6(k>IR96-G@|HmedkUR z1PE+IMccRbrTnAN@VBq{#}WSL*61I`WXv(x5z!yjV>`1;IboF7Yjg;)sz7$~k3T?p zZCA)S{SQ0ZKlSD%(1@s5LMbzzc}Wc!J@zdyv?4LNKq8-U5zHhnBxRjNx&Gs{9N6WH zll6cH@pE&yuKeWNLO!r}<&u9G57Z{Voj2!-LCdRUU1LlAm@N22vNyhj)ImD3#L$t6 z9Rri0G-h0nSAX%bZ*?9po;tt&Lwbvv+qW(~Q&dW&n0zT(yz&LNENdFEk5Jz_9*qS` zdvz-R!u5b)jdO$Z4_AkCaJW8pAi{{bQKOgx`BEh3;KFgFWCdf+|HaSEla}9(F*dFI zqz3`wI90COd=Iedf}6UBx1=-wjC>X!eJkGPL36*PhlF$E^oSX-d`6fU?O*Z2l0QlJ zV`^gBabvxj!uarA@Sa7%euwJ}dOU{jB^5--dYUL&7ID8u@Nc*@XRCpHRlz<X)51>; ztc{A4*TPX3AW`~i68=Lxv2@*L1+%SXr{3&X&TLxBUbYQQyY@D_X`vLnz;VDko)7Ok z>#$Mr=K#-g%jMK2v3t^N7BOiqUOs(w&u;#L$WJ;E(dXVHCb~emT6)iaw-~(UI-D0b zCn|?>Pg_Ceqd5NYM6A~^IfUaP<tDnq?zh=TvuF%c43Qrohw*5<kj045d!j&%>PF)& zTl4u)u`Zj$q(=$z8T1@fZYrO}KF?&9gPrQ?eSO>{=FCzsJ+>=MI~P>~FFTU-xi|C% zUq_l&k6wK57VA?s6}F5L)mw={gROmZZnG5_@2zS+&>PV(u!$)1!ke?}JuqEr=iND4 zs5Dt1xG$07UXjX<fhtX?4Z?SJT}97y!H1ILwEHU0VLs4ey`*7F3HSZcDQErK)R;0n z5T=gY0p_PAP*<?fus?k@>daRqAGb?ZfUdfpxtAW7thDdGC4}FHJJj#|a`$g{jhxJY zC|2d>iTFAlY%#K5wLcWcr8_m2PWSEC%ddhNCV!qPtf?SGD?wBr7GCLczGfUYbVafb z-+1&ynlu*|2u8MDTKxE2hb>6lMC5y(NabTaTYgFJqS9f$kqp{`jT4Al&v41QQw$^V zXHI#{v>OZ_ORgyy!Oc8tz`_EMJl=q%ElhUTdRY`w3N0wMgj^zFkc4fH&x}JFHRV`V zqK;OECgy@YHOSD*2WRCLRK@Z(1~X->c#8_zuPdti8W^Lryji!|?~*z&YGK6=xrz>d zMHN@-G$R`~U!`5QR@%2)AzH6C`o|w_HU20OC^)GF3`dFdN_bto)JR7>r|}YpZGmt~ zBA^v93o#k$+}zk-<&2Avn#T1XyfdY-77tp_DBEpk5(5`vC0Pe+kb2D43AnmM(yca; z^9iqO?M#$;$6jSwvBRf{L}pPtBwZHZhj+r29l7ENiRv!`QKv75B4~@3+Gi@bxi2M7 zf(_Zd;kKI9H=2G4j*Nz*K6>PK%w=)4QWfK_HWau4k3{PW0CUM#ufFb=Bq5K<VzX3$ zNfCbryx&@s*a+QbqlT>MBO48OYnn+py+$uyJvy=xa-T4%8KIirpNFd))pbxVPI?i0 z^XBZBV56&}{|lpa8Sjq{TMyexFEBjcB>Si3h<Zmqe5pU3bx*R6bd1s;{9>2%JceDi zzZCKc;wyz}&b<{9zx%K+vgMjHdJ7X97ebEJ6u!lb&}|Sz97L&YDQ^NZ+-w1DinV_x z;DNdZNDnslf<3b%D#yRDW4kQ|1It1V8tq|r164W?y`YAQMwWHX@<$6o4y2==J99d> zUsM`T+(lhYPmfAHDO*4y(O0Q4Lz(oU;wj&5ITEPPm}wyfTW4J9j)Se9(`6eBp&D1l ztZ3{DufwPe5yD+O$Z2o_)yiTHh^^6_n5ly`;c`waNCU!-v#zPEw=ZvN?p?8y=@Cg% zCRoy{>X&dU?U5hQMH9X1&uHaai{C&>+p%V~qu-W}Glx~b^UwXTudn&#ao(#09}h_X zq|wy_A&%}f%qCq2ZJ#`{jmpTyI)Bx~kIX;t!?&W{n$;-Zmol3^WI|})^%%ZyMLYqk zql*zzb$4lf2npxN|6oJBxJ6=)Uj*vj3k>_osp^rpIhPvYo^9V{)7y?^N(Z%sg8U2E z8$KS<13y3mu%(18Xr=WCSyz;aw-%&eSJY1*FXg$*ve>PU-g9g^L*A7L1Zs$|NwNFX z*#W2=&Yrs8;9Mqd$|^^tTM$Fxu356aR3>q1zx478<NzTZf`o_SR&;FR?j1u3J=23g zK+G_%D7$@IbLABg9!hH#9bG|lKbFKD`Rcr4U33qx6(EkN?wZefe|WW`PsJY__LYrp zwG6oY#49Dj?UWZ<Z&~fd=V49t?YXz=l6^ex{-(3`u-mHUebR~zjQmXhw8EDs$4x(9 zH%#ZJ5)-wfmyTt9<_OxppP=A}6OY&YJe$*tjr0yUgK)!l%JQoxLWgck+b77(@@To0 zef=T6^Tj*<9fflU>IFY^%Uf&E5hS~Q=+=62WsCi%@i-rrCS>N*!{(6BWn&%^PFQj! z2yF|EE%*6ep;Qw|%I|GRq>I4={2)ILJZd1^cg!klf&7@64akMsSvKlbGxGqjDn`E( zR0ON9gO?c*D+1vGoFZ5H!}#qqpGIc9Mt}A<Snnw8go)rL3eCH|-cdKN{mtvpBN0ok zk*^F`gK_Uzn-%o8Ar~(x;DR2XOb$e^)^MoFU6o-lsdk^zMd^99-&I>B8I|mgu^v0q zw(Os#uYC)*)8T{<%Y8gips=u$`D!Aa3N30>ii}#?Xk8FA^>X@oBX7+0WpF#}H){Ih z>OzwXnX8Ttpxgv<TJ`%AXe<j+MSw!^zN*OKE?CIGSg3@z{fyRN>Wk_Pr)MvaVOp3a zY_++-4L>^XUJkJw?6DaQk~Op2Ag@9~kQAI<L~L0Q^}zsqXEwMIK-@DDfWf?8V6o6t z^9%9Xd!0%Cb#KR`N5v+)pA+^95APEC%vVm>ofmytKI!W6O`sy&yIW0tU<g&%dur`A z#0|~-5wBYowDvgiog<V~L&41vH?uceJ%a)_KI^`I*m(a9L*nM`nAkxo)*EwJU7O5G zy!c&7sq}8Bunhd^mlr=*B?ZYSUg7QH=-7Zy5par0_fQxP!tZWT9{_I%=XXvA2efm` zG(lv@!T`6uvoMYvCREVCG(2?{@5;udL-10Tdy?h-v6BbCvy#f%-tVI;hm*;A`F0Tu zOLu|`6iqe1oi=F9V$#9#B~_pn;rk>!TzK9@7bNJiJJC2Y(75EeXIcstml{8ed{1BE zJQbflI^KAOQ0grnm^QzexEO73I2xjT%Fl6hP3s6910D~IMQOk=*2@*?lD0*~D*!3& z&sDJEkOI>Do$l&gg~2~2XIv#8zSUk6x!`HOs@2j4u5t_7q6Qkhpf64D@_T3c8+FDM zlobk`Z6ZzyIb$rP&+BppN35WM*ariBzW}{l-<>>q_*ZhXX)-N5tSg55K3tx!b+0;f ze73!^l5-DR#bdHaUB?7tf-iX;ggrXXnuKs4Tb1hppxUJ}Nc8gaQLGs>a7VnQp>+Uw zbn9h^SB&hANK@mtm)i^S?T(fyYmzM&8;_r;C?yi{u%Je0rD7b5DV!rtHlAShh_y%Q z_EMf|t+8Z>DB6qgG-5NrLQ)nDcLqjm1Uq~x+7AwKCre#Xwq!_5koI9e`p7R$gVLU& zcR;Z6=d#@;+`l`gV$Ik(x6`${{>kPUR!aNw6?u_TZ{O1%`Wd8RFq5~uzJa*mLS%HM z0n9B~rI%iebl1)n?00uR?em$WkXawrldtuQ_8jXZ98X<TfAV2AvrO8pRuEfFBXTDi zoUA_cW=?tzY@?$i2D&DEutdv}*5TzP@`hp46V}VKNsjpV7fFuKP-PimU6s~z-(82i z@AjCi4m+3#yrW&*kEViG;efzNzN+r1N=P|DqebL@eB+jZ#EI(Mh>N^eDGfDnX*0!U z`=BO-ysERFR<@sR*y3yaNcZrU0NrLC6!3;Xp#To<UFZv`<CTAau9Ll((5b>MjFe+^ zg~jz!Ym^cfC5b1GCG6YyXXD-qU_Lu~<htpq6zq<ZodH_~dtzQbdX>9lBdKG$)dX&4 zSQTRVp{t`6-|gu+RXz&YxqsCk46k>H{O|K(x{=IIkq20s*5zI$U|EzmT$sug)zYE) zfcf{|1&F}G*Rp<Er)j;FGQ{H<b8^2ne+*3?Z1qlwcE&auQC@pE`w3sf>TyA!aKt#% zI08PPN*+kL!SdS3t>Fj@>$M%ODcwdkQDX{_(x@7_wR3GkkPmbq3tFRyVtF}SpH@F| zJr;KH)o&C#fQw1#3)!FDcyLMSF%DlBQI1PeF)B^c{1GD%&hsbi`g7|XQFaTCSB3n< zx#9zVwjQp4LhbQNikr;OX1s54{$ix^+-F+X?0IC+>f6l8`dsfzvIAYE3A;R0UtElQ z3MhrM$LK!k$1=~PZ96!lpIT>$7ghB804V^(Qbvk5<#)JwJHVppm@u#N9RyngabziL z#(#lPYo}rHv8FDe{pA(O+LM&WkC~G2JW|YyBxrXOqtPgc3<$-m%}i>dTo<$H99nBY zy^cj>2RRn>k~%3?1^|oE{~}ASDQCnbKw8gEn^fu4%dD7k+h;3PP4kd0^Lx|()2K|h zq$rPlq>S9?>k4D_-rVfB?>*3N37**CaD8!o^s+vVAM2ME?}IPGOwjByDUC}lXED9{ zWk81Z=ZZZJc$`Qp3VwEvTE?`4WYT2qHrmobV^Z^?r{uDus`GY%!hS?&ex^!Ru{pC2 z5X*f}5-6+m%=vNq?(3%wv&B_T`WSsKw9%U~`&?j9<qQ<3MfJ=6JIv9hEmaQo?K!E; zzRL6A?TE;Qw4^PxieJmv;XmbYYI1+DwEY3$lls|9bFfw5v60xs^3u0jOS|ug?=o%P z5oB9$G|N{{4}6CK>g%<P^r@@Y@wSQ3hKYJLhH^DM*vq<xI|RK}fyOZM%o)nI&rHFk zfoPz-baSPI6BHI3{C82gZ!k>)V=tGFl4s4A(~BVp-0y)4OW-oey}GFqZokc(9Bvn$ zfwp)ODsYHSG8qVDJ>U=3(LdS?M^pSfp&q0T=|!5ZW`84x&<zpH9}XDn4=h|3sA*Rp z<agQgYxD!#D6ik@IC9}Z%sNYLzVp~_E70(}p4eZBJuLRsr>Mo_|Hy(dJprzIQT1u+ zH&oIOfA4&=JU|9lUt<t8MIf7h{&n#4^`DgPUJax@lu?ja_X)pFM`vF|v`!H9_Xz|C zt!dzm;R7vw#Cm8Lco%nkHrE>??Wph@iJjAxqAc1FwBPmtj^35hQ{g;4Yux5ey~YgC zv3p2}^~tw;eOD=%R^wu=w{r#9fR1Pp30JgQ4)(cYJdySTY%^`nbWHb=Szbzb!<?Fy zXd6{6n_kB4L&aMDU9Ej?ykB^h3*})gC+PVo$`o|IzRartH&O5qtkY$4UPq`7f(dQN z;13j}1pFX^=qEe-K5Z@zB*D9i=Q<g(^TRJ=@6+Z7ox#%{6HpMfVmaZ(L2mi7Nu?C> zQnS}{AqQo*UbN9XE5;l83`>lQH6~Y)RR@0f2Wmj$Uw2~pxx;qyoYay<gMlnDROMWM z|L#b*Bo#`oY9#V0$+mjAy908aoj1|YKkEl0^%sVW@YFl4=BW1RfigZcARgIrU`2Gx zTw19!-ja(W$~?M`8xhRfJ@w116Wyi^^gmnA$eve4Kv8X`k$ThhC1mC@t(G?c$!$l# z7Cez8>BE5^F0qH{?_&mw;0c%%#|8m?^zKVo_TbHSij7SKLA$h`mMbba{x^~<dOBej z+$apV#{V>jtd=Iw-~0Gj3;^Sn=0<Cc3W;H5Ji#&R2#fe#ahWiqC^%imI;leCRcRH@ z9YGa#FSgvzWuSa`Xx${Y?2Y6=O&nIl++Y?&W*>r%#hjUF<e`g6WcSHNUZ=p<<J+7L zpljCTT^m2DqThm$o8@GjP^yYD!*oHb>{v(r-SK&!{$NO6gm~zateQ35Rqh$@m4rce zT7*pm0PF)Qgb3=OBBJcjcUaTjgA!s5ZsvKTd7U<OmHu_4-uN#0g#?Ql28I~ja={`x zHq*c4En<1;a^BMVhl1wT!p<MG(jhk>7bT0Wmc6b0HOMtIUc>;%z%qXB=s_yQSCwm{ zshZy8;OkYOa|MDrVL=x3QFNN7ICTx%=Rku_H6gBk<7({LTsYR8YnS>rF2nhyGrm*+ zvtJWOmh4+fXj<=Ge&l+osP;5ogXprbP^;P{Q=Hm(2^n!VTZiR7>l+v|Pfg614)MqQ zIWEjy5(8?u;l2FxpC-+b(h#_u#(o(;AMAta;>2BX7qs^34d~37(-==v8PqarlqN&R zHTh@=wot3Gl*G;J?nq_7j6J4ZgspQ&ZT_Y_(s)b3yfXXFbL@L+TE-D2%d09LO2^>Q z=Q0l$7fxXF70@gNxACK+&d=|!+P-$DbzQ9rO(SS_j(793lXeD{-Lvc~QtXO$>JA{+ z`l{|3&OOA_O9I|G`)oWky0LHq6vo%qexjXqFaISjQ=`h)s0jW?lI>-GDO&rdiUba$ zF8uZVJ1j^ggmWq$LCes^=t{D?*fftx_?<zuf_O)&1(5KzN2gN*r|P>bW*&A?JnYII zS?fvajoi35?u8cmIt)#<Y(K8erVJ44M~RsFnRszSV3XiJ!2=*mFVBBt2KIt)ASD(J z^5I5ze-_+mPgitAo-`++jUcI8QUVWjp;~K;cmiZB!Gwv^ox<<r+N!)Sm3j$(a|^p& z%{#k!=7iW<QZ&q3Lm1uME`8q~zod)#FhpHKj;J80X(B>R23)<Sl=)d8i|KZU8QO|V ztFGuUtLi0z@(P`Km2ZmMqraR;`%ubwvXwvjrtQuy=kK&mtF`tuAJWoCTb;B#qHca( zU%x&mOnbuGpOINQ;c2WtH#AvjlL9w0UZpV*?<Q_BoHy%^ntbAyf9qE(YqCo8Jxv4K z$@DPr!>fX}cexhjBUd8y2iehL9S7H|H5lSav0&A!U`>D$c^*A?vrJ9XJ1n9dWm6yi z+DSavG1Grkj@FcbsI@iz?cBZ8LnE>8UyEY)Tg8X&q8ESQ>7KDOa%?h)(Mq*Nht3JU zJO@6-wq+pxUbRcKcd%MwHMX8$tL!Fw4Wnp4c|}&B<LQv6rNj*8``Mzwe*^7@Q72=Y z7%1N<Tl(9E-ZY%ho%wv$d}(O%t!&F6=%M}{rPQdzlvO83<K4JoE`>h1d*_r2Qjp?a zMR%8gM<m$gk1|&1Sd8IEn(4SrRR7dSN+q^mjh!Q0ybZC}?!nC{!`xx_c{u#W#XFzA zmW+iz?^d&JexY)a?WI6AQPIDa4AmpocaV%sOiRr(0t(BO%3d=xURk}(w5hl6pnF&# z0<HI2mcrWaw8wCFZ`XH$hNX%K*U1#f0)Dwru~nCPiiBeg7IdTI#djd{p?PkC7o-es zR4lR^>i;MdrkNe?M;6hX7Xeu#2P$7}9;PWI@D}~Iz2TrOb*|=TPQk^c+}?ed2+58+ zfe0}Mh$Rj?NRHq1#P0PX+-tDtp&8g4Uhrv&*?-$dUI{)o#rbd?sWoILN0dP@kl=p2 z%|J*JqN$~XSHz%1;@qdUO0T-<Pgnr-tiQ=jXZ%i$x|9htVs$LYE+w;bs1e2H8S^EY zxW2BEA@i)xQ++P6X&`yC;v<=G`$uH?y#_*lPZz(_9><bLHo1j8X@Clb6DHJ-<6&2o zJ4`h(ZE91N^>x?hkBN0BeQWcOk9fV|;W6#Yxmx-%g|9eEy5z^LJiVVFN0f7|Is^Rm z63w<`c37^}zhtS~s`#khS5MuMWhxI^u-ae}b=~M$;}!OEW4WJ0_<}VlT~4>KvhHu( zV#stNn|G+HU)xU8TSb0<z9yZIL0||Z-eSx_Q?h~lQv3^DIMj1dmtQP<P!stv;sq3- zHi7k_P%37&b~;8W*6EU6@@Mvoko%YiQ&GcVy`3!aIXrcOReO(76bjMQp<s=v*SM*3 z*<tLV?GTe7UO&47QPGG^NIMtk<yJ+wlxQKol0PMiad-a>uSGTm52sPK1NW4pMCR_) zpW@Cur<VCr5u8+_eOGMyV%x%5>M7*@U-H_lU3Jz^KXxeta)~Anc_d;K-jr08D30=0 zC3D}IfFW3t6-aPJrOR#MQHT}D&B50kI(331B+g|8LgD8I=o2o4zY+1pbF-sRizF5G zw&6O-`zdLy%277-OQ94K?o~>vJu6-#9OT%E4ZpZbp%02hw8bpu<SemDCrAQrfadYt z^@VKmg0*%F3&pmv{pkfc#eO@4S6CR9pC0yMR#yr|<j<?T`3j?ro<Cs{G>hMW;0>Kd zw}!92z3OQuF%Dw6w<g-jIr<WzGwf<}=`Qy$w7b+=q%Heih|kyb&aFZ75r@k>^gBHz zg4YmBHxIo6JH1^)aZgVhAeq0^X=gh+dlNKQHJ4HEh#3gCsY7MbSAn4+Q@jz|#gLeX zQKaFb+|L;T!XhCDWo<rMLJV0&h1688cYEjASh-;p0SL7FbJ#;P3z|2T{kG+WCc!|K z9|vUcWe6#9{$<#J#1T@dp={WjSqQiDg$4mK?D&Tax6UeGM_iA||G`?LZJ2R<{4W7w zJ@*hYl3sW6qedT*g(ZP)I*$GEU5v(%MhP33w6*dGrYX|ykvGm4A5aSodOZg{tCNNr z6@L%hgOP+Y5hGz6q*Ir{qTl{mXTJ|^(a9b1I$Z)+Im8;gj-ctOl!BUnHzcS$W!USl zO(_Eht&S;bWH?LUT39k19p{cpaCQCz5oFdt!3O{ICcY=_Vt(}vG`sn`zVJM^f-Ayc zJ0qQEsgBZiHqnL@%yRrd2C~a^<#QkV6X`P`&OLu|leoi9)32^2tjEdAJp#kFHd~BG z@JfKh4KFWkFluAKD>ay<m2LchB=QfZ4g?~k3(qkTnaZy}(nO6KMTQPg_HXz^WR#tl zUFO_rQ}+)eawL&9R#)_XN>m8<NVL4)N}mQv=nX=qY((cgqTf0BV?8*rL~;`nTTYTX z9A1vvCj4bZdZIPbFWof^zw|&4InDHb1IBj!A8i*3sQ3*2m?ty`VzK-k>7n$z05)js zkcItE1ERK7Gw8@Z6<qGi4?xnfYHrTynb!AUH<d%V7f+qJ@}Styv(8-C!bzjeJsPd} z?roc^)z3^FX^tGPCp_mkTwn$Vd0NVGe0mKry3bR8;GE>Sr2FQ?HuJYsc776u-8E}U z)$2N**SmZH!Rw7gzB7jRiO_9_7RlNzFYKggNkw4?_hi-MUtF1Bh=B&ZS<YpoxENj6 zKXbCFk4@apWH+dbrp89HfvFyLSeC+UxDFg0MO{{#iWAR`emFF&h529=fy94b%sxoC z$8PRna4*%wUmbZT4%H8&BB}&xzYzIlW$@V%ZHW9-BXs;A`?*fCZ4-3rO6Fj9A-|8J z&PH=9wH)D_>gfNBUv#HP2z&h^Cm(%yh0{S?VaP(^nGca8QYEC|=ir5yAUu1|fvk9Z ze_~{8qhl!;sxybNs$06toAU%$V4xE=)|+;ARV=gO_W`yLTWyQ+nR+|(ELUS_q2;CX zz`^mvg3n1PlW#&pyA%cCQ<Xy{C1Y15tCCA7RbM07#?znB{?BGoPB=z4x?aOb;@xjt z0!i+K4(5WNUAk;o<rH>)H3aU7>fMpuFB)<$wIKkmMWp2rL5w!%)~9#HDv?RXh-f<_ zVc7)HKKs13kYS_fP-O>{nH&b<YviKezWNjZyC-oK(n>g%EY!8fC4{oSO=MhzH5h)6 zc+GWOHX4$wKyS#zJg#gvOW^wXxHZ-1MyQve(nya61FP-vNLTSfnGio8RgKrETMCwW zW4b^g3;i*gvoT2S6Q!{l^*Q29&4Pl?d3@(=8M0kA!q-6^`YH^dUzl&R>PgAsX{64@ z=Ntp>zZ*~<jrIykFGawAYG=zve@Rv#l$i(AmTpLutsvqrmYZe<mpM3jenfXEyvSPu z8$jLz?xT@3uPWVH)lHBO8~c^u_#bC!7bqaa1JqdRFEx;xYcQi{R%{MVBot%pAAQ?H zGL67~cg|JzjI3<(t0?oU!xvsbaebU3_s_XP%O2NS#c$9SO6pDArZN|Z23#fe?Z5aK zs&ErADF07Y8`AqHcAfE*>H!Qz*os(mJd7ev1?1-P2<>KNNducY+3kRSEb)e18HwmF zO$Wi4v-~R1AD1SrV}cj2{i8C#gs(o~D$4UkMLsGq{^n+kRW5yWe?HrT1epaK+)y`@ zi19DWb)TfymM2N1@-KyE=5<C*KgHzWFlM{%XYudjWaUi@gWwDL12T1f3AoEkQcH8K zGbU0Xb)IjCL;)}@+f_zHoY9Uq650lDnD{bLJ3ea_?-g`#YU>~*S62cfCC#t;YLCAv zP2=MGL#}=vUlUqivk}*(WpP<T-bK|;woJRl!<^8Kn>ZZ-$9i5Qm<J+<J2rbHy4|=z zeB{GV*>b2%bQSfN`l#Imng7JGr;FoBS`VlND;ku&PEmmVxdzprn8o`4(inoh)tP>v zAn-5B>EXs~vVX0XdMEdlXTuniH45C3VE-LZL=t=O`yw<jPtAt$qBQwuXhdMyW8wqo zy0Dr?10V`UQ6Ej=WN#yqetni98wtC?&k=2$sGM3+9gUO+GI~_UM5T25=R){L-cU3c zDRtC2^A)!;-5o>D|Bj>g_E*=jjD*Pd@Ux=Be=|JK2>n-IB(Hj=rH<!v{v{C_E-GH} z82*)yXIq$7b!FC+hRBUsGQo)>M-($NQ575z#dG#~(@sgOxW7A;dbVj(tS!KI%xZEx zk>>xVHjW9Q?O&^p8DJ;U7SDoL{qrRSz1yR5wx#fnrE5RbQaQcBsKox!SL0$AP|F=m z;%9^R!3O%ya>oa0qXv7jNIz9MK~=q-rc*z(=ES>DJWUT13gh1`u5I0}1X?eR!X=YY zRR+?8ZCiPxlpI-3<JmOG9t4V64MBn#F2tH&Llz8lfxhKLpQZ|LaU(kAfxi^`p()>> z+B;wL)V}uHSGK}Z7OgN*{%SING#aeBHm~sm@4n*cej8$H{gUrt(`2ICY)D$lLNM#* z_FYXu$l6|gmWKMSHNU^uaH|!&c|@HtRho9O+a4DBdWrYE$xYgg@U{~>KJM}8@>5hq z0HLQgI_kB3hrFnKM5x=6O<YXz$(#QMy8ni=^e<Mu$ZeszN@<R@v*p~u%hZ>B;nQ~_ z_SmkkfBQgqJ;I!*46SKKp&bvm8U2W$vrPhULZ17p5SD$96B#4U<`3D4l@z06q5G2v zX%~5;`9rzOYO7v0Y>c!Jb1FZL26$qtPySP7{(}g7CL(iHMZd;-TBqfBKf#o;{YQ2j z=&<F(&2Ex5QgGJK>uTm|>Vs9aK_$Do>S(;u%MmldV^2|6=x39jx?scytF*|}<#eE^ z@89nE$@m%l%fV(vrD*U@%(2Bv17x4xx6bEv+J6wW5BD>j)C^XYCv1xvz1m)pmB{-< z$Vn0^tib`_EZwT+6cce@H+c;R*_k?H%|QuKvX1{s7W@}Q79cm-KmXru{C^UU0G0TL zWBdQ$7aUw>HOU$Y@{fK?_mTujOQa^zS#{S&f++u<H-0#m3&S(||KtG`MwgWT{hIz` zqqD`gfKR*%<!pldJxRW78K7=Q&~29cp^aNC7ta9%HuL2v@G=X7?{8VkpK$g1yCe-z zVE;j@Ilo>}Z^w-L@~q$D9?kUkJGwa5>b9}K@8Vff-IrYC%YKQ!x{2JZX|>+f58s$k zyWgZe3jgXg|3kjGb0<t9u0l*+MZQKI*!|^v5?4ot#k9FE%~~dj3ad&;ol$KQB`rp~ z<(A9Ce!z%Tc|TZoCwW4Ls(Ekb;F~0|W0UOdtmW5g1Hdl+LF6aDfny7g{{<YRyA-(W z-JNqt!Qr68*gPUHSs`irs}Hr?_3~S0|MmMFC=-1l@7NcTaQ5TEb1THYl@<Yjs@zX~ z(4_iTBl*#S>n=a<Zoj7B5JvgTw_xCu)k9<a7oK%Ci7e%dmdj(7@Xw9qY=h`JPTHV; z(v%%ul!{SaAor~xe(f#OZva4S`Mp#Jg$9faN_qJUaz~Q!%B;$jH~u~D)QL2|HF<>T zQss}R(syA1O>Q!8pLxM7qH|b3vAlYeX+r@Bf}8%jytY%zxWIXS4>Ti1?gMZXSGxXU zKBc_}=6Hhs{sYrLG&PUi1CFw?T$+nEZyg_esn?x}n>u|;G-WJ$MzWoE(fBgw<@DWt z(d<;jMED64K1JdVDvE$I<M+$w{baonyrd(=3)a$!QR(tSbRtFKktm$BrNkKxR&J|X zI)4zt3&v?KY#n6xEqt01+c{jPU)>*bXY3?t{<M7xZ^;OC{!duXR!LcF>k<b^5xt#S z2S`=&UPu03bs_%jU)b>Y-Y-8MxlSZ_Hi%9~>;;$kv2tFXIC4pJ`j)95I4>I7a$8X& zKx-{6nc<v;EhjyM_f9Tut6Pw)L{akH-%)2DglRM<zp#_}ja6B8+qhrf_j#-=TSSiQ z6@<<OpJ3npTr`~g7OIqxFS|>(X=|J5Neq;8zm&*wmW#Qmkn|avWiWpAo??CeX)Vnz zQxtA7$ls_20v)S!9#NdWKVy<#{kg1=080qb0lK~<*qTbp{hmH5p-?-BA*kzy*V$Hq z>k=kul^X0Da3~nI=?SLI+NSLnaXFWC|5$-zApy!H3;?qNb(x)utm>VCFk6qo&L|}) z)tK3lSVk-7lqvVRRQEv9KIM?UE*V`u$j2{QQKRAe)tZ<Hd>v7CI%z^4(nu7&Yhq8a zB@h`lPkg^}pn5MDms-t&MOY<{5tBm3G<^&>aEvHGHtZ*<uVFn|!OqfCKBPNb|G?Er zTu9MDWvwV*P|}wP_a)TIuv1{o9!OZPd)WReL7A&Q?_t+oq_k!ns%SGkg=L?HIqFrs zxn&j<<cDn8YZ}P-0ierm`zfFn(s1%hk)VwEy|vAPMOo&Rxe$eYY8K^wK3G}(L`Zo^ z<E}4#qAMGZMJWZTGfneTGtGhYoqX|hO-Hn$gZigbAJ%-9eL=^L$?lt}h_;BfOGsXj zpsx#Fn?Q1c0*GpnEr7J=D?7@vU_^n}>WhXBn8m^@tq;U`Ywl;;lKA#F!gSDMJYJbR zM_9{2@Hx4Rew@93Ma%^4V76!inxGQmW5HXzGg$8|P$iAk%>hVri-neTDG~}!CWeSh z|NUsbq`k+0EfeNFj{uH8SG527pCZ{g+ie>MRBCVByvm7Aws(s-|3^X$F7owRt7i{i zzTB7ds)qm2f_GDk&<EZSYHC=iVXtY{ewd(Oc4_g_w(y1&rH_$HrVq@xEwp-HPVTK@ zG@#v(=9eAe6-45OmlgOz0A*#BmXbPmc%Zt5wh`tJcM@n5MoYEqv!Cux)B!@6qQ%+j z!|;4D%gTl3oe~q?GYona3&m4KnjG(HShdzN?J>`xpsA%WBtV~V+Q2hu5g0n;<BsaE zC`X_7!N?A5ut%av$7pyQwv;=c8S=V9qlE|=?FbR)UH+HBm`}U-WzmJQl!>u-0X)9_ zQ@>@qW$l##xxk9ur){^p{mmAgCG*QB@entCol(r8W1JJ4FEyMt*1gRbJddYveqVLF z56saD+*;3BeL^YWjnVY)(?^cj7xoYdpLj>sD@FF9PTLh4pR<rpM&<T{0^Wp&?&2yr z=6MR93%mk#OxIDrYkyl=9NlQ6H!D|HUWm{d9MP7>-Q4JyRuWF&PR;Zev%dt&&73_V z#ur)-ZO%SFpxWDH=sxiUJlvK(Lby*1h0Gu3Ky%OHH>{sMolzLdDVE_DiC94c6v!vL zCeqm%E8QS=AR$qikzJXz5Mf!v`EmmH%;lZmXX^Y9gxq2h7gA%%?dQ%|CGNFsGnU4> zTHbhWD)p5_RpWe2Lfh@jxR{W{#UNK|H;-FuU*)rq<O_D^Z<n4WJS_%O?^E~E@O2G( zeXuJ_0w-n+HXy#ug1TUhC$lKW(bY3#qscK_7q=-bg0(5iyiK@gF!e(_km@%YfZGI- z7RUWu|8h4ifPGd{Je@F~yP}E32UfnXwX%fQ95*SN<(qH#&E!mlhS|p(^2&|L`;%GB zl%T^3lvSo?9%uMR42cK;YVqO2RB2ZGRNPTp$QA3lj%riiCJQ{;j4nsbThy1%TA2=+ z1$=kw@T=SrVNSgp<S~d+7*@(h$QssH9xniC(lxV7*=Lwow!4FODZ;#hv@d?Q!!rhs zVa4Pdi{A$G+UC{Qgc3>q(Nb@1YcbGk<;;T8<HfC!7Dn$}Lgl;*tv2iu!W>KtLoV&4 z9`jBm8XavjYr!kCL&1bjE}s*Y#5x$zUm!p-c{x>DE%P%xk5)X+jY={AI^iv~gV;Gz z8?rz-Hua$le=k(RxgZbPzA~^mR?R*W&{f9BTi~mrp^t$kwJXv>g1kjOt@<V2<=;Le zc06rI%Q|A@cWbsdDletLv8*2!6?aBrjc&g*>2c0eGs^k?o`+<g;Z_DXeU*AR2!G_~ zblp;g10H%+jinb)WQJYJ6Y7#onb729#MpYbL3QoaN<mD|>M8&Y<|RB<>aI`@o=(<% z6>R6%3qI)4VgnT+R}J{kZ{AGinTO+aJ7TVCUJ{e=<{&xH>m$tiJmQIAgK2nBB^Sep z1+VnJHC}22Mi@X@L{s{sk7eGOkShaOcx5C(89FX(@dO_)=p{}ERD+W0Q9+=Op$YEf zL=H!1R`aeccrSe_ylElcprBEYFx)J&Es>u7lQJ@El5k45d0N!8`~z5$Vf_TK^8JoD zQy^#qYbDKRkjA%f`s+ahY^+@=3hF$M@7L2FXqoM)SbxY>ef6i+x{BVrH8(F*KQULQ zd^)nj=R_3L&oNIWyuC?X;iElY{C&$FpKAa{x9f(wPtwTFmYt;)y}4Dc0x4(XuMPqU zje=xff_5KUyMu9jZA1{2fPQ?Ra6xGJp!E&oZ6ZG4`Q>fn>S*>ucCOf*_ZkHq>mISw zHf)(72ihP9$kC(3QNg17L`;wlor6uyHGof9VYpGUU}mPRm{_4Ijnsb=--;bmPDgD; zGFvPs;nXbtZPw5;00ODpK0CbG-JjiF)3dk-J~;Do0voy^Te%KLjGBA)Z?&mT+pz$| zj0^NBVynxI8!N31;fYY$7o=(u?#^GaSA#}k^o{Nx+Lhl6248Mc-}LF##s|tW<iH&z zSFP$W_vHNgYimf2DmRDIo0D0T_i;)`O)9E1@EQYfVL#o0a*KBvr~b%#ec9^^_5y`- zL(rK$H(TI{SH3Mq_s?g;yv;6dSHIgOAB=<YKg8G%(=#>ifl#R34izA3Li^0S>LMeI zH^u823~DE4ytRJQ-vuj(4yDo9L=gh+m{qqgsp#q;yWKvYKUOZNNtr<0C{Le?BN>+^ zNgmP?hpn@L`rnU;@t9A;Nv_Mft5?nsVt?sBxI!labc6G6_RNY&U$`05m$pOah0gcu z@eQBLv%qsE@}IA-W2YXk9yL;Ub*;_$In+^%RK3g@kK?s?N$>h4bxe@j>oKE7H7buD z17oEWn=9!EaUT0pQJ8BuLHzxiYpk5o$D`Z5%TXhuh>h>~`9uTtj~_f5c1kDqJo$CP z^fL_#fE#~aHhX~#8#T9z?bFw!6DvE!2==Y$(BoOw+?Wl9W4nj6nt(%QlKZhUKO&i| z4oE*XBa|AxE?xElmN+n*-nEwr)(C-WMm!#`h1Pw@lqCSIFF^FX(sTZ=_#ChU<NYx} zy)N^m-*_Gwef?H7FE?dDX{TmmkFhjk`t&PT#my%97Q_umlSgke>pu2hY<U|O8gbp* z6Y4JBMoQ{8_^qj3toeOzFoPwACGcOQ%0|9laEoq=(F{5COhYItzYk}M@_96BJqt_7 z_VHwn@msce>Sq%GS}&UIECU7Fs6QN^jk&svh}JSHBC{7bi3L8!hgq6=_bmloos8-% zIB)*LHtPZe)7-!^L)tSz=06G@d|PckFiVU&&(aF%SI8$S*SVRltNmT1REIM@s2%<% zWpHK2%D<SDEB4Me)lo@tHss>m8$0D;48ZeXx!TW<#ddkwF2)V)TeA(yd!;%;#dmKb ztkpmdK;k1ZnN9ncSVXif-0XJTk8R|4%b`Vh*a7q|DuWT#y!<{YK$Z~iy>YOQunW&M zRcN1kzF|v2<v@S+Oj)R3KyGG&`DLUgk~6lqcVWA3b2#pkAO;^8<mbZj9L(cGyJOxj z#J4f5;ZKzb?Xc;m!!5GOSr+0<&ApXU8KbsXfUeG&lRE85tD@*i=0fF7cmKS$l}7MC zz1*Mw%m&+DI*}G}klfnOyz7zG<*r>G|Kp){dp)N!feS_n&1t{5Pu_LEvC)wmwNO5~ zed%t(`HB77jT665)HTdGy$OstSZh8W^)V{oB7W_vg-n7-tsmt-eNVUE3lmmu0Cd{M zYSIQeL}YgDdAV~g4J$&U<Dou9O#RNp_f~2HW*`xQ+aoq-!$t8=<6HJRWD#MUQ#!s; z63wP+JD!HoSlADN<dss_n7WE)2oYYZe4y!`wYF8F$5fg$wnpxE?C;ER<+=T}Vh?KX z-`!65t+Bs9Z`1>^ZBLom?Kj`Is+rtTR4!Y+M#<>vhkIBv_lE7ERu_yTFr{5{^1ZNr zuNwhAdEs!;+J+0^KY2W6(1~1@jgi`!(;6Pnk^_QeiWgIHNWWv=-0~?8Ef6aXI$_VF zs9y~%H;!I7rJW(L67)5l2aHPN|6%Xl<B~l4z~NTQ)>CS$t%Ju}Ejwz5c~-2sRp!#l zJf%X-(h^UJs3?cZO}Cm_c@~jc2U8R@Q$rMLwn$OwAfSSxQy@h_Aw&e>y}|aZ_UwH> z?;pRvexKd{=tr;ny1wVrb$ze<w!E>9_<CYhza<lp&YWr3>i=}XPr*Ay@b8m+!bV%S zC%CnoNbB(p?MfPFGn0UZ!=1Ffo0FcMRIb~|X&C2>DVK48Z9{#P75pwz0TSaskpo<_ z0lg*TjeUz}GImhZyL4NAdhz3mF)Sxy5Ax-Y`yNdILf9*%pe{VLQf`g=`iv#%{MYM| z8@sDzGl5WYuX*>DoT*Cf20Cw>1-A|L$Aw?)zw+mvzzoDRW?whEU&W=+orEC{6qM6; z^^c6egz;qhg>o&>>Fz4F#L;kY94=lYNg3Gs8u1XnT>U$4!B%yI!-3$~azGHLzJfAi z$v)K2zk0A-%<qRxuhQS(<{ExEyjqQFU4R)2`*9HvCitt#c*>l-@}1Anw?6*)JSR*Y z0!oKjzDqfR4eE9478bRPW^wyaMa=aJTbC_8%o0$Bc4gQ##V=9+#HEfI3<|G2tzKEV z{I#GbUNasf{;AIKMy*9*@&hdnM0wzoKQTU?iE>1PnT@u~L+JY!hN#wYDXaQiZiH7u z8Ntp}3c9#$I#C3)?gMGS^0PW%CrW$Pz2H`8CSU^hm}BcZd}eAwNKjKQcQM_C3{KWG zCAgwee#cEE-V3$K@n7?RcG5%r^E<tJFc%#H-0gt^-30bMGI`8GGgC*yk!<}fLGL0= zz3!2oZT#_L)wA;cT}$R1X-ZStG3JiY$C=vxkQXk&9^AHTk7HYwJMJ0WRdu#$#lgtS z|EY+3esndr*&LACoLgD_)6_i8;XVuRg5K+K)mY2nlbaI|HtJ{xG+}v8*{JB3A93(? zHD0ddeo0m+9CP4QgHcwJ=fi_+l6&^XYbj^G{(89iIZX_jbdnlf`GS73G4>t$H(?C} zFIC(&_@h!uwg@+!WX0-C1<sH;6NkrBJG4E30829DJ^aXpg_?f0LKN$|MUQ(<-`8Ym zw0R+2Qu1vwv;6rSvSKPe_n%G9;X^fQvmQd+!tkQ4NpTCpi~DO<dM;Rwv)Cye|N4`< z!F8>9u5(Wvc3aQ|lqVUt6-~oUudTzwG8$s9pMKPnb=7j+A?#xDBVb4C%(b&N7OU3n z>0h_zdK~r4oBEUKaT`xKYB8T5s8p0Q#gBT8<(6%=4z8P7w#x(PAA9-{sLp=UYiA$1 zS>eKQ7R&0I)?MaRU?+wWnq140`?!b`j^h5_gW&Luiy}Rz={|FTJY&&Bc8iZuh$`(1 zNkjSrd=gPyk4%o<N;l&^6HsV!iu^nqEfINVWn$mOhx+?5|9q|o4j=pa^76I`P&2-5 z(tFLLVNV@y$!yqy^wS?1eV6^4aYfy-4IN(xE_bnDF=f|pT_-#-BGRcpEzj?mzRE^- zxKLtB+J+yP&y=)<vwyS}%YgkNH}FAv@^jd^NL@nnBWnVhNZ&J2iR#0Jx4QWBboK8V z&USm&zdJZz)N^KXmuzP#&O%<d{>;u*K+z;?Ip|n6-oHOuG}dE^=z=+)O^Y49+STNf zOx!ioU{+qcY!#`m)4*j}5YWQZyL%OxzfS4Ok>i?%^U;+8L~K$*Rc%baRC+rp9+O!o zUaicHN%`cTBZItagT3qN*InaM0j!V3u3mWf)a8ls%RyoHFWzcC-RJ!Eq2<uuZSQ=# z?wg{;K{M=zo*#Dx3AZ@y4#{DJuX)&>T8FxdOb;nl3}uIzM{LLmLzGf3u2f!PxUNea z9mBQl{>pq!tvQn*t2O%h6~5j#a4uMVB_=d0y)J3P&0tCo^Ma`JJOTR6nNRj~J+oz8 zK#Y*DX|XXhu<sVe0SUJYL4$P#0*3xYw@;d@*rlKR+vAyS7s2Wh(X(iGfce(o>+~pg zPOYnHz@^?_Y_hSzbCG?=T3-3`me`iNhG)UMhzPPRtF~3l9^d<)MJ07L0_?jg$KO@` zhmTc8x1E-;O0SzL*4-F%IJjk#sbbZFL#l@xpQQg_90VD^Y!aM#>8b|v+oE31F^KW{ zU)FylX1XjT?)vQMiZkD-UrTl{T$YoU2M0E<9DD=pm7iWydKh!h`qjYOU@dk!c|!7= zRpj{PfE_Tl?N6;27=o(Xx%&%kXTHk)sH{la!Ss{95os_SW;DizJ{|x04?`eh<~WzQ z{Kva3s^Dk)`mXP0;~1kkvDQr&N1gCu`!5uG*MAnd;m-31q0~OD)4gymoK|;4vQeC6 zVdpBElQ?^f)Z%drz&5t@r}n3e5%Wi}zoe(S6Gfid&LsL&+AXkWuQgyMzqKL+dzesa zox0K=dSpZQJ5rlqLx){60%w1o5GAySv+J`UBr8}m{X!=Ok#z`vfhLxc`U3l<O>fiM zvtb1>ssArw=MO;cxjgK}@D=2JW~ypEZP{?unK$wE6ZyTuPjI^izdFMWUvt1Ifi2wr zQFd&L|MOq=w}b_5AcU<eDrs3fR>i{rrx1r`am$+$>1aVdx@&D|%Y_^nGOM4vbt~uR zeylpyUUY%;EWcOB>giPV{<tltBc<ra>Ms~D)ntd_jK#9-BkW!l5MfKZLj)<30je3< z+He@(H86t1>M~o6n*-hFp2q0IPefJIf>1>}Vnv-U?3yOZfPKtxLOW1ToEVp?rqt!? z@Ql;_g|=?siN3dE|LhmpfM2{cx8r#dTXGs@yzoLWnvI(HUgTrG?%tb`z|4CdO~Cya zng`s0=*Z6(QdSN=dN5F2<3BC7+OWsy;2+yoBpooae=Te(<qBqT(bDlA(s!1w!{2w) z{`26N1a`*hL7l6Fu<KQIQrvR!cNe20-Ab2+O=Q>tCyN4;i8l6w&tSO<INS<HqbLHB z>3iQ@`jIxCV%tF^;lDl<UnH;RW_H~^D&9G9%44Ebt>4xTaArruaOd2giccm_Ty+!H zC0t<N3uMg~=>Ha9H<~QnT$-CH^e-U)K4pKYb#@otR7z*Ay7Amo`@_lu*VgMSA%e8e ze!ZVP=N-23{&fsJ_(goQ$E|`qbI66Z1i1MybR#|Kd#yWlWxn`DUd7zQRV7VL;AuPh z2El9UOZw71_d1uxOguyYS2hi~U(RHmbW~qtJc_kp{!Bn!M_UblfpQtR=k)AbbXPnh zyP{BCziMV(j5F7|F^C3A{2}j|6QxNP9Eq%tuZAJ-zA1)2?bSYSpeKnHo465Mu8TH{ z%@nLXt|F;V3#)$$*<jZM+>T~iM}zP9MSNy9&SCW{?G(H#oYy^3<B>EwdBrWDZaRsG zK6uBYiS!VUsJ9%q(Vx-C$&MKba*g87xHI_sRlVa*_wWZivSHNi*o5$^tW>!Uxbxxj zf3~;cYl9&!%a#H;H&9yqN8h`4<yVF~hos;9W8l1eshR!)7T~s+`8xKKKYOS2;aM|q z1vqf|f#FW4e`27q=Ve2?umbn4-1(sq%-FEs(Dyp)^N((_^%~H_1Z@1FZ2nRDPags{ zwScv$tNs~vpp#!W6NeF-Cy@0J`u%grm1bff4%H5*Psr=^ePPYU20H08Jm48&0o{hc z`?H`C(LTV1BV7L$n<@;uC<}rCK>hI@K<Vdmk0n6x`kbr33jV(jIEzG^#l@lCu9zRl z&~HO&EcDS|T;>O{ygyYtREU{*nSMq;Bg#rQ!hj4#{?pPjg<-47{ZIffkR_TT@Kvil z7X8E8`FNCIG6$4eR0SYX={s16$-(RNxW(-LApXpbTV=Rw=P+g@M4!-G<(RGfYRmf( z>Vbhhuib(S>-oO6M$MJ$G$K{8Zy9#JrMWi}KL=d90%`!Zx&fkt3N*Z(hO82(gLs7> zHEirJ)d1+7+c$~2c63mL$7rttw+GA^8~Ny0_E$*I9{3jdx>9`Sp9KZbR;t#Nrd|fo zO)lZ<-kh$};{a+{x<uP%N^NB(%aBN|N`wLKW;Vt2xIHvX!{SU4epoEebk#K5{<(hJ zwMHg#u9ZJ`m!v1D%In-@CIqc`nSM^cFv=ApTQ{+UcKy^K2Qt^yID_yh%h6wKz-Tnm zFIWl6Rt@8%XeEW(7WQAR>VJvdXlg~9`Qxhf)U7*)kyNtBp7)6c08qGly<UbYXD7`* z`zsIxr0KdGspj&vWIO829|Ylsk%QVbIspke2hk<#@(r^MyhL^hBc(s{MIywX5v1Yg z^)LDA0VVn_VrC}m)L<O2Up2weadmSZ=sAprrpuj&<(f{cQ@DC8HJ0r~DD=HwLy@T; zPE&wy`8IymzE$^!skZH)sjZ$G{))465G@lk{W4PiN6rix`~k*y1Xlrg?+R}X`*x&z zfvWx_IRi7t$GlyKn$e5FLJ#iz-MfFLamc4Yzx&kZ)=tm(nnGRYLDQ}CE5UIn#iWv| zYyqS^?L*LV23wyE>9jhe>gA%thS8|7H4s<L=l?Tzx}Gy0e79~5-`{Bgrz2p^-i z#z=27#jVIPQ6%!i$vs4MK=<fk(79h|TBcTe(9~ugo|q013>}f9a$_=Q-pS6pADAxS zwg+nKlB_9+=81>u&69Wh&Qc+}nByMu!kKlD4+u5FZ1enGxc|1niUC{V|Lr?{N0U(C z3j%sJjSWNJw-vw~w)&oL;89E&iiq#QQ8;goDraX5M)5D4u6xhll;w{?8tmbpbO%h) zm&*+{t`bCj-?9PAjI5l$zRcem@_r2nSS|s$S;rtjIi&Zk6R;c^xUNMH8jH?(-xUDM z{U1k1qwAs20()P%(xl`UVGuD`cafO5^^+wQp6700m)ls(miAq9dbIzmLvOw=SXglP z+LkXq_1u1?d|lewO(6jp60bAE4Ld^2;F}Xqxouf<guefy4X=(pGj951;j#oo0`=gW zMYW=$zA|7gD9wX1rnv05cPuPzVwU_y#P#*}y0OV?E;4sL@NGljiT@1y@t0mV?r7`0 zR?Pp!>B^DIz>wp*-akJAZt?r({lHrly6BZ3&yUIeKKJG(uN&PMhouYW|8lU_>nQNn zm<RUV*UT^8?%A{jcuRpla(RBp<73e)^MJQznVa?B5eA;YEe=Z$|NjDKr*uN8J$FJI zdV}5m;oA?JeNkh6w`HOm4+B1crm2If&gI~>oy@q1v`Oh4B}6wKr_+kcw8<D5OGHyk zAT4D0cU}MpO_UEXODtwZ(0Cn*=SZ3?e%kCo-CJ=8y?{tY;zpQXp0EDfSpD^p1czl> zhE8rBJ!jm{%wT3U^3}Ji@et?)sKS@kfj1_o<AVMd_QK%Ul^Rowl+W+ub+?l?bRD_8 z(|obrZ6-$HkNx;|lXXGt%Dg+24-wYC6xuX3+D}WBtE4_Lt3y+lE5c&d)_cW=qN*E2 z<U0c&u=3T}r18y9PD6BR2(NF;Oh4YqktfoMbmFwz^w*o8@TVIwI+#sMz@|}$2k(mj z@Lu+j%dG8~VdVjOMOHeZxMFP8Qs|<SG21NA^;3B5@78WQkwUJ)g-6jz!))LlUscU4 z7ET^gw2_Vy!<S<QUT<z{A|fzWsSx%Ajd0{LVot2{cspxKpjV=4pvAkSxt?*HNd$Tv zDj-&>?eQE1EvE(Yr2_kb#MsvLZAk<`$+RkWh-H;OID!hBs+mPtrtuHl>l?S%w9MjW zMfFRgX6nhU__Ouvb~?J65QZNfISa$Y1;swZHg%jpKx|bleG3}IO^qNOD=<>ZXg)$Y znSc{kik?jdP2f3m^<sx^*e%;!I+kFI#Xu{EhFw@U!cVA5nRrtYXA;kuXs3}$$o`Ig zJB15@?nd$s;Gp=zU@xW32Lc}!LEf;_(PigOM-b~7q4_(4!$ZO5auiDd+$7j;91Eui zwO!ptjeYBj3bB>hfppw#8lvON!s2b#xhPs;AwMwYDZ}?bm26fL#fSI#$2PgXjpNuX z6z;AY;w-eQc6cqgaE~YrRrR&uO)&*b2KfluZzrmn1#{7L5yu`-9J~)Vd3>O_qk9;} z5#*QW&yz2lm%M%{bdaz-2(&OxxdZeF>Q^37T!8Hu6K9ua9}^fgl8rAb)bP^2cn7xt zbz*B6y|l^4udq0(tu_RPU&)a$w|RKg&bc|>V}-v)+gRUjnwyC9?~g=;6%=44GUsEQ z#@N8H*s<MJj4qVf_x?G-S|8U~_X0SktKZJge1$m0k8?J~I#BgF7coA>Ojs2@C)H43 z0;!N+EtI5JhVh%-tNpA(yijM}w@-H&I~?z@gtwYIZWLJAIM&zu3K@(emN1UdJ>E4( z-hu$^ttijXuzP0cFHXleVEa}$)-SEr_VS)pR%9U&xO=vH{RbpuK4Ul#ri`?zD^Y*k zw9Qe5ldlrWda9e^%0W95##T)UCAr(|LNIY-j}$3BVTBcp00f5VP7guwhL~f!&9u?6 zelZRTWT!Yp=c=3L$$xeDRycJpd+R9r?B9XSnc&WR6&PEFojvogZ!iygJt=0oQs(zs zNL}gR7(dZu)hhG*7~0xRG(HY9UtAu05L{lt_%1)EjVIMJA?s9iQy8LO<PavT<qT|c zYTR9yuk;E7+4%`<PTG<ss-G{bmR_w!x+z-%WT_)exl}?a^9Mo7C7H<1@icJvTp2c5 zR0g{tIruk=m>_3SvwZzn<=2gKjva{sK_QV`hl3O9zMg5<)?uHRE0qb0IIo698TAz; zY-hsl0Q>riZKT)TXYaMEVPi^JkJ{mKz06N2w30SM3ZhSioU}@7o06vnB8n>|6c@BQ zv6;+u+yPnSr%cXQs;J>o3RRU{PIYi%bRtzQ4E6a@`LAZNCqrl3GH%Rvd|=4|!*-&t zaq4_fOi4-KWW5*~owGFu??Fmf6lmYr3+2=%qpqBSj@o=7gqHw1?Tscm<Qd!5ok<Vg z?r}agYE>|ZsCJmLJZ_a1I~?ht!I8zN?+_Tfxrgm2Qhqa4CFi|pl*0Eani)rG%A3nz za%k+nyhP03j9vluN>pn=Oh!qiEw`Oi9mBG@hJ$$q=VLZvzyc#;6kqY&cPp>a3mO!g zi0yk`sGQIfhr36Src#>rzR2f5$CBkUY6-)$J>0jSS|L+*!ErKe88@i$Z9?_XS%L<U zPr4muZVg+-?+!dK3a7lKR?DU>Wt1DU|6}O4x!m=n_9X_P%udD}tt4HsjQu8Bz;EM| z64L$h@ec|F(sAxv`yGx|M`G@`i$U<>3R&urxq(T-X+qyV2i^-Z;Y-f;ILI6rEbNi> z_+dXQZ|A#?xUkBn>$znDo9$ihrm_pglu&3&?=}FvXS6W<C&dtZO*v+2Nl0Ov`hF|~ zq=gAMp}-EyjoHq>vumyEy*`T|dT-;_xS*KBl}8RKD2Qg1ptf(SAT*XEwRo^~8%>fN zbvtaJ5ila#NStn|AjbV}#jq%%uE_^c9PT>Zlc@#w6E(OmJKuq_JqPPSl5PJ%1nNu} z6=aiJU=#ni_8D$q%ow55(_fwUz)eVus`HDjaee3(;VPNj19zA2>D`JiEl_g14^Z^h z)$z7K$}0cE7=w=TgM?tQohGQ#WcYY(p%#oPIg8GcZg_wFIDyT_J}5KMD_@Ha8se}z z7Jmo*H=kNy-pl5#J%Q!Qpb3PAsT!iUKjd~Fb)mgugg=7EzSTIK$1S5_%~b{9h|1<p zRa*Ve(KB*CY*W9t<A~+!S~^&kMlL4$Gbw$)aLU44MVwcizp5T}gc0*O8!!WvBK2f~ zpZNmV#?vhYvORtndERtC$ZIa*ou!Gd`B_&JA8%^qHWOr}Jc{U`>>zP>%Zifo=@B7T za$;Y!gPj2GdcaD_>)o`aUz)IyTVkF*wo06^@%UUF+1R&QgVU(G1qHlpPe+t7!x&1w zPHI;tjqULh9B8cIn`@h0MhInvd5v*Vo&&X(Y2zqbpmIMN1)r1leOyF0Yj}&Oc_+`R znju9OX=RshTa~X@2OqD-BCNfdvXi1$x_OfaoDKlsFvQ8fxCz7ZIHPL;P;E(kJ*daP z<>4c{eYfTq>qEi#f(>V^<obWV!^9s<0{$(1&>TyD9zNiYXvxl}yC(y|nv%kM-VitZ zx>j(YJZfy!>wE2d2HrO;wr#>G@inK0SL0BB-A>8{3~K#|GQ)$^)v|6!9N#$yGhVBl z1Eaq<Wv^hh1P}0G%ydt@9U@F7YE4))F_|j5_i~o6Qdev4bcGe>Cz5+NrKBL{vai#d zugY93Hv-ov2^0WD<+knOL;#zuf?D8%>kBR}dpkMQnj?yjpOCp&0HX>Gbe9v`#c(~& z1)K>zQh@N+${vZjHt!!rHzuZcR|t#<yie@)8gcey=0NuB$E`!s5t~O7+Z@A>#EdZ1 z4GxzDfMMy3_oH{w@a96Fout-3aTwrxTQmErTr@3Ykq>B%$d+RUZfM7wGDz2~VsCbm z%mx${*AcxHqU<{i=H@4@{~zR!yWMP}C;wx27z`!{n`qfPbXq-kg29b?@>e#4_2fT` z=+_g1AWOB}o|7g4;&@3ixi*UbtUR1y9N_I3FR-cdM-;PnH*U;nkrHwUychmQVxF~g zo8P(!0);-7u8!$D`!7Cvh%%S?jP`^})h*4w1<2!q?!eEO?mn5IvR0sZRUZ9ls;&hj zTjiv5?^2}>C|FN#*g9fAHnk)1Z#iH)y#YdQA#wEJg@I|Fi0Eu-FG|_@CkFD7wp>oa zOmY;wL*x4R>~oHK7xv@3>af^tfLj^uNl5pP>1b!1QF&pi$yvxuD_Z4|6RrFZnW9S7 z24i&}pf+p09S#Ts8C{mHQf3s#ro)&ZhG09jZ|$_Frjrvq2X3g(o=1%H#vKE-VD!qf zAX~LVx4%k55<0L2D5`X-V_3+nYqo`}WvZh<>@!e<?DF@P@@JcbP0X>lE_b_4{W;Sy zD?2cpB@!Av#~QN-A}(*}jRtqo7U{Xb+Y3bs;w|^;OyZr(TDl-@9Msv|n~5|=mKQW; zWocjYLujjC%(Zr7WHs3siXdxLC&)_)4r?TR>-{;-Iv@LbK#Lg@eKr2dW9Y0&Vz}rY z`Ah5{Q5$<ElZ6g`LK`OucSZ6cL`!nN4Fzq7DQm3mQv|WY@%S$uiIpZO6VX|@DA<bT z;BEq0<ZYdf93NZj*BIH;El-WzOT44y)_P6m1D`P%OmIuf!Z=Mec{I&__Do|wFF}d0 zfuW&H1aj~J4@mvO&NSA0)wCN2*CWnHZ9E<`9aUJJb&TvB9lJL)HlZEwn-38+iBnkz zDw~b<{2-qu#QLHag8g;TDcC{_SC3vGCy0vj44SLstq5kJaD~<PY$;Gr-Uh)if_UVz zpYmYLLx%dcRypanM83NZ9wMLOD^lCSh0OByBbQqF5SgOQ-#uBL+T~tOs1N~Xo4>4J zMz8!q{IEBg>IT5xpp?zbLk{m6It>W6X=s5tATd|R9UPbXpk9S=AD`fCbiBlRcDK*1 zkW(9lU6N^zm+et0MeU>tOKle5W8p0_fU+_{d22nN6wa>o0Md(t<`2iH&wb%o*ChM; zC)Jka#Bj7(7<h9|qd3*nRe-pm6p<a4`h?x{SHA8>y^N)IOIc@Oo>A^#1o?C;_l5^U z?8536cCfpJugFO)>^{_PIF^t)AbNJ#OSv8U@sJBnzCE*3rH6g+aw#k&h~7TDs8H|> zFfLy(j8n>^wI^Tkp53X6Sy9C=20~DF1@^&77<S?TIqNC~05;<{@>mZ*uT@h?mXtn4 zJsHND=x_XVhhw*~9z5oyZ~*_960Rb&Ed$a@fx-qRXULcKx++voinan4(HI9~!GlvF z%XM8;(T|4G0g+S3QKhXS&86=Z=|%bCG8n9kWHIgE8oxLeFc-EDc;iX!LegCppI48l zAbQ3wAn0%q+DTl~R%p*8G`rjk?H@l7r*@Eaq8@}mQ?RD}n}S%vN4#$w{lo4#JQYZN z{jt9b9em4VZf#-&kkXR*1oihE8*RPs<&MsH7<Q+8#t*`D1){A;L@D_mQO)CgEroKz zovO;XaJ3wCJ<$Vj;zUlajV~CRGGWb2Ut$U>pd0DQD5-@Z1vRAgm&EbmfWlp`_a#CF z!shzQ&O-`i)S^&=!`2D4x>f9|sT_(*aTPjT+gXtv!nqqp>bsfG5x+QcdEybgWh{jf z3!}sdV@1@ll=|y2DSRtfX~i%L?jfMY(9H*^cg<g$%(al!r`x0xYI9W(o8P8RP6;z5 zGiCN<ajL~}OGjxA;Dj`}i8=$UKwZ-VG}Y(vP%5}!@HS}@3yAah_Ctd%dL8q!bBWLw zVj8a>xg6gg>!WEO>Wq*lSgq3I^k{)&ISN3^EMxlx0I5wa&w(n=aT1UmawfV-7DV5T z9pQ%J&6>fyc}}_OIdx|Z5HWa-*#mOc1SC_1o54D7dBiEhj3;ueJ+yD19*sW+M0#$6 zd-@G9JOC;`-B2LYz;+4`t`c2rLG0b7A0Aq>#MBD8BlEI1f!{g&QQ;^cpkAo}s5kC_ z0qlB!7{#}A(X_Dhv3_*?5MbdQVY?oYW^ryN@#GI-xfD)I=)qpn(_rv7OL>0lK-hAQ zUgF=Ia7!jy5FZB0(GOY>S`<!A=%HL4YMLk1ti%_LnRWD_A&$*Sn}6Bll?(u~&-oAI zCtaWF$HyC+sM=3M?3Z)OwdwoImTj9ai=lz^U`r--!_3Ru2l3Ct^&=Ood2=m!t-A8` zBaq{K`KlkNroFJ3%osG4C(p+O^?*$;S-a9Fps<s_#Fm>iUr|#6wExPRzhd0nQ$NpO zd$a@AT5xk0fiWWes8FiM2B@;Bf%;O(U4YB)jVgp4En=OODK9dT^{fOcZzzD?o&doa zt3>I?sJjhyIbz2Rl7)lQ58}l42kCh>@>F%oNhHtOpkFsWD+!Q`CmTXZvpH7~KKI{M z<=QsyU+<nahjF^#lQN)J5(dNIlmbt20!{Sjf)&=E%S(^y#RDko<pSjEQ{=ZQywVW} z13Ld1pPi>5L3{iAvQ_U<@DPQX;AFQ<KPvP(7K>FToD0(Juv-$E13<7!w0NG@*#NDJ z6828zw*q)!5zdB5uec#(BlQ**Y!3$GBhGF8%xm78xoD5Qm<tG!NT45mal6bAj1AUk zgTW?`tn>gkzX$N+yoz9cL>*$Q-4gxq&|>V=JZcI0kse!4yemfn)rVd5DpUQKo;@0~ zm4H}uV@{OLm}bonumm#xU6D@ul_6dDc`BWovEM|ET3hqqGUz4e)?ofHJ;$Ti!>}5$ zyEBfK@syLZV15$idhExS!T_Pq87rL}e3G6cVComVxZNM({)hPB#rI0A6hKf4@i<dz zq0T}-=FvfYG?ZiIKKt~>81!3E-`l-hfMeQ#e4QI1TrdS(=2pZpf5SgWQfl6ZIke#O z2$Zg^E?lcrG56Mi^m94o0=0j%n-K!vb%|F;bv%Yz9B-nwyW13lNF!LBK&PhgS*T*A zT|C&jMJuz1&WqnMho!&u?*{IFGEnIvW4FaZlO>^gzQxQa#&x=v2U~Yn^A0NE6ux+O z%2d%>hu<!MQjKF;2L6&ZC(}AAb&pt2zEl1&&jGg#Cf<6jmptgA3M>O%D6Log5!Pw# z{>=U@itMgcHD83DAYJ{L0~7=G1?q!<I#G%hVMiiRdPFpAfU^}DUr!J`%I2hYoWbXm zi}rzO9N9r;{u;=S$C-DL=qGn8L`dr9zXa$SG=g33xz_;1qZQW1MjpYQWe#hbr--Mj zIr8MKrbbusZzX4JzjZISDU<)j{lcy)-^S8lYhc}dc72o|phx$vH9<S=?j}E{%Ch1n z#<LpHlW%kQTDfb5HVKU15yxB{$ZT8V`K|P@kXj*{cq;FkSNGe|D@RrqDPow+j@hPp za-K5YyXZHIF5cCQ^g0r4YNGlQmn+pJ5Il4kEsd}(4wJ9+>rY;@=ZCM3-bqtvN>e0# z2AzcG9l6Z2i9m7XHQu&yGy`-CbX|HUqMex<_Vv)Kl367$Q4>34hgLxCi~?JeWoTWw z2WI!=?35k4#eVJUCKhTyqD$S;H@pW)hq))IQh?5~<c%EYP4F^B$#c$9-aQ~l(r+0^ z>b$Ojixs2Y9kY7+OUOTOIBjxNa)*f?@(z(|?LTSeWu!ktDihq-$W#|H*KJO)(HnKZ zdNlz%0I8?g<yQ0Df8?o8WP9|wf{=4NHz&^$ho((lTl|MAe-hV_arn4DGt0s@X;NXs z<o~Ft3LLBVC)FbRpE!5JqW@4u{#POsxE-^J_<3eJ<pQ4Yc$#RkPNzG#$}@+Rs@O|% zw|>*C#Wj|hrt}3~Ub{T1y6;piO5o+L9CHq0{*FWwgMnzv^_tkeh~(|`>`$xR8M6}E z;DB`2Sz2eGW|u?T4Kjb~K$>zg#^gV=85XG}rEn4}rRtFJxr;0-GRMfl%$L)|kyMFL zj>s}f|3(TDp3`gVH=xE!-Lxri>Jn3=r$eG6i-s4`>H-x~pq4*V^FuKHE4+bi@%~o8 z=J6C@D+Ae9pP{FhLClOl!OYY_VvHm1sDs=KGZ+1yN)*X;rKq<jQFp-<RceN_HBQ`E z=F8H}2FAtC(Y*jG?bXP~OsQXsy~G|Sr`a~Rg89jJkA+>)`wmA(M{D8HKP*@jTblM+ zBi{5lGUg{@?KFI|!t6fhw=2?bv^8LcqNI!Cd?%7wsN7uU>8mTJ&9qkQoVEhfk0z-h zTB%3xY_cLiOL5;+2Rh$x5Qd9sL$^t*Rc|BeR1wBzn=?!g21@T(m@I`|{9-iOIP>T2 z&CAFcKk=(LO{8t752NxspUQ=*^869ZO#@jdD6zFRFhobx9+L`a-1K04_d$b_?_0x8 zS#zPqTV_x%wa1+5wYBaKPDuKvUJL@~XiLGEJzSW)K8>?G#L7MO?ax1p+U432{V^Wv zt$@gG>i1EM4ydGW(!p(*Mp0_9!V4A2)>~&@2_P=74&qnC3r-IOQm1xGEec|5r%gsu zCj{atpi2w(HO^dye4?ljwLXQ7AgD<}0Ck(@sB$KRKbtHcO6@8?2vE2mxSJ4|?qXdf z$?_4ItQvUwV(?~$)Cw(1=SLgcZ|Vmt%-rOZ99p30iW?^09kAn!Rm`hvSb?UxtGZ!e z11Zy6&o`0S#zS||S2v)W;k(W8)_}RQ$+FY+OE>9oWny5RFtC9hkB^P@j{saGxrtkT z{^-fC;JFT{uYr|Rl|)TF{>@l(Y8D$>53W=6b#mk%v7CoL#~-?Yi(RcfcyHFUp#Hpm z)9#h3uW@($i!^pLhhF8|M4U$fXSAsGHZ|gokqHSSsuoF4c0Bpvj@qv!12_`;+ltz* z4Msl&3W383*19*fOi?-@&UZkwHQE+;U0508EZe4HUb)L$&z?FLv?VOb#^h7u6{KIX z47R(mcYVsY9kZJ{&l<=d7**HXta<qdXpP2o?~Xho$>Ez0`IE~4*^iWA9(|{<w!*s) zc$B1A9nw=hy=+dca>Of+YbmaUK-KBhD+9!X|FNk1dAJTT^Xj!!+Y14q3^f02Q<yK@ z<UVXaZk5cfmf#|q8Mz`pD&2z7%uJQz-6c|ie4m~M6DOcGkvg!pO{0X{<_6>Mgc}UW z#bt0gmt@|1vYV;Mk3)(_^_Cc#vfc|Gdmr{qnd9`^!)1j)FM4ndh&<9etZn&ObXiY| zy#YE&n9f8(SWjYP0kiF$M|90iqo<k8w;xfX2I;gJ1^ec{8xLFpEo6#Ka9f#mHIZ}s zU=C=5UbT?HkT9D%kQa3RMP+y0x3k{fnHfeIH?k+Lee^vpcZ`=O+71ZYDMw&f<8^~- z^ELuUJ2W%E*i7;{<{0#2w83U9+%)=PEcC%70XWnEJ5koC7t!+(s2j1ah9KY#@5Zem z0v<6js>IC7tAC_z!kxs0j9KK0EL-)Vs#7U|0cGAfxxCfKD()fIXpXgqFfTd$nd#Tb zpqWng_mCT1{<!e5&|DGzO^ZvU0I?i+b!I03ockFAmvMmqX@jo1DqqtASmB1%Gr*QD zysJ(AqUOWV^B7yEynXZ1CgB)pyfc~4KiTd5s*gpsVIFK9<K>I?0#iRq0x(FOhYI+{ zbvDUr_g8s|g%j(nsD<wh=rovX$#X+4Rl625U>-Mj3i_#?S7R9Ns!Y>e2@5LY0MiYg z1kAyhyMZ8&76Il_5c*Rfe(txzGJ_jhJ$WOM%Oc<J@<NnVj#(6l?DZf7=s{r4ukg6; zTV)|^0~n=bY7?MSwqD8pxLlc=;^p8Z0|5rYoQnW2ZAL-!#AqDv{Y~8hEa9Q)B{i&; z+(-;$Id5h9Su4PQfzxDKtN=2K$2+k>KX@=XIfuq7Tx<l0%KUe$lOr6&x!vIgTM*Ak z_-J5O>x-e*#~0qARO$iU;{+rI7i&ZJVgb~4&!cvhydZ9#{{LOmC=<YCR{=Ca&#MRn zuNrr(8EhRu>k31zCVMqajD_ZjY|{0z&@(@*Wu8gN>w#e)^^bP$25sB<$CslGm9V3F z!k&r$1Q^au=!^zzLHWFfW{)g%Q~wc2i%8ir>Qf2x#ih%d^8^Te1(@hhECxbSujKrB zLB17w(bs>O(pozn7;-h$>m;DLJJy*6ydz{jR9FF+kA6S|fOq5R2_JztfB<(+9(`4P zLR8`+8bO>Rx{>s>t8cfLhw5d0`@F30vwf^o(BpI}_80RKD{HPJ{WX*dOj})SfFbdV zHG|%jq{S)fIY{iznJ-6s#!4T&-G1=FU!!jTqjB$)=sVlTrlaCGq<S<5GbOJ6=;EfG z)m<Clwrz68hQDw$)f=n|16%i%0~|}0_bVH<y`g}S-MQ!+uO<GXUtE$$RA-1`qC4-c z1BXWdvh%_~#+xvJg{<GS2LBi6A@5@|g_RpFk~Uv|NN_o5v1eD6o&&%>RAlvkXMhpd zEPx1IxOwWX0%oZHR_ki55e|};%(OdF6&l<{=UvwlTUgGwx{MLj9)Dcgm_>K?b&t5e zfDeg@i2=Z{n3qcT++~viwzJ}XlwM65Y#RxwfrEf{J?7*d3Ov~|Ew+e%UjEq$IPtu6 zi&a4(^u2k=&j7(qJU7q19Kpio{AJUX0|KR<FHbx#j$MV!^%<<V*%yb`fLor4>%k<I zo9Q^AYbQK=9rbof5SaRx62uD3D0{@Y5AEh9d<B4<;4-Rb>!PRaD~>f)`K((Vpf<{= z*0cn?Q!POWIUhQH8X!o<JSvOZ7pH7rGwz3T*;m!KVk*-^&0<e|sR(I_@C9D>UIM&4 z_u<Qc_}vlD+ZNkU+s7f~Q+)rOvwWm-uG~P4Y}g@%TJJf_`MOc&8q<~f;G?996VY;j zIdzf*dv=Q?ck|iKgzDwDI-!%-Za)opUZ7%cuqH%6nt(YWeSp7_sZMRkN&Zej{M~V2 z_K7@P$iMy&t>`0g2DeJrTLp+R{J=~^^M~Nw1&mo?u7TO1wV@Xq6P}>YDTq(}Xl3hx zZz%%Y7XPPN9pr+U=x<e+5jZEQ+U#mn=L=_W9b){$nSkslR}wfMZX7ZI<#AUZbTKnO zEl0=f4_NebYUB5){3oWEX+kw^`*~_=XE(GQ>?yLC3{;#XagZ(3a%tIdE$+i<{Q!o| zG7Px(Y+W;09a@7Ym<5!o!|1IU(tQr!<)K4bR<lz?lQq+}f7nBkRgNKl>3#sQmYE-n zPgEPYw_pjwz&+>*saw_lR=rDN^F1Q%w;+bAt=%v-0=>@`ryeRi;Z7Hm!Aw<s#t+r0 z6Kv=VPx?jKb{+adE|>wN6U}qM>FPY`%vls`XiqKMH7Ao5xT*Wa?Uq%&`;IBAQ@a|H zmvdk$9%O2-e~W|I{v^hc#Ei+~lz$+cY6k$>f=ITKB5T&y8%y^a=t!FwPz^dVGgs-w zBxK!B569x_NxLhOkFeu~4Fk?w&nkEX(JA*eTuvr$T6_zft~go6IRmC~ZAp+1L5_)J z+Y~F#BIZsSXtbaUFv|c%`S6%)DIro&S*12s?UHNuNB2HD<pw-z0&#*_F<rkQS!EF? zwUV4ZS|m`!e4w3FF_20DR%)jmDQ<qL$LmH6T51KAHZ@n1AU%P`$c*&g9r-Ao3a%Mq zqU+)Pa7$J=gn22|cKMQYe-I{?GHvfR5%w0*pSB~;^8<*Ja*7od%8uEtkb#VOxwyFZ zxq*qwTd9s`(h4K&TAx-KXN^HE$5I0nK3zaQXvKe7abW@xzzXMB(Nn)mTlb%cm_t1S zpJO2SNCagb^NUcxGXT;d^^8C@Wp&I-h|OX1MsPvoHNCa!kwRmh`$=y`r{kKdTi8_D zrX^W1Gp}V5!7x;+jWL!E53QdhQdf=?^~vwHKxe2XOUv82eeH5HfN!CM(7lZbSt!U} z@oPX=C4q>Ld7=E1;E<CZ+TTf#<yR!!eHXl92JNGrKEs-S?E^1NFCTJ6h}yZY+qsyZ zb1s-~;mjRnjyCITtD8+f1l<mZnChtNvS7Y+e${v*gFTx&83p8gkrC#g{_^C4u?JDo zT_n->X^oTi%RdpdtN$i(C-;c`e2W!Xj6eS|xEydc?TaaB#}i1v<={;d`Y1O%0M8xM zk9l=tjJy|F81LRlL^(YoH0wTu=q$Tj9?4D)Lg?y0kYA3r@6?pu39&h+nrDkS4U+J% zBt=mpo#t*mMn1hO1&Oe3oPUF}jnh#b8V<!opv)rJ$yymGc3$3`(hV9$l)vp-0({`g z$&Bhvo}D%4qj#Tar+<xNS(T@D^bK<cJ`ZJum*!Bq{6MSUb86c;JubcxD5Xaom9pLX z>W7llgw+SXEo%=M^Xp4npS1_x+Cf4o=-n&gXKV*6t<sU9iH?Hdo)1`CO=+u&!vU;~ zR5>+#i0oXIF)hQG&nndiPX`v&hoPnj5N_&#MF_BU<PjJ~Zyu-~q98K`bHA#CL2)0F z5bnV!E)2#?V5;WxA0Q4yiD6~s%T0ht<<t&Nhd!L#$@DW=+nQfPFPd8%9;<irU1tGQ z>K_LYi#umGDNdHnM^mT1B`<Jv%Ik*yEzSeR&pWj%2_MDAZ>iwO+|C+evGDnDQ9oW^ z@VfZ9U?Fet=y8$&(LSg@{P}D?R@09+#DLq|7e9J(>kgP9&wHY;9RDIH>8<iLj5pZr zfV^e#>7HIK1){Bn&~Pooz79wf^y3YVr)tZZL8D9qLhnbI`tb%w9dyk0v9ZaQdW8NZ zz`qjkuL}HY1O7FF|9XLc-Qd4*z`udezmed-q42-C!2eyzP~k2oO(C$0kJoBFW^L?d zr*pI{_E}>6(41^eH?bzfqcGK$>Gx-@B=jy)jK)^>-U|Mez2m){5c1n#jZpx8w!0ks z7LSs|#e@mX)RDSb{x6a>SQ|LM{jYRQwGt?UV57nZ;|qdxu67%7$KT5xJ(PpHGQG-U zd(5o}?f5|8NTTwYhqmnvy23ZKgRx_$HRns#@KKgjYbw=BL+E+a`G4hka&jc9&o=T@ zq3q{0XW*>bQ4s+6f3sG6T6|IK9Y4o$!Hb<3aPKCkuzu4C5aKPK+N6+sw$l|wR+@U6 z_AO1*ikW$9W3Ay<4~w+{SYNSy5Kdj&`5pqiI#5jEUOV7e=2;+3{WI;HgqfPaiW0{% zS}d9bOhR8#x5HL{g*^pP42@2qv4qVU6>v;6^|0;<=B+iPWAems=Y0`6RQ^{Y3{qD4 zas^dlp8OhRn+WJ*9OM1({J=p6AkA-)PNH2EVzm6i9FsP+<aSnZ^{gwrHz4Md4th<c z@8@m;aSqE@5!66e@I0s9OV*k0;@3&eA{V9G2(euS^g+5Lzn6Mzc<HScn}Jj2jx3uc zWpXrwuDG^g2>9FpsSMM90aQ>9a!gdR4lswpQzf@j-b;~1Ws38Z(hImWtAs$@9$juh z-h25p)7`1#^s5T<Ckbx~JUEH1M-b&8BYzN)zI>^CE$i;`W2E}N1x>=?;=l1ya$FSR zOmU6S+DvTw2TtBMaB^-W9pgF+RIsK=y3VV3mJZmv+6}qjs+p_Qg2DG#M25L;#)e6| z(HM&%NU%fj5JHnltGbHl4HmwTU2u;$6vWw1L#K?wp^$JlI6lDJn$yJNg$L`@FpP+X z9-xM^nNjJXsPDrmw($Njym3yjE-eLBZl!DXp=c#UK3CCf!7jUvz%+2(;g;oSS|y)J zmluh@3j7}e1+tfKva2JMv^?xR-B}Q2<)>5kI3Mo(sc*bW=-L|WZ<#z=b)KUkWgnxP zS<#4Tozwy=4_EjB`{>w)03F6P$r8p7_s7)5`lUsWp>e69LTy=6;onJp-1wAiPiCHH zqXyyMT12J=w-SJMu~pMDWwL+z%^|7B3>1SB@2PH{fUypcJu^Zx3-FSpY+8O}jMFB! zJ7zND9KY6v=8l?KS8{DU(QIJCyj_5LZIq5^$?z}_@O)B1^05kFUY~WHn3cLlfFnjx zSehez{4~5D=Eo0(o0l8i^rsFa)k|F_w!ddPV9eQv{k8I(lD!fvY-e#?eN>Vk?1kI2 z6L6kpZb#&im;?FphZRu`4vyX>l@XT_zf?pWs2|a;)%su^g`tm1uoPBIRWTS$^;g2& z(eU2VtWHZm?*FO?2m3Mfr#yPlo}ehfy~>J9eH#{$1B`Kfr$%|kBsuX&R%Tv~xVDw# z)<qN@98h{!aw*XD#mxL!KFqVtV$!yEy;GG#q;R4$JOELAR~ZAAR)jZ>HCZI|6PoPG z9g2jgM}#oW`S<rS@oDVx2@@Q~!o{@|+qRrQ{)2cV#&06k@0`8;?UN7o$B#}CN*Gpq zZ}%lzo7zE?3Z5n^N>b367B~?CowoFQ{dX+xKY*_4xsk9U7ic!ym5bTy@n!62ATPi< zn(<MXF;%w5zh(O)RMxlooLel=1xcwf|GHa!#Z(a>&`C4-^a}NtKx3ZKqe`Ilk&3!I z!J8||(bXV{<9$CBwbCm-ikPdLW^J5+e&06-VP0)UBRaN8a{)|E3d^&@Sd;KE=aZZk z6v{iF4i!YOUBGG_QkmATovs_ZW=!@2VEwZ;hm-{48yLN9yzlOnq`f|mHD`xz6>Vn` zvo5PTZCb-kBKN}RI4o!6mrCJ**5c!qG?cq6-l%}HA;-1ZuFLthSpnzt7>JZ0ad1qp z;s(OF)1NaYQaQR#-z^>CQPlUUhG@x?(<Zq|#u){9o+UuTnmJ7tCv*iD0_ST@cv1WE z3fs6`t+tyIQBFNR5{iGzsT>-boDuc?P}C^z845H}3A1w-wIaXE6by%S8XL>Wdn_hd z@ZwZnCDQf;40}YBL1T5lnCVd`wHZB)irP5B6TWOZg5XP2TSkVRlP$A>66k-upk9*0 zx`LBBJrivAD$l3+@fYrPbnH-lZ*RY-ik1s2kBYVZ3f>#WA-8h_clQ7ANZj-t>i8es z(GB*^yh-KQ42%;Q_4*Iq@&rzE+~^LVTnv;l;|U2L8L&(f8g2Jx-162ow`W0Aj^#!m zSnI9R&LizdPW#bu=F!F&n;cjBr>+f18}SV;`3^EPXC3+oN&y=vXLV;Zhqbb|Id*i0 z(U8O)?r2bix0R^k(nL4kd79eE$GS{+qo^ZIcKsTl`H%iyne?6e+zFFkEA?bqT!^jf z?-rD2yDK7kQFXufU34n->kP{&wV!AjuPHS>(%F!WyDh904XiEYq)AejGt>_xx5FCi zCG=)vZ*4XjFn)%OVSPmy@apkVMTuQ9slW3+#*xeZv9glB(gMK?8%LfXqq&0S>TmLU z66!ejma!uPCx4`RWaXl%R0G#fx;mbzqSs|Z8t+P8qZYMpb6nkp%DMmwtVGdC;v(8! zCETu61|lTXxWdVR{AvffYvl-!WZD@rIbH&4-#^g}ce?v`_tzTyoQOEob+({mBaD3< z0$f~zpl*e??I!S6I4aVdp(T{i0vNf%SHlRmcD-GKr3n=SjYB{$T54NNC->M+oS@C% z^ke;P9|OLv)GV*GPLt*zt0^&CEvSj&8#Og+x(Bx{@ycBOL)2qOP5+nJu7a=JB1}H@ z-D=Ib+gNXea9-j5cCGhGnMF}ksJ-KJoElfLUvxsnV!|+DprIA$`0XL@hDrL_ege5G zM&Cs#jz9cfP2FH`GU~385zlMmT&fw}Q<W7A(QakT4_I}7Men(;l1k+3w<r7>?MZ=M z*QQakLp7y4eZtB$YHuM_<qEXI7GPHwYIn{xe5hhW-MdCh1$;unDh|)67Fj;gy{{s_ zwY=hqze(dgqsuVCT;g+KHCcvsqVEntpu`Ov4{RlxZKmz2G5bbXGntpL&rxadTzy~P z)Z`;3`?9dxbnI@}CMoxB`tO>Sif|F1uP9|EY!r7-$#<7Eb!`0~Yl2{OyKRa*>djAV zBKYML!%8)Xu9ABWE4%XPUFuD=+OusL%>iJNTavv7Q6FbpVs0*;6`U2eI|^+7#+i;= zM~nefdN&9z+UUUfiyn}TzbB8Lj~a&5Cmzb<>?Em)3Go5C9XXhD2z3{EY1~+?Q-h~N zZ+%pJP-{z*)dcjXx*>j1n#l<4Y+KFVIQ51ce-q>Kj3V*|_$bg~EC7U!m)!zf_fVce zKomEdl>d)DiFWz{`n12Zv(jX14&n_D&Nsg=-a`kAP<wwJVpcZCQh=UZ)XsP-3_tWM zDZ?)o3T)c^a_{}^3XzpX*`Du8wk>f}$)iStPgetHm^N^|zS)1`s0~dEAKU%|;Y+-Q zWBo_sysq8fnZ+K$QcSafNIASxJ#{U4DF;<*{-(eoSzQPU%n-x?_r|r~I&E^O+_vuR z45CHWg9!1B8A}zwnd11Uze`v`#2$Oo3dax=pr_fWo>aORcM>dh({A86M}E`l7%BR0 zy8B+l2}H350erQ<GZs4F>&H3xr!Sk)aVeNJj4pNdaFz>4GH3e6F21hb>+_>QNr~l$ z!gno9ewMUh{Mn-w`+qwHf41=8M{%bYUitL8<>{lVZb$TfeBzVz6;E$|vG!N=1O7{k z3td3-xReZ&?v;0Rh!@BGQeP)evq}N+D2u+zTQlirn(2JQVoyy+eZ-*ineH}BZ;T}P zu43@X6r;nvW+z`*Yw5aKkiI$xg$hQ8iHqq91h^8EsYo6xZh_%xDAu(STE@t=0_qZ( zxZ!QWu<X*6Ok^x<ediQha+XciYDz}LT{(&#;Q$HAm!q5-+{%|54k+a(3;p7Nw8Jr% z?7k@(Yj567mBFn-sC?C|sNv?_q`7`<&wbfh8M6*~V`c*cA7)Je0=B$r{Xu@S_GL2_ znXMLHChLNPC-+W{ym%;%VW+d7*<oo3PS(+ZrWp**ZlyzJV_Ls#&gr2=15y6CIZVK* z7tG|xK{cvs4+0ZqkG>d*b{{({S9G<6&Rr2Ai+5nM<QuHhTC9g@*kLh`QH7dQA=)`O zYpDC1Jv6Dwc}dz=d?nHvaA{jx^_ytl32>_J5PXAq+y?Ko=SMW&m+7(YWNV=qQinC3 zNWBpgd!kh|+|(I!LWOAaV9HRH9IRd433M4d7V*Gv7%c03RN6n<s%v&R)2&ka>XKy- zRNFJ<Pm&Zje?wZbc%-)Hh2*wpp1hI`<1b|5z9>!Q60Lx?K1eq^%<FN9olx$h@=9`s zi*j15&bVkPFzbUYBo1OuHTXNSUsW5Ra>?HLkFYaZ&c>Ry87G`4xZL60_nC5`Tgqo) z8~c>&pJ>$dq&H%wa`ak`z4Z#j^YpFOsR*B5{h`>Kx0)~FQ=Qx*zAJ%tK8C%ypG)&e z(b&Bb;8`~dF_<m<M)Lm2duASMcUsS+T#<>JrY}Wj7C;!=@GQ4LZOdKhB_!D%Ptb5| zOjSMd@M5)9ul=u*Z8|Vjq6s1(E$I!?e-IXB!Cm(=nmvk5JsP<uBP;5>t|^wRvy{yy z$9$u@*cR1HFZ&farxFEABIwAfOih0#xXiCX-x_gli``(kl}XOJT)n_pxQva^Ry%2~ zT+#Tqs;%}F=njX@nO#Q~W4Gx1Tgedeg=iH#OuetclhIiBR$K;nDR2IwOf)G}DLtWr zR|pRbBl#S3tQ-<rn%1wHPIiBza1m^AQGzV|%KS3)N9&5(UF8*xK)##Ccx82i($*Jt z56tJKze%|5qKI47mK8N&8z}H8!ROk$vUg`elJhYj4qWS#Z_VhdK578@(J{nF9*X#) z@K?1P+e&ITFL{89+l#q7F?s6o-6f{B`MR-FbCo~qVa9*koD}bNN7F<^t>@knaC?>e zI=9ivSg(|iZ}GRYc<FO8z|2FIgmxO=RP83IctU#Z+a_`FSSD5C*<PC_K{?&~n_k9t zZuS$9?)Ep@2atV*w0%W!>O9|*U0IbU<>nxv_U>b3G23;F+UOA|`56i-R4FRh<5y=Z z-O8i&5!s8Q;K#o17Q&*$@k2LmdutK-rbQ>^4zN3fB_&z(N_*zh&c={x#6@sRwb@K% z>iXVS*B&)YsZr1OB2Bl_h~?;--lJ5Uy|u3v7Y3neN~%@h3R&+9=z&(`MX<fHp4z3( zGwljV?RO5fMS&m8f{2CHIk!MG@AP=8H56o*nJ3B{((i)blXW4-s(rcDy=N+slfc1n z8^`;+!GzbLau>pBOr>RQS-e~0>8_aQ`Y<8mEZL*eebiA&>ca`O^-rTy5+<ox+xSwA zoGgQph2fn$t4^xzuNh2G?1{f)xbbyKW3Xw2nGUDn%&<XNd;H>%bQbjzR3aT=SiG8n zSFfuh`k}*MH{JYI^;HGWXxsQC6vhssE0N}t8|&hpcU-@4F7~UgCn=*S>-HfyM@&oz zB`RY$YLMGc_BtPGXnyOPKBlNoR=US<gy6(Lv2P1dKB&ijm(Oh@PQ=E_+p^9{9HY94 z#Uv<i^2`eq#zslH6{cJGeG5j_R0y(Gk?&_Eat`<l4%9z%7pO}ypg>70i<kpHFU6)E zy(476+6|`#j0KA|-L0pyy}VmH`w|k{JV-?KW71T5IXgSJm$#@ns<|C=tlEe)8>d?E z#B+chs;U)n-ThVY7{&T0_IT!OD4XtXwHkVd@v`Os$HGi5ub)mk%Te1mXARA@V|p)W zk_)J8=NoKgc=lt3jvjsn-Rw<r^$-q8g$xco$nzXJhppbD9G7jQ*;S@mDw+s)X^sEm zJQ}a@f}V9w>``IAje=Bm+9=!Z_D99$RM0#czhwF3n!kD#Cwb0sMPtVH{|IyC_Ho>s zkyJj+oJF*RpNF;=GCUyL8Yxj4MR6KLHWyW&J3yC}8VvJ3&p(y#RZ)F|#d#T%U{S=T zcA6n&ia^S<DzXf+!6MKzF#GIqw&iu192eeS(cTCfSGN$5A)c+I)onVEW_z+XQ)r2C zC#hQ=h!v)B8+9QqNc0ZpRNEHFun;7iEelZth)ETdpd~P=?gcUODARp7l~>ERLHMjQ z93t2z?I<4rawds2#i2Ord|^fPcI&lHCqg=3bp`6&@$oKboFdZu|7q{qgPOducr2~h zSstzKiXZ|ity)lQBPcOIm6p{~#Uc=pM{rT2fFK|uk0iS62CYOC<)y_>Yps$X!BPm% z1Vn;@@F@}^3IvP+Nq8oHgoHfyeo4Ec)0y3w{cC67KW5Ik=iYPAJ?Gq$dww4)Y3y{7 zXO%Nw2Zu8HSU1?Aqp9yDaOiQ((SFKbltbQa8t+pz48?K}T7-g?;rF&vOt5!-xg{_` z{Luw~8)8(9uJWV|AtpU75|l^XyA&lTEw7l|#hwyzb}AF=+J|19J4UCAc((GA?_!S^ zNGU86@zCC)0YT<V?3pAw&pSQN1;6?WkK9mJ4deW=VL6siH^LFPDwwC6m(7r~-#MCM z?VVVUqCrZ|gl~DdquaUIo|0AT;PP=~aLivv6;5IOHg|B07nt#fq>t#4!n2pCt90OV zCoWBKjWg>NAUqc--`TU353ZTtZHdnWRx4}p&8TgwIv+JG2pC~{m8qta+=AIQ+X9^u zdfDe=W7XBcX1>5`l@+wez>t*w{TfA8T&Y)GN_gM2oJbT6z*)*LONJt9I0~Oe&LE`> zr7u&^c{Mio^K26+j!~RX5eh%uM6PLJV-*hjj+MV=If9Fp*_w9}gjP+2un+6!L|EF= zW8)hAMMlh;#>Ew{am;5YyK4J-bhIqyv4>T}j1W^@{6LAZZM%QXZnwQteoOtey=pm` z-y8^CPl0H#c;XJf*}X4bF83Rc;JCNV%7`ilN<*D(`1vls9(aWNDTTb2NOeNgaa|Z( zEiobDpjlzmUqzQy`SR2aI`t!(AnAMKhJaaCR>7%xQnF*Xdh|Kt>|wRar(EZEv+vfP zT=Rv|lgS*G*mb}DnGsF)Ud*P8&k}o$Ka?<MIikbh5-K4swXWc(1oP!^dB`jiqj?sE z+SnftnJvxzP-r7JERegd(q@_<Zhh#iEpJ7bSN=QB@+b798cB1kI$kQKX+>qa8nt|~ zRbXTy7ytU71(Ok{d4onD=!PL(qExIEU)NPn*Kl<i4_f&;ma*AjYh=}|FlL(JTZ7*= zQTf(dJ5&;a*--lXOjXX^a;bloZn7qr8m4(p)xllTCu6hhaijG&qcXb&QA<LL{@K>< zWKspPJp+-NYvr;nCzW>fp-SgQfdjlI80|j<DfyAcYn|)N7vf9aMC1d((9z>aRJv4P zGM-x6pCz2WLESQO>K!o*(h5QSCO__7d!kn>*e`v%Kv6CBIe6PlFkf7}N;&v5K^Gsc z3F6e6y)6(t^LlH3;F;Q+KyIpk_4V(Bw;OZ-2;LTrlK-C7gSqNQ`#p^G=pU`UiCBmA zzXZ-+?VNB7#$Xn6YS(9ua|ZX@-E!`zp?7376$5T`&dFfb=OoR!*SWwl7gB$}{>~*6 zZ<5CUC5^IGO_VvY*NtW$4dxfjs+~lm4XK7=)Ly4`z2k$Q)T(-$p_tGfv^5mF1}<Xu zwI!e(pEjR?yszfB+Q`#0-iiwaXQrDjkK=o*E4@1o`;&aH>?XK9_ygMR?(SBs#bj$= zs#e)04$Y3K#+{L+SuwDVFxr)S{8lvDm8(DX?0$gU<0Sq%?bVFO^y8)sh+LI;I`5~; zjP|R4z|(HvdBrCX$!ncNo*wF9I5Q(V<_0cRV$zau=vI3P9z|*CzgC7e)avu^o?n=x zB_6Hb%H#Mn-36zLbx3w61D9}&PD?zHeBg6H9gPyN)e|3mF)3;mUaHEc_9k%^mnL_0 z(EUg|3SE=U=uE5wj8gYIRJd=i+o!)ewU4O$?LE0E<37Q8Y1?uBiM*Fp7nJ;!fUFd{ z?bC6Xq$k@6fSLsI>??fGt2(Y63bvqZuZ~Ilc|@7BGLLVWGqf1cuZKb7ri-D1ysUIE zh{uP8TBONVD`VfD_hoih0BKv{C&`?L;-Dr#ePk2Zo6Cm6PSQi5Z??RH(+T#*+3p+M zEgV)da$ml$2zNpD0}3J4bqCoVS)ea+jg?whAbG@)r@BiP(>EgD2QBCJqm1@bp7R?m zH0A9-;vqnPqVXwco5dN_w+J9(q#OeM|IP^o#F168>K8BAwr)Zz6MMHF$+x%4mcwU4 z`Q5uoyRLWx)_MeGU14C|O@KF0axG>;$%<4~Xszb8g5q}i^_NUYgENvi-k0z7f$`Uu z0Ip|Xg18NOCj3W;BiF)0wQ9&%y(jyct@bs^f~yok^iv&fEDBFbs-q0btpmR+T<h~2 zFW9|jdHTiKlt#?I*K%rgFYB!JpB&QBl&fbUP*j}W-osx!op;B|>z=|x`M0RdbDgmW zHuYs-prr{&NUkoba33BxfbJG%is7w8H44Kp)MjGDoY9(7j!q74_Hm*pCU08UQl`XR zyOs}6F`x&k?qcY%DyR(-8l?u1Xbot6QWXLEF{yfq{i{)MClvRFd|>sa%9O4b&wZ%W z$v!#E_A=Q%%7p4vmUx-vJtBeRS@>~s4B*C)e?b6fwO$oR;*sJ8fO}DBK|UvHNSt}F zlv_E?Po>I6B8s*&mSspu#o*(ZL*vza%Uz_Cq=jJe^}YLn^Z<i&9x%JuA0WuX)k8+O zxDUx}RwXXrYF;UuW}vh?&i~l9r-X7qLb^$z?L<ZfzC+5iKT)RScytzHF2PVIQ9sO8 z2Ue}DWuuR#SQIBB*)3gkgT;XUxfr&#*@!<=N#V(dYyd+g6ix1E)0;bH>|)B`0RPl< zz}b&(7SpoX;5nVF;NrNOE-lLM{Mc^9Elemy+=P+#)(ohZ3GIbffTqh6#_a$A#oe&* z@gE8c>(mk^i_;}3XclpF>9(P2^(c|5`F3CWk7Iifx6Sy7N=}|M6~~2rLN(D{@Ig)S zAnXKYQlEd@XAh*ehvVl^lo4P0B7hcv$mMd9Yxu7WUfzHpUp<&b<ye!n6M5XjbeE1D ze!^75El_x-k!4<W`s9($GL>a9)TAnS2<-+#o^S>ZOFv>bYwxz&B9bkPG(y@DDUsy6 za;GWoN=|XaQcjj66@Z*CN9uBg)Qyj)yBld+0q@nn1Vdv{dQF~KlnOvC*Z@-`DeQHZ z($w83xyUMw*Z3mVBZcC+picL|A~@o_jTchR8b7HDTVkH>Pto;kMIH{YC8S%FT-gu` z(bfNHx2iMMFrZmy+43litA_`*qcuO@^!j7q0qU%?8A7spSOBlrpdo;_3h^h?X)FSH zl*Wo6@7QehL0Rkhi6)(U5%e)^jrO~ykfcv`=sg;y(wO%7q$KD*3xXi-W-Q}q_k@E_ z{6TA-Y7rdfUNX?l^%TLA#~0MzCVm*-)^~dx83=0hLKoW4DMbzu^z*`e3f8#M3;v2y z$IC64H`wlC5+>qZf7-FYRdCo{l&t=??}kid*^<R=fuAfJkGI2fQ+P+nK0^u=RIJ`J zTv~?ort5!pz--*V<j>NA0_Xc7bgb>eOc!5y2}c3S;U{Ng-FN4sf=?tylO=}gL;bS# znctI0Bq7HtK8$OXctza5@^&#Ni9^XfLXNdG1}7c!tPCmIVo={q%mznJ$jV3l&W-`p zLQDElE|aAMUsG07ZjGBZ+5zG(b<yPS4HGGrzj}N<FLINWhj$gg?~SE<{cL@;laNQe z$z*k}Ul8W*P|0A$QF>7smDFgm!r%lkY-tW5OgVV)ptVPoYcsEfwKYg6W@(pLWu4$p zIQBC*^OcDx&k4_N)g^sJ4%0ti#bbfP=1f8jck*fl)52+{K{;iYbh{)>(DgiDDk1lz z2$_>*Y0cuH4*)Y}>{3syVV>xJbANLh`2W{{`E#9-{ff*xpC!QWp{v>c_0F$Kwub-x EUkXzYB>(^b literal 0 HcmV?d00001 diff --git a/doc/comet-plan.png b/doc/comet-plan.png new file mode 100644 index 0000000000000000000000000000000000000000..38c47839e8733ac7ffe6eb916dc29a7b605f7a74 GIT binary patch literal 310869 zcmZ^K1z1&W);1swk^&+UlF}ue(%s$CjdXW|N_R_1cejMZ0i?UTyX)U)X8sxDd$~BA z#eRA{b+3D^9V#ay`W*2UA_N4)b8#^t1qcXe5CjCQ9y~1Y#P4Em2?7F{$4pRAPFzrs zNY2j2*v!%h0zxb_P8Ci~u@5_4Gdhyr6cRcoU=bRf5;7;?r@)c4k^lx#?8|5GyT8y; z)jL)AJlDpOFCV3Ot8XH|?^%(VSsV`2bV?56u%O!N+-h_9aKvZwXumn%U<4t@)qx&r zC+`Sx8UVAJNW|x+V27AC{tV3rii$hguKtGDSV`%n539ptQ%e)%L#B33g84($Q(H1@ zM8+@-M5rHoTEg(4|1}zf3z;e@9fZ%@jln$ZkervSnPfo-0?lG!%p3IzVPhLE86XO} zAQ)Rb2nnj!U)V7rCI%w&aqO70g3S54z6hlA!~49#*FS1*?&A&_BdQJJB@W5G{d7*~ z!1R=t#9Q3AVu)gJqh*-4PTz~OVVC!mv2nc=pFoWp_5mS9BA%UHgwsT=PLm@>Fjoy` zmG^_25NTI3nm5|9;UijhERFcb=6OPv<;^O+K_8T^FWg_e-*&;G!6Dj{e1GBL7DN>J zoKfq|_Ph6xTqye5Avh;z&-Af)HM?RAUR3<NMP>{ySRS_Rs;nPXLnY-nkqUg=qv6%e ziYUM~>LwGA@||v$i$sI0pz1=o;i$+$%&-+!FAjM_SV1b1+{V$x=<y*kLyM3aga+61 z_FnSrG<Y-Ist==_<i$1rdljwnW?nVC1d;*5A($o*4%1q&!}X2$2SNLllZR*P_CMUN zzhA2b4#CO$>VEQ|kwM=Op}e<M^s<hOWec3<Xk$=e#Fl;QgQQRzVG@G)b#>i!Rd^;f z@~tcHwH+F)T07xSU+7u~23*5u??rV537ey@prKIE{H5CAE)k7Qg}xSx!benO!rvpH z6Rx91oW)$6zJIgchEjE!&C?rsf!!~%?vszU<trD7R7O@Q|9!<2VHv>?`YB|{_cPMj zV4(Eu)6vbFORmlUsYJghWZ|5m=dKvfLLR&H^od4br+&29%5ozUcz&?|j$42Cle_f} zGaZ8yAp(91K|5eB2D*DSqY@cTTaHgoKKz`}!dFv38fPB$%5PT=cdFyUm*)_9jlq`i z<qS+Ejqu`V;ZN_;$2nxLdOSqFN)lYM2%@{_5^sV6^P*j*V&amEh>NJLWL_<s$&bUs zC+d7k8F0f{eD3<;<^I#^`=m|;TwyE_oRfYm`UlgDlmt53vW7(#Z#6#AWW4@HaxL1N z#YJ6W*lO_Epll7tRTt7b`5gbbT8!{mw;!k&iWAcd{o0{6a*rYGaD96n+2{O@*xR!u zG~Sa9E$#AX{+<uw*c|uuYwpuDBr{08Haw5}lvevm<m(+d*kvtDRbskW`}%W!9~%Ug z>zo}(5<S@B6^~RO>SO3uB8UJVF*La74xvO(Y^^PQ1}qTt89&0?X9FFt${{Ub=i2GY zab80_Uw+GuFys@y%v6tZ_<dsRxsH$5F^Y!>T0npt`m3+OFG;XN@K*yE&}>K{5Cruj z4N0H(2?c~eCxwDU`y=CeVGXGjKJSYrN5=HJ=3q8_<`!~`n(C!^A*d`zJC0T(B$|yJ z8yfW0b{qN~W||*%7Shk$ZO&@M1^>xxijnYb$7&=`UlctE#U2L+Ob~&A9t{nKWCs%i zBP$+tcYg&{F}6j})eY5T;Kc_*TVk2coq8Uvjt&>@q3-<UsuQwnSPvYXh`JTklVBIV z$Itv^FVXhl37}cughq-dO1ym&i|K@hf+^E!Qjl`|MT~S8tt7<ySY3+T7}EqT0gVk~ zFN7i_t?Ne@Bt~eCXeEhj6tVbg1;~Pc>h$X1ae+pGU2;!31r;V0I|W1~33&`EEb2Jw z0~O9v2jQKhQIkr}8NdDaUr5h-$NC0(=lTZMCD*M_O~PQi!&ZeuvketD<tOFMDf20> z-rB^*#Yx94#AV09^xMZxQ64Gs=HJWRWQXO2l^{xzDGPtCR5Hr%M~IuFviYV|NH&p> zEI^w{m#d;ED^DoL_ia=OUumY0dxE17tMH;wRY|unYJz4w(HwIkY`lL0rBGX8Pm!WX zv#@HMW4CI8dOT&^rV!Qi=er(LA@g-Jnvv-eTb*ptm>jbL+|0f)t;`1Lv)E08S0c>W z6{cFV9<xrfW*T+a@&gj#C2=Lnc{Qp<f<?NTigW5*@7<G~0xua4=<ua}C<K?p6je!W z`3yZzlO;hGWe`0Ur6G{#TI~<HY9w*=bi8TIZFFp$a^%|?xqv#iJ6G9e*<P6Xabb0l zeUZOwGs;)8#HL`8vn6)eI~oNivdGB)-150TvhhnxW>dD84DXnbnO@+QvT++to8%a* zTf|yb&u-VtmoW}b&#CD(&(?Qeb7-dw<!p^kjm&L7ki(L9kYmJ@kmHkQNS8>hNry}C zNgbuggI*fr8W#;}4BifY9_$=6NuEkvsvy#2psA#p)JUyyu`x4BGHv)-*-`mJVk*o^ z*-+WQaX8uhz<{T_vHv#t`AE=*7=x;j%}QGHJ^d6T%@6K<C^hyqnA29(d#{CGZtzO5 z7n+YtyJNZMd1iTjeh`Gi4`v9~4~7j+$Q~5U|D?lsyT5Joh3-qkmpS~SFC(upnetqu z&Jw3&YGt(Y5%Vn!3Jjk87{Z-nzl^I&V_tHNIitl%maHETNKi>AE441wpVFApVY9D$ zS0_=If=ny&!T@0{m8)Z6*z?Ps{8a=&DFF-t9QWgC^%dx3{$PfA$o#eU{H1L~i&P7{ zj%G_^^Uk8#qGofGhm;39uQG3h$DoI6E0;$GqgCPzhzFG5^~FoSt<&pBouF8~81pgF zJJP$rThzyx-=jUq2hKOoH_4a9_u)O=duBfwXiPsre^uxuL%hm{eRC)85Y67=$XdC$ zpVL2|XGMIs?Ckn5xMJ!ec3pVvj*gA~JtRmBt?!E#mMdRBmAS-tl&QXhl7sO1SRx}M zv%yf$s6Iva;TKPwEu1=<L+fV8X3|!c_q1<VvN%^fzk09sca`Rhh*w3^k?&C;MdK-o zQ6f-sDZuu?ebQxa<(eR?A-(8<E<zY1PtEwo$LPFTA#NVS9)psHK8i8sl<&DBWLL=k zY-j(x|J;i%yea<9Z)-qvaBsu?MD;Yo$i=7_wB2{!uN60wvJs=4K0%>}zzVNp1ZQG1 z<duP$RK;w<z{XfdCre3BTPFUE5`<nW&6><@WtZP8qeQ%$yvk@a80%&G%|(<qn(s|p z1wYbbZ=GZ@S(_LEwi}Mh3$MgvhKuEQpUu9T9gJD!)2BZczb)Uc!@Wj6{4|+F&b&3~ z!5+?Gv1e)Mbh|&vVQh~L+DXTI=RC#T-<;~&hQozjM&rfBXX|wa)d^#d!~ICMosf(% zfnnP>7?_U4M*kfL0o##=pvmQ7)h4%j6v^m|(T0(3{Ko{XYe!Joo)^AWaz(X4(q5(X zK<Gf=fKBnSdWA)9Pyg4r2-U!{LS1QH?Z(@KWO0k31g-=d9Z7X+^|=~4OYJqcvx|W# z@0rM&6}LcLVeQ=a8>{_Sc&>P5mZu9!O*0?uXEtRup*5|op3iFN8mTL{ey%V1I5&2H zFgN_zb6w&V{P?93U)LGY+36r`W<upaMZ0BA@2c`uwTQx46NTgOM$4l+p2dMh-Q>=A z+`+Ru(!2B{**Whu{%zk$|4;rk!89n!1bO&$C8E3eQ&((;EDkK>Y)-6!?`l}xSqXVp zKN@by$7EyusE(lJlHq7wzSfhJA81Dy32lu@q9Cl3F>i6F+8M8@-LCyImX$x8hQMvV zOLJy9z)@x|I_#cm)7W1B^W(^czGsu<-O7@-7vKEKth{&Mfx?>6)xcod8fS#=NrT;K z;nM{3)btdcPDK5pE6HO|NlF1*ll6)9#r#mq!>-Iq+OYH1BFQ{;OPN>Oy>EwqA)n>r zw>`T9&dZ-qAMUcJMV$%l3ENzvpVlR>{PM^1H-<lDC||K(ygqoo-*WG9e|CuT6({3P z;&J@C>NNLaK+yZ<abhuJeCa{r-u(C?n$69|snyYIjuVaLcv*X~Y4}KzZ`*VFBI@L1 zKfg<Mm{;~e;I{IxXmD!NCTr>N`Q~$!Fn0Yf`>GKB&j))UmM>%=zV$);Tz>q-q|^Eo zgG;<r)sJ>9tJW$Sn?wy!bNs~9i2ArN53RT4RR7?2YhC~$Bo7&454SyC(P?E`Mn0!1 z@frRKQuAwT-d3)^x4|jT7OLx$2dsj!$?@5ZZh^iBUwY=&0ANpq8L5dIOG`sg0Wv%U zG~_D?7(jvqKD>~4|CL1`-$Fe5*K;Tch+s1a=>O!A0lvXMk-!IB=HK7XqJtpdfd4Rn zk4q-h|IZB#%6#_!Wmr8R4dT6`ptv~jt!Q9pWMpk`YU8j;h4=}0fbdaF-5vr0n+*Jc z6jvZU2I`+SQ&MwKla}H%u(6{1U}&RnMCW4l5nK-hw+kmAS{XTfAab#?w6^DT;UWH4 z4o*M@Kc*)p`d1bQ3m#%MX*nW68#^N+Ryqbc24Y@BA|fJgJ40hm1tF3D6bJs}AvSez z_{d35@9gYM=gdrJV`oCo$icxu&%i{_#6%0^ptX0kcKF~zYi&>R?@IoEJwitI26kp2 z9n5U3iNN)K(6@1P;2|akH}v11fA`bK#q57AS=;|-Sik`3!722NbPV+Wts5xH4Svcg zXXavLsV-z@1?UWDgO`z&nTh*f1^!C<Uz5KSRr{qV6BGNdMSn^9v#7GYk)5E870{>y z@Bb$3KZSoy{7*q{dhpnP(Z#<N{jaBhrg;&$>Hj-xyojOdii*HQzBUt*RRX?&Wd{Cv z#sK_z`|oc+hVtp2=|seYfZ&G^7kaPc0=b_C)2t-?)Z^`t;N##d2ConXNkv3SmPI5c z=IkuyoRH%)I-2?IB~f!`W{!ii^P+Ql!_dior8}3?5%1ZP%T+3eJ$Wm$`_qx<weUMW z<$yqFF`s|<<;M&W>G(w9`wz)J(2pei9s7LP&BTxxMF011mR^q_KNRUd{IWv509i7) zi2q}T7a$kdf9zhnh>u^=b69%uf9k)FAMP*&LgYXEPQ%MFRCO$q{bL3>hBe%O7#1*G zJV-=i)|bts|M$3o42Z_u&;DuPq<VnnK(PJ^|CoWl<CDohjtkln`fcKj-f{Fl&dl4y z6Xbsyc%;u~2pDEOWS!`L&`czdoc@pF!YD`2)HC+)ko>1P$H>&{Vg8312gb_^ML?gZ z?Y>QMl`(A$`nwk2x8g)@@1hKCv&HWd<?URxULZvJ;GiFib?kRp@!j0aGtanB$R6^X zoDrv{QvEf1{H6$oVlSk1+&LPXQy7D~<cMD(finrcgN(og#p8RDFQs|qaV#gS51eF+ zb3g702xTCB5&d%sqRT8dR3_9vsIAXy^EG5Of7Vf$G<LM>iS(oVDtfKOP6Twy2J`rB zhh<M|e&yY%eP8^$#uu&}$)us3-V(<LYkd<jzpM9}%GaRo-f$~$%TCo5-!qvaFIGx$ z1f)6)H6;xd^}gKzn%O{IMJ;IoTe@2}Aj<Y7JNci3bHVU+r%J!bQygBOea5$=@-E&` zS}Ge+c9w7`5c(?<WCR)rt;%w`lw!8BSPC}GNdBhX*6PnChV~lj<Dgclz}31!x)?qp zvySY?=?YqeNd8^~mjFH1qh-aJJSi&+>LrK94?!NxH-Gg)NEoV#jwgXn`8s{5tS9=l zxS0_))>WbfF{CWXGP<97ux`3oSUvFu(~skKys4;v4kD7D6tn(eTDFzrUaH?H?9D{g z$x)BkA_|Hu6z_YYQJAWtew>!6OA1bF+|`7N|6>vRF#C2(Fbu@LO|GUDtCKQeZKi<4 zfOm(AqDxF`U8B&wKMHM*qHA1t4fRu>wqbPtYl|LHlq+;IO1_lL2}XPz764EEiZr1A zs!^a?=?<FatCWX7CNI)Q6<F4radSQ5F+NjZu}1Sp=wF$R(0~*!_rbVm9}2LD(A3WH z_XvPp>!<GHavcdRM)V0mjseLabN|a<TbXDdoaCOj9|bMe&I24Lhcm|re<{-^7~IYq zybiNSp9ru>f;VXN=f?J7!vs<Wn|?IJ_*?<Ichl3o2lube4}(+eHHTng_;WChB|7#; zXB1D*ZsY7shh-X^PMS$2lO*h#71Sy>Xy2B#T(k;Bppz(M1x+oZ9xlXXOwz=~JiU%w zgL{f*w_TIWOLZl+IJ`eq$xX&!R@py(mHhB$uYguE^{@-BIoJ`dUqhB*f@BC5={slH z?3(1uUbt)X?y;(^_Q)>Rex~4McU;eu!kZSBJ&0Si5o?)Tz|V6%0f{GZW>#r&I?rY; z&Kt^qo!M#$mW*Dfl*Zy~3Gm6t$T+u@Qy*q^O08$on`EL-;>iE~qCpo0rlG51Qw*4Y zVjZ$XZ4PgdR?~-2ZjX;kvrVZmVZAXwBG$XIJXN%N_8W441h`_*YP>5N8l(S$e=RBE zWd1c@{rWT0X!8_9FeR$RRY5I{pi$)udP!dloBlD12&1dvB~5OLR#ov=PWyYu1UB3m z5YScFgfHsIIjuTn_L6b_*qV>PseVEVKC4I}%P>Ge)n-Kb5d@<iqegvn_`%X~(4jHg z?(j00?sYX?+13UoNX4a6zHcE_W$Kx}=Q0ZKI?M)Qu4)udKQ%BdHaZDkS}qmpN+z-T z8}>(uKHff<aM-$JkV-#R#=5mt-bztk9W6v1ES1EC=gUg;zuO_x9f|hi;J~To@;bKR zIGrNAmA}0Ih)+v&f=OPPd^p>ve&NkuTdb9{_j5RfP$?lmJX$<z)Ck6Y_tIAQfkuAR zkZF4WD;@?v{lY?HK;!B<PBX5^n7iocM+8Ct4oUF&>ryRtYBOdN%wk;ok92IMI`M+u zhp|W|GQFL5SEN>Xmi-H7Sbd2kb%J9h77w(U?Il7pVU^xjN2kNOP3t%IeBIm`>Gsv8 zBlwH9trb`DyAM=$%Mti?mQOkxDz79@u_@o+R&GRZkKo_66A&n!ak<|UHh3I<B6Xz< z)RE<#ShSXhx8PAD{<<#LoL%5``CyYW-{>XWKVkhg^o45dUIP8wy2YoyZsvSa=^h#d zrG~Tgdy}?QK36HeAL*;9K5s5xk=a(RD_mbVpR;)R&<?>$T;2zgYHvObK|l{G?%qVT zJw>KWl}Xfcmzf^Yx1@^H{fNlEyZUzVI;t%HQ}ym{?QUV2F1>1?buBiQ&adafH7BsY zTfLy_y)Z)5pmazQ-?y9TuJL7kefmTe%_(RMr?Eu)BoRNHoVmAHprFZXKBBIBcf*rV z)`J?g#9HEH8j+&3!z8ga!c2?aZM~yhzOJcJ?ALBAG4FDc`%q5F)SwI3V1E`=Fqwg` z<XP$`T*4XRj&`FJmyWq*v4S`ZTT@K2CG*-d3paqanyb+!;?wS^6ko983_n@ojBx8& z{+EG=rn(wo?Rc%5l9$$Xv0}`Xii!<xC-X?2Z6(*^){`CPO$GyQNz8>$NC$HA2V?OH zkJ+*%CK;-4^U69_FYJ_!1`4|;nYCWH=cm3q<m}BFy6IUItzg>N*m9(Po!nkiE8I2` zZwfnE6?75Z>F&%?-TC~o{Gz=A$`{jUZ1kqm?vm8=L8bdurQ?E~=VwCCe9w}A_Ttll zpt#D9-{p!3aJDI`Fw{Gsa*duw6f#}R8tnIi910X=rrv}0nY(qTg}OJ5kPN#MFfXcZ ztr{iG$Q-`Tx4DaIBI~ew3M|ced(4+6G|RUZmVR&A?o`HY+J_5GAj&EdEhhO531xtR z+g5EV61lG!*)ogYOvN+zn(Jp8N%os@hihA{5Qih3aI;~^s2EfO%m#f`4n6DxZb-v= zu1@bmrFv>GzbpMfAA^R9)MgOWd=a8l-fBlTbaCm*(G(#d7M|;OdNfxf?d)ISCasYv zoIXQx+gsp`qH)r9jI6Imb)lO*^iWfipiL9N@*;ppttfXWQC+|1(*`%o!_YCx=Wck` zZRXDyVf&>;EGe}K&P7u(?sYce`4Nj%x9wUPyslr>_G@-;rJqDJKV*Fm(cXM>%St1i zRu!00z?L!Ondo+BvLTx+-tG2iU0Ad!6ZLrsG!fCklU;ewRb1Q&sxL>e81*dovc>`J zKdij+o5>EoGIF`=8H&)UM=j8U#aXfC(a^jv*Dc?lD=VG{dGH-^DIik5$JSrG9JHWu z)|7(Uf`@^!eBB-Yhta_w;D;*<2|y4bjN?}w`}q@+>w5*+?^4+MRbwLNE|4+uMX46m zn|5t9t(rb7mL~TjB7N^(j=!MYE5ep&_YqF2(wlTY6%fy033lpBVKdI0kR9;Kc6iOG z7StgCiVa<QEG%i8q)DXP!}csFvUq1keKc{crfZ|P+IwH|IH{_PR~w1w_-e=PuD(=1 zu|pqGAhJl|bRkSZ;K8KM`p$jJW)>0UU`1uxy<dP9{`<m5z3|3HJOf&otL}rPvN1<j zM4T9b+{MYM0pl+*N2=GFiW61~(%}v+dN@txuxc3#DwCFLZnd@0oJ}=xhoGI{mI&@L z?1Vg?ptshxh7YUsMx9gk4i@mXYby6SWm7bXZ=7wG#_7W1_^b=H8Els7$+h>C%V)o6 zKW5if0>m3|HHsZ6e@bK|YqZ1CW-=&V=FZ}$vEuodN-7qs=5>8(F(80GEI^gmVdc=Q z?+b;*5@PU?p2t}sEe`w91jZfaJ#ru;=pF3iZ|03J1Q>9jp%MZYbpRt|89Hc^Hkk}f zT22oOX75|(i>{DY6EjO`=mI~tJVn&Xtmc|ELRBAC#L-HL9;XAT;W-XkkDcqPG(XG) zzIK>gi;;+jncXH^vN$pFC}Xvg>)sG37^@85(74*)!L*R7$FF_Gpi);6u_wVGm`$kS zSt68XLlY(2W_l-|EzU&-nwVm7(s9D!=jN_Fl6B40f|ZYq8Y8elW*o8p_Czq>M9o*C zPUZgedCR!&vNBwIEiTijICIjAMRAfYwcxI%ZVF3=Z{HL7)~;bg=H~v!en+LvZ{v_t ze?$a}ZJ%t<stZOiPY!g{(c&KeHUu_i@;<NRVBZgy=7i{S25DaD^F?YES?U50YYy;i zMGrmLuO)yiiPB!=k~x9N=)ma!aP@!$z9ao+1$~qROj6Bv9)R_Oh63Fz!XUW(K>3W* z0vaB1$*J7CxUxNQns)5dTjRxwAF#};iQ4s*KUOO>3p?~Jj#K0E@Nj?LlMGb$!y7xj z?=|0c@P&9J!Lb>wxwKtCFgAs^g{c^c6oa7i>ggVH;Bk0-B_7WdCxt)VjnTyUZ5_S= z5t~t3%Zu0=M~FbRG~cCz&L^pc11b2D-SqOZ6!x9x^33=}8&`-^%x!hu<b>I}1AaZ> z$#lFF9&MBN%xWQ*_Lc;Lz0r;_p?hkjL=^?H8iRE?)*fAs-%Ry;ZKL7ZjQOE4?w1Vq z1vwHKq1eW-@O;0S-b4_9IS=L2DGI=v4>Ii*&{e>dDlr_S{0;)1g`{mUNBeqTXj!9v zX9;3==JdtSc+*~f4huD#h5Zr<mTp5Lp}+;J3YR3r(Qg9~a7a}@0WB0&N%RLYXjGd{ zDn*rG3W!T#jTLLrKIK5X#{w10Jhu3>vt+7UQM0O4=?HF2u=V$U74IzOzb!9hwvHb< zTJze9z%#lGX`Rfz7nY<9FFUW?+d86=q;G;ShR!d?0Hrd!qX4)KYYYkU68))2c5OG_ z2p?J!-lsRR(RIfg@=P1S_~Sw#Qd!INm2q$L1l6Rr`Cf&#sRLQ!8^(4Y^?~BckAx3S zi%On0T^yvC{0fjS%0bZ-ZLK}1Raf<N+F6aY>Dhs}rscoh^^yGT0M3H}1<L~3`yTHz z1#hh8<c<{%f|eGI|H1iW+|gBKBAd~_{)oTPez;^?q@Q|YLL9(9Qf5Z09lX33A|9ls z@^--FbHY(P{rJ>2lILt-5umJIcC$)aPLp$?9#O-?06lEJ55zpJo?7l6)Z9Xs!;VNM zJ5E@8Y!$0nJBSwsmRIpAm%UDc<Or*g4~@2QL&#yQMtsE-HOV(k<=;1KR|4K>z=z}U zwGJ6`*o&bK*bSubVCs*D%{<Z-tWYMtQRDNw%qCJ7$QU-?wz%h)svp>r)vRc%+@dE< z6TNNP<aNDQn^p4cG{;lwu)M`t*cjIJ{(01)eR<ITxb=*2<hHP45SQ5x_t5rI3io2r zI8KSUptVLb2DbxrxN9+#<$Cql!gZ=Ns7ac<VZKX~D6R2O#EZ9y@AZcf!LZE-rg-?f zFpjA@Ky*S7n8kUBMNjEG?s*H_Qd(5pcvTMJ*x5u<j^E^aFy2eF=n4ZL@gV{}VqIC> z(7eX;L<U)=0#LZq3e`KmB<*qmy&E?G9#pPV8vQ`UqeU+i+o>F_-jd_t-DIwh^S)<P z8n;-`W-=ToV0N2*cCn@o+?bMw0%c|+2dEogL+Xw5r0y?qtX#L|386crsP0xvGurBy zf?gFhCJDOtTPsH>v^oj8R_Qwq!l{^ua3jigAg}35pR7_umRKaH&j@ssY<1n}>+;R3 zX<iU!8+<QOozxbI6(R%{{66E*u}-&JhWOVUdb1)1Qd@>@c;osy&D;*MYXD$ve6stz zP;Z!SMh6@_#tQX2U)vFnst$F(twS}e2UPl`7U?JYQtqy_T6Ad|)C4;decYthP_QfN zf*3LA;>Y?IN%G%ydd&@~k$7{DP6UAwA_oN+N505Rvc%7f_`B=Ll36L9IEX%;*{kk% zoOVhu_>)aFk35UzukF|<N0Vy&oI+CWCg^&8?CYS2OZb^Tkdx%)%B(d^kh?0(SIjO# zMLixkYEI|eyX%H%^-$BqY@U}bMVG*$dJpYCk!J6!1`Au>zB`jyId^To*^-H996`u4 z9!i9UG}NG{3nG|?smdMu(I2lIo|ZzLb<oy;*Ma=mOTA41TjuqpfV<*I{T@H%rHrdR z^<}+Po*jsZ?3Vnvj{lwG{qH=0E*SviRGriJ$bEa?EIaz)GIR)-cCOCEmw)(a)R(5K zZtSQSuihXPcY(0SY)CIO&!k9%9p_T_bt5|X7Bl#=4**<_F{?&=ks@h#dln%yRz)oZ zWoiD8bIjDU!YK8jUR>UV`wO)bmB|8dAr{IHRjiz>35Ul^goz~3@7GkqE{MWO5NS5_ z#A}To&v7QxXu1a!?#^Vu=EdW|P8%vI&$Twpd9bM|;_XAdm8^IISyMsuAl2oZ8%r&E zIAS5_%S~A9c*W13y>WWRvKl_DZxC)#eGAu)&6sTV`57m*KQ?)^_54uC4B)dl_dFPd z!8aLHLto|Tq7d9vG8`0Q^h;WZt9~E6*#Gq>-({Nh`4Q@Qo!yY(Ox>BP#UM*^bzF*S zncDY6Z3g$k%9J5|7Jf9$;n-0u-q`hmv3{xv)Uc|%4Vx4diz&5KN=2eb6`Oex6ABi` z>t@@H=wcOt<IAH-jfo)ygkren-*5&bAt|7GIJPuN07H{5$NAy9@OYl8!N~E^)+x;w z1cXM5B}=KtQS0yX%k!!prX(n*(u7Fa#%^QC!(c029WAOrZjM2_rXS9qSZg;CCyc++ z?+PiHOd}Ih5$ip5RcuM%R(@4?QlZxu$a;cFitNnV1B3p3sve7P$Ex1>F1}02{UwDp zcio(yBX_;GY;<$%{F|!0IUFcg2zktkf#|o+GutEC;S13qy2K=FG`55Oz^dE(F2z(o z1w9^k-$Syvmhtqcv4tk~?2&P;{ZQih8Y7<Hh?#3dusK&brbU6^0t;D&1>$~2Xo9W) zRSI1dd5IN&1Vf$}xS_Tm{yXPGv<<!&(y%na_$wU(rs^lNA?h0<{tg=GpVx@LH^2Zi zz}tMI#fcyU1&C1)X@Z3=CQsyoK9c0Cxvml%1x)J6iKbv++SM2B-apQc`WsL=HUsWz z54XDp7mSdPfJpqB1Mc_z0nQFj2h`m`)^iEm=vi-oj=Vd+ru(fU-~)p{0N|al-z0Lt zBSf~hfjNw~|2;?kE4Xk2RS~oarxGCuQ34Hq9e1*0{Dc1>MPmUFXn68nzjBmMKA2=H z;OcaZ`kh7vYE%Q-P=D+49spU8fG7-MhyuueFwSV4Kq~@2B5VMj3pxVO=JJWY#cw(Q zFG^oAuzoQz?Zq&TKY&%@a@OGRXRc#xAf?$TpIsIs<Q2HV&z|d4f4U`zBmj;LJGU+& zoCGogY$UGL24Mc}82LyJBqhNgf$#nppjXE^1FgTIsDF|E-vfXohK^ntpcOwbTZs0) z0{3sluLF!7w(C9z6T&qypx)N{JuT)xbGhIGDLS<uQt1#}IKX17Zz}JfnXbW$sI`1R ziv+<11rXD#a8mydt`QH=!@SQ27GTcn!E;WhHbePmRgwH^U|KK!-3$(pO%G5Pj7Z4v z-$Nu$1BNKJ1SR7G?GXkz95QCD>EeIi5I)e&fQp!p=Glglp_}6X(Pn{TDCN&g9w9&q z;jATH95lBNAfiq~%lz%LBGLmo54G8u4@Rc|Ojn;iHMIG2(Evm+Kaj-EZv^(tSOcrk zK0FBZ&#Cr-<_-kfk)1&p@`v{D2gIdmlhVJOG0n+9O4%Z>OgeP)7qB?TB$N18=Ygfk zyfkNDng~q{UJt!F!xeuk%TEaCRU|_NI-t^*;7J!DntA@W=C8p?UY0Y!B4`f>G~d!2 z!1DJH3BZ1{U@Q21K`8lDpvkwb9a;o`Rwe`ksO#%@I=knj%RE3;uERREd4JZmTm_`u z>`&USkfKonqA;Qpip1YL@ikDDpJ2EZFfIYG=1mb!J^$QeM7|#>fSovxZVv$@{iAd3 zSPH0%+g_KQ8Y83-+~P#qM&qCJLxc#{Lm+F6<QqtOV1I*<SVI;6I^A@E_CQmz5L2>H z=-^&;oo<@`IY54>U(R3t5U_pB$k{6P_Ymd5eR*N_MG%S<Yz4WX{Mjd9%>YL_bHoy{ zOeFNz7r-8qH|J`2`KKB}lED=XdTtbowJU;~%TD94-!Jb9!xvRB>3uE2BVx&hdbAW{ zPZ7}vSP5jJkKX}Ox*CGw_`J`c;p1(<e2XUlUi&F**C-;n*lc?HI`7Dai@HgTi>GfN zsJDiDJK?U#v^V!Lctk!y+<odtEz@l!x;mV{bVZSOjc8!~r9-~Q07RnkxT$mo9*P2^ zdhHspeRy-WHCm~U(3i+lcb>}5C*&!#DcR)pC}aHmz)1Q-|Lg%|nC6x&egZoFD~mVC z^p3AMw8Jge+-suzz4s$E24^K_Q1^Bodz@@2gRc&w871q)?gK)L89fg4R|2xYbl#{* zKQRDy+(Qz)pBC|Fnhjf*t1N38qp3XeaBgSoEK)4h8`Opfo<uRoWvF1n65{P@@NVBd z;Sg2${j#~*_W<|qStIRQfOvZmSdsn*XT}|%;iCqJZ}KAHx?sxZb=iyf{ToCWF{9bc z(n^y#S%uRUiPPD+4N}c?RZ^Me>(hhgg36UWj#1;EE`6%iIfwp!Nqqg~Xdtiy03n5E zqi15EQIX5!NnVI3UV3*-XNg=QSxWO^h1|rKc=W@xp%KT+g?AeiziloTEbu8@qn%2F z-~x8*d?*iGq)F6nanW&OD<q#e!-*auF@X*hiIAl-8!^7`FTnu7O0aV}pZlg>nn(wT zP}UThnPK60P)of^uknyZx$%?T&b%4i2MpB1r52Kh<jHYj^&&_EvL_3&jmry*(}{pL zr&mKuIWhxur%oFhkC>~nzsxcu#xWBxx6PKF?AjQ|cK;%u-?5jh26QV{R4xZOZG2bf z@Kjo$5PJ(yoSq#E`}3jMr}sC^eUY>Yo!AR4o{FV9>=Xdot${%yS(?$&eSDG~+?b&` zRoYwO?_KbSAz9lDqpZdBq^4yOK^PToA|p2Xf_{(pKp?70&uOinj05oP)He2~qk4AP zCi*{Y0$wLOk%l3o-sG`dcl_H%$A`k4zC__}9Cz|^rc=c|D!El@cQ(4RiaY>?4`jn# zk%m`7Nyd^U334%fpSBmhNKg%Hp~Ko^8PUhllGMg3_jMdAzx+&ETKU$3&!N&97Dur7 zB+t!Dkw^43Th}{51ThA8z*x~ioyDbQ(c;1>UTY333*!0FnmZUPebi*nwdH=-s?+8n zFZ1o%yl>(zNg2eV__0(5pXb}Ar|^L@<epUG*5+B?+7Gv<3S9&&(c+n;e6ylEr8{rw zZd$>8)_rYL=qif@I<Yd@qFt}#%a_D+ZMxJ;w7S=RdbLg~QtEdTQm#!BRs~~UtCGHG zwYbQHekI(!Ln!i!pSE4q7zP8KU%<_x0C0TQ?Z4O(hm!XPvj=rHS_!2W{gRJgb4leZ z6AbU1ElTixT+UpVqN1CATBvB>?b=w#8*TeN&91DJmZTU>mM@}AvOXk_k7ILPXgDVb z#{{W+wnyg!j)F?d9<`Z=0qi1zQfqC|;pkX1=55X<J(cxAXQPdMaTQH1B>?6xmR*Dj z;`^=163!3Wm30c9&lxYf^d+wGb(r^6kDPYaEs1?5w=%jZMrcF@S+o}$p*+mkFk}Sw z7jD-{7adGQlR0~tvFLsJVi5@UEZIBf+0`7L?*;(SXNrpV{$t&hxZ96-_h8UQe^^L~ zAF9(&UTL82K(@{dCr6oLY>^_o_5h6Ng1N#X|GDKTPw)Ou^o^ROfu_M6Ub99%FQ{&? ztrxJX$|m?MNT^#=zs=2b=jgJPD!6kEg>?tPqKS9(64)F7zP;0pV_prHN>r5+d1}fs zegBz(#RE;Fp<XW(&v=izuiu1sm@Ohcz@lF^VIk!T&7hLXuIOJ{zb<`~`2Bma+uEeY zz6P&Fvt)<mZabad37lucJZDN?Hkh~{B4i8X3`*XK$oO^kNYO^XC;U5GAmpG`x_u9t z*%BupSAvjj?0esl=Hwp#**82+@6T~*jHG_J0g3(3lO<1;%r->H6yB)u?8@;feFbw9 zV5f;g>Nh`$VzVqE&KD$!J9rn_QWFslptpu;_trB|wCNs;CIK&~E^4j)K7|U-c@_$1 z0t~k}xzQUqNULlN;qou(n46N7u>}sAB#A4qDtcgJ^2|N=8t=BDh;S2MEiA57P8ul~ zINEi7;QO0Sz7O>&XYzXqLtWe*6C??^kTw=}FXEK7xchGUi%J_ORCK0db0^om@ZTj2 zH+pByZBX7PZxR6P&%`{XTS?*o;4+-@wA5~rl6kb}*KAOJ%t<PBKfR8sF3`}>owAt| z)LiK({27uA5QXcL{Ydn<9mrEQ%EfPRUqqbuGa+LelTYHf2%bwM6UW`3>|xK8u=@(9 zbgYpXgVdk~D$}1+@3l~=7Wo<~V*BgW8f_43PhEzGMqf>=z7#mao&H+d>Q(2)v$3h6 z2DkX6qwfDk@oktk12eJ8#~9&yOpu;1tVLy?MXfkMs$aNl{%91gij~|#Hd}PBf8q6j zuQ>R}k@*UIWEvN-%VUIKKsPHeFk?$kL%RRGZ5?Y>IdoIM=m`*Fu$Z;~_JUGdrR_-! z;q^gm+^aC{7ryR#bLKYY9ldJ$GcZ+D&v_G2cUdQoR(?RnvDo?b6PVQ@-kIN{`8q24 zqqO9X-zgW@@|EfkFG=B(p^c#V5O#;?b`K~;%5q()21$$*UzXBINmUy5J62z<2SyXw zVk{u1=Amp8dmmg2zoOiuIS;m5IY+QE3v$)teNU-}M=kRjx4|Vw@GKMF7!AptwYbm~ zAi#$07KH9g-b+~e4N%kze7oN&TVTr5M4~LEDSuT^lVazvM8M&-ZIkwZewc9>h7RJj zo>E+Vt2>yL6xwNwQ{Hac6PbRxRmP|0j<Q#Z&u1C=0w4&AY_2n&I|Mv*^3GoM!|MTj zLd=!<cwNu4Gs*|99rrsIc09DU$b-u9ySS1|U+sDfP&%(CaS3%B-L$hwk;FM%ILOri zHjKKczP)G7{`U(D8qhxX2OM@djAMK#9g4)6K#D5qvmYq&Rw=5IO>bgorfMVMjYYmO zKD_EM8cklRu8AsOEB&N6b@US|UYo%l3*K99z|GI)L!S585#XXI@_MR*Zgjtk|Dqfx zs!mg*7%%x&gu#NxG}TFz#s1?}=zd(U{tnH2RttJ&zn7Y<Ig3SEQ;2B?GM)cNe43|` ze1-Q9CiIed5hK<)+ObX4tD!RnHWM~u+Unaa*|f*|N6-#d>=D|R{(_(?t~zV61kMcy z5U1MQ%m4ukU%gTzW_&u2XJJYA7s7i@&DoBH$`AQojY!OkT{hk|`b$ZzgUn5O#_2Fu zV{v+y%Zu7Oa#f%F87z))=(?GM&p)DF?+?35|8_^D%K&#o{t?=iD3pA-&lJF7OA5jA zyEAqS9mMOsK?mhoO`9D}+U?kvvOy(Q>BSQ}a^iBE<O&kj10LMAeYw)?g`UC5_sbu$ zu+<n1UUd`$WSzz`!ak3of>zzOq}ifH;tQ|rPlEPpl8!}BzF)>BjLxMVXuiTQL+AeR zf>A<~L^%P;p<xaZV4^^thqox|Xd05aQVR1g)GxFNx<68^xm6<`Yj=GW^zn=x6GiRI zBReyEDm{p|$aS58H|`-GiOUSHWe&a^KTKaCOZ<SG>RRuWY#++07)c5tpxODwGk4uR zPKxDs2SmFC81p5UtO|(}NuV8bbnJWcS&c%@?^W*;+o+c2vW=)cy^QMLFK%*k_QPHM zz|lncwQPsVlxO~)C?4=>$ZM#cMDPW1uqSY_l}YN8LBH9=miP!$^;u$CoyDZme=Zld zQ_XUNRzYAEB|Wo<M1w&vm9>jO@SbT6+kDe`Yx6l)3{AloOygl|#4OZ!`|?QUJ{*VQ zx=^HQq_3_^H)9Kr_zP{KB9504F`LbB!?6}?op7j1<m0Kc8<-UvkytNaDhBPaU6f7w zgf10|0e087LFhXfW@!?@j&^ureG>7knvvMoOY|tw^7K}qYwK|v9zTou1@!_JGuSe< znV}qtf&L2X`<6P<u|QJ>&dAy6$6T|6Z*d~eh*3P~C!#Ss1_`!RB)|=6=64Pc%et|C zD>kslof`UZz&inXNKt&N4!2bIs;M5nRL@vHuSNF^^SzUpJ7YyP)uaV6dAXMml5zQH z@^pThQsI&knYF2ThE8PZ(Lw_)unk9Ni1w<tX?opuN`a%5V6lzjz@m9g&ups{6*Hm4 z>FkR_o(6tV(uyTCGxy8UJl|Ro{~{|tq0o}rkk6AfIxwCDor(;a+^zvcu6|WmxEMI+ z55<kV{y>M7w$8{btpA`BZ2)Epr0EoPM(X?RJyAcRF{WEx+_NU{sU(`H5e%3^bAQdr zsfvp4o!^lSh|KsgH~LNd&b-_7qi9k%hTf-k@a(?1<UeQJ7?1MjG%)IU-2h`9`nCfL zB;`TZY=L7!gz?;JPL3{(t3+F4Wi||B#1FT)m&lm-8{fQ?^iQuY-#g=eG010W=**gQ zR?@JL+ckBIF{4k$V_m7Ax7F($*#<CBFNV6)^n#pvB+A7j=ySGHj?~MD6Q9cPRCZ2u z=v9S38L2F&Xm0>+Fh?%Kqg38+DMQs}IH1UK$WXl0m>0#m+?p!G8F@=JM+h`qPF4#v zH@fcH>IBA>n`qO4MK6tAeyDYJ+#ZHEENSS|FuBpCG0HS;e^*ZU-Ob({1zsBSEJI1q z%?bRmSfFJu*otQ`ez;n1p03sayyo9{i~;)T+(<J+=G-y0e!sJ3CubxeRAF));KEXA zub2i;WIxnAJ8@KN*DXXY_`;p0b{Wyw#rr>AHjP*+xv$kjLN`}mbT4Mn@jxNArg(t1 zdW0s$p(le)gIdIsxpT2IB4567e-s!s>v~7VE|MT=A}ijO_h?;(mes!dwSTt(i0^P9 z>1@-W5Fc+kl;HQ9X^0UuGW<cVAvCC49c$Q|E;;m?r+CZP^&+ETJb5aI_YnaD?T*@Z zG@;sptUhZRhFPuceQdy<1rW4F_Jg{tKgl4tCWTapCM3|Q14LTn<8EG8QMEw376?8C zUVUG{D*gJh%pnkUWB-K2tyOKTonG~4<33Ds@vp4)?;IfD*=3FwhlF{s0W6Ajy7S~k z+<F1xcpREoJ@p_cTXNl~=t|R32DlE8VGDAhWS(scbH~Jr-PEc-hQ`mEa$7ugz_p4k z&0stPCG24QojWjmWs<6{R67dG+zZahW=a5&fZw!eBqwcMS)EB*ozt_}QoTk^)8IV= zqWO(B8Sb^wg9L|@UYQq6$95ESW<SLsncJtADr`8zsj-iP=0!xe$~geDuj0v%15Zat zEt86~T7z4WH`cooP2&Awq5E3Ls@jy@`X)l7p80{eXvZdHQteF8bH?1UrpR>u8bOMg z!`3vTeFDw*N*Yqpa1a?pHL%c{?9~ptI=aN=&8*C6tqvf3{M$#PP6>5h`)0>4XamUV zyg#liAE4Ndq2ao;1byW$qAH04E^q>PoAfUCdTcg|PISc@G@h?__LMl@W!|fuYzWOX zTl!YxzSB9yqTM-0oTuOVa#)6moCt?D{K^^M&?u7S;t#it4Kc9UQ=G!>$sms?fcP7A zF?m5yqvu0%mpn0YS9k~&U=5AY**cy4n3zVpw5FbFW;eI?t#Q2Y(sM*Z0%Bn91<<bB z7sJ18B@!}8kt@V&j%15EbJ5Hby-6l!HU{9|LM>9CKj45xq`8m$NCfaE)=fUWxYROQ zp&1NObF7<?0Y!Jt*5bri!*rRBZ`Aogi4`D=50{}i(5m!lQzpEQK{OcDwRetzGnxA# zzn#tBfel#oF<_t+Ai{Kr&^BPIa8y8uzd@USEx9r<eY{uu36mVM3``dn)kVYrA+EnJ zM<U22FtNpUnN&myDGNRT6`lOfe$N3e@OwbQ+r+?UN>0B=L4u7Fv|7L>38a<16ai$g zYXS0XMe&dJfPYcJO+I7wRV0EW1W!XSYl<Y<pZ&4{`_(j^znqan9)bNlTvwl~$p6fg z0WO|jn-WguD+Z?hM(cbq|L#2eCqQ5A@FyGq>-h_*{D(i_S1Q`)e+=~>xqP<3!Lc0f z9Z8Ws^k5OG3B>Y;Gda>H0URg0f!Dhe<C6ezh?_=uJv)D8q6d2>!kdOzV)*|@!2j3L z@gIR74NN)_zhwo&$KWvmy!sv5V4KqwFnC_y+vgh&o`sQ0rM`1W6CSCv(QX5;+86*e zi#J0f_*YBtwQbp5UZabjDj<rd(zr%0A{0l_A5M(r%dUSfSc_&5XA+NJ-DpR1JvhTt zr5{+(WFz&Qd<_tR4rSVK6f*yVg7N@qUiGc9N(%?Ug&deyf^vGiwj^qBa2Hg!q~yS3 zHjkc2I$Pj#-Qg&kE$KLwXFdJY-8!hF(9ZYe?>9f~Q<T7cjsb!vV0O~lU)`+gXahWm z8*4GKutY*!9#1fAP_IP>;k7}q)b@l35amh@VCnUmz2&a|QP@O<V0VDBbNn7L<S?)x zXrQi)cFrs8Opl&&4l+-oNnIr@*4v!!i4||1qx&=7#e`iXt6fBS+5(145=(JH^Q}md z+#og0Ekdc@<?$oW&Y8NS5pAsEtS2KQNX~6nCvXNz+hLvB4>z8-Ko*Gl_l`hTFBe&* z5L@?yE&7#)B@PFa(MvpRE8flqOip;aXYgCt4x@HZF2F642S;V_xswB~@Q>fZ_fy71 zQTiflpXa#DpytSLJ-$hoQ9H-f>ahGzB%&2M%)qyDn)G_r4-LFqtsdeGb5#cvNvz+B zl&q!&sK(9KKlxq#YW7Egoj9c5w|<f?`$IR&Fu31uXMrP2!6C6=?rhY!f0H#2Yc`#| zS{8_HDq6!JlZ{j9CJ(#(&brj*EPCY%FlFLV<Y|Y@GV?Ad2SY_KWwz;o+t^ODn}gHU zw2j;QQA(2mDAbcxa`ybT&Ev9~27m!y@TZG}Q`gN8DaUdWfcRnf;w-v!4oY!lk8e3o z8tF4Xnj8<pUUOY1bjc?EE3msl|Hq^t5${wtGsU?(5KfD+#gXF5?RL%o`7WV#FmE-d zI}~S$+wLy(ME1d+fAcf$Ti5W|NUDr?s%t$_4(gR$&1=IhZQO1jWh=k6VivHgUSh<0 zrE_}~PucG5Q{qz~8L(f06)BbiYj?Z-2Di<ws2fILYR4d$Oy?f@)UNSU@=XJ#sDoBD zsNr<SEPL?8-+SP5Q34Q>Tf9qS=!nMjY_4cI{W%x)mPM*)m$hUJWBB#WO`p7@Q#zmL zZac32WB~~rnMx{qG^IO%c<gvo+YU3k$@1mrz5f1P*^3qlQg!>S1b1ia;C`~PRPQc6 z!PnI>yIQ_Bc>r86nniXCGkta#F`?eA-s9ZaKt#T<VQ;eTcUvl?y0liERE5;)zR6W` zrFy_2L4Lu*a#8=o8J@8pgIv7u3eT|=amXGp?|1DN0S;SLTJ6`jO3=i?ybp#8iX;@T zM0*i*70KOWEUvBtU~AZ@&0(pA@?|*T5(P@M8cpzcKWbsizM^ZhR5P1cfznqzTkL!d z)owG_EV3Dazy1@2?I1`h8UPLI7XXO|@Dwt8*CJ?9`?(@9wWm6tBMy{NrpQ^N1#=tC zCM~|%Inz+zIa!UB>M6hSpd%c&Do#l{*{wOG3nzbGr6Zjikbc>j*IS8eV01;VS?mT+ z2Jn8B(D2+o?d7Jc;jb5q)wF@Y<yU%oXDa}uUorBEUt4b<vMPRuib03{L&$xtSxrl* zp*)l%z&RJM!}wl2dw({t;<>)tZIAMxWHBz$KV@w?0VB%LRL~W5NT?bMMAnP_SWztD zC4zP=QPugu9FED?y8n3NzK2`<vpfPbPLe=NO!_@x3EY=BfB@Ys^tnW=DD!h?1+ka% zCIHcDd>(h7$IXDa<aDM{b3f*YdP}`Jk4b1C1~|kwSj{C5dJCDqKDD&Cs9XdfYRy@# z$_zRJ-v!NNv^~5XgL95KCG@0fs`c6`pCYW;E{5qad)5SYiwu@4l&U-f(3!;^f!~zY zMGM!r)lV~$Sqe6iYo*UF9=R3uS9@iotBI}(Dnn?pXh`HEtCz!72=_b>-q|&-_v%hB zb|X!#DZgg^#B8`|PTSx1(+BIfLjYiI)pPW=GD!i)88|r5={u|PL!i&CmAe=$R?6=A zNN0TFyVE`Cc;(T`q_y=stnli%UQH|uDx~as4J|(|(qhM3nL7>5j8raLC2-6(8d0Iw ztb*PJ40ZaTB+m9TMyDCn_={xp@?(gK9=(~M$6D_?l{(>!auGY_>WPt+XO-3~QV&14 zH#6Sd3`l78=LhEc2)zWA*!gevN0XP_f9r|?Vy|~@^ixp*;<PVTcy@SC#WX5A6XUt& z`Gc(5dtylFg5Z#gO}L4R@!C@Dye%I!GqvC*C`ooTs5ghNgh=51fY${Sz8XM2grVCr zQBh}&VilUnG{r)|B(Q#(dLQbBmISp62PHim6juX*=u4{E>IW5GO-qfl(^yM&skW#Z zzIWnBl8Ecuqwc`yhszmoo!>2PR8Va`WN76HJz37}UMAw{$*VG$R@j|3JIE!GWbSy* z4K+*s4*qcFXIkUUhnH)%fQw1|&`<b+*8r7K6<$hBFn+$QH;d<KZ2qTHf8!FFo5K@L zL<h$+w7ZY#7wKNdS&MB;b;Wf;g@9q4h=;-2tW#RJ>ZWtdBix~J+M*&6)~2XaN?pWB zf^wP4t~(54WsI_u)z%rP%-u4z=q9V#@=$xC=_%3yq0fw1{Nj$KB=_$1^#&r@mo`$o z1kjlKz1N^@PUGzmv`VBF)YfwZl<%`?Y~>X<slUL(UtaOypT-1-L1(l%4J2CA>V)Jw z{)d6HQjP&$26zZ|ITD$uw$NBA(AA_fFR#X(Yl+JwG%$jC4}Zm>>#F=ok^pU(U_{jE zv0i-N@AJP1J4t}Bo0tt%?{uRsN*YvXlXj+jD0^$f<IwmKr;~;Vz2{N%j)W|-y6Ca? zaDta@O$2u3!+coA`5E}N2*$!f>>xL(L<-wDJmUU`CTByr@J0<H0nO&M()w^7z>55| zqCW*hU>~T2@wU6{`p{{JS>_Z!h9wkaHaSh9&OeXpV-nu2o*f+mtxbI{S}!^gbU>!; ziWWflq6Z6!uHrg0gK_}GBe}s7uTLM+Ua)5;c+ewYkWEfK-9H3>+PQ52zc)~+e_AJi zY+5E@&wB17!}m$=5_r34miQ+(!9;G-^88#NF?Z+Hv#kc;r3t?yfsaEFT7GwIMI~3& zQ|o0kNFDjI1Ouks5R;Z!faz-<UiFgMn5HGRMuJ}laKKpIgdXLf-%QfpDu4sUgDB?Z zy@5FDP;{Q4X7!4o4PuK;WVRh3!he_d<eA^<Y9GmYtMX%glEkB&pf{?UtopZwu#zBa zIw>b*waXUmhWHef;$LBJh$(;piIdrXkU%bx19l6p)}<TUt=tiIR{;<?UeU#Hhy7}g zSQVE7eGUFl{n~#)*~e3@f~VPAS2L%*zmKl|Wrt<g*Uz)&-Jco+)t5oI6S^i*$i4bV z2ZF?j9N`+q1V99{F#_tgJO5ck^#Bm*%T?_-4)$df+S&W#X~A4%w9q8+iZm%W$<w9w z=z|{TwVE&+rSu${kP2LXz^9al6iyVklsZjCxc>0?o{OTQ7-GZ%yf#94V=ZhpU6A4R z`gMop!dG>Pk4bN6?><LEkuAPk2uBB{!o<(9l~Sq>BGT7?{C-g<xnLJ9s@{+!l>&)i zoW+4RNos{7!cY~S?Q%ajE`PeVOEzlT^^W#3Syk_Oz0rg|<K3%NW{4{tHCttsie0M7 zJCH*G-_x~YxLx2Ox_Jlk`}5BbVPBr^FW{0&wTjbzyxSU>p`DUKJZwjcK$k5Q_3zg> z@4o7K0blpAgs1$@nHGZBDQQ7tu-g9W2Jm{G-yney#}@92e>I@s3BiConA#n%UOq|# zk;=?cONPAfBz@hD7&m&9cOUwr*K=Pqx?S9xReGIRg6Yn7PwH}T^l-Jy`fEIdqXrBx zhr07Kw?f9Q_R`k${7701ntAKwlisEl>Xbr#;;Rnic&_qggl|N=uU3;9s1F1FA6s7; z5J!|X8_3`efdqGVm*AQZAh^40a0u=i+}(l(cXtb}!QCB#JFh3Z`)zjj{bHKwneKc0 z$~kqaY8X^sI^cg|8mNYYA=5gX1C|9ZcjqTvP0-G2uIA45>B0$WH$0fNkG)>lT~QQM ztc&K52lVh7)|`2Vs)7}q+2~b8-=S);WfT<aE+5?Jd<BI;GZO$u6%d^#>;H;O-PFZy z+!zX}a+!zx#Dp-44ZG*|nseSKa~7^vH&E5|RJq(FfmnR03;-6K{Mb>RSAG{uovg|o z`OYsda@N9z+Hc^UBA;?sbJ`xAAwm*~HfALN`lX+Rqa$0#G_`pVp_P4q)RO<))Yh<p zkjABt-v*0pjTY1*$<&=M807;6PnRq;?gpYb<+k;yqY!$m#H^+a<=d-o9`TPyE^(Ay z`;tH{p&gP*6&<=+Bh$ZfZrq7jIfU8{|GOv5B6t1OP?0E&29bUsF==-5hRhxdbN11f z#C#M3xZvRwE0_nET<6&_fYxB7`XGM)OOu9QWfZdffG+eWa>!kfA``2yT=BEP7N8SM zDr&^+TY@uNVMv17MNFzIyMCj_6;x;S&J5_Xfi(vBPs|ODJB;d*CV}CZMA>yGHO(Zc zBXybds<0b$3)RLcPG$VWgVIM+qn+e)O7Ca>CZdVb0ofGVQ0n(+?{bVYp3dPmJ%B_p zgUzt+3jGOH)t~70u7RL*3*Rvu(f!(2$_<t#1IRJ(J3N87ccE)wUTX;Sgv{7E+>^v& zkMMJ3Y~GHd5X*Qc@>%jKRWdI!H6`2k%FLg^4Uy47x5~2#J|{vHwj#H>d7Z;F@?Ty6 z)s2THKuvbRe4IdWb;augR5c*N{N=NWTO<|I`!W(2?70kQaB|S91s)1aUxLf79Ts<f z9k_9DO@dfiId^lhZceEYI#P^kEv|p|rdISe5#<wB>5~-GuXLUUb(Bz#G~Yh1eol4R zQQiei7rOsu&-ny{)Ao&IdqA;c1m+BMrXY5<L=%q@Xs)Wpfz>Z7@(`WGIgUS`%2uXM zI1<@=`f!%2+MIR|IG=FS>8FA}d*uHJOG&OibS~3rqauCJJ~2b5N1v1|<u&4p<Opj= zNZ^|s(hhZq+i*z?%L~6D;K6E?+hP8}1Q)5J>uBj?yjYDGA3l=@P+h+Fcodu&V{jbY zf8bYmtd8D*X3o<M0cArnMB3Pj+Rl#64mO=q8f~l8E2#+Sy1nU%P6b1((Ae_81K6zk z=Crq%&a_j;&kJJhndrtAL*$Fqd}VL1L^u1X){EG70C5!CJ89{7Gt?F5norD~v*96! zaX0IQ3uc3IkcKIwYUO-vaNiui?}{atA^AkVFj>{Hf~gk4_>)Q46t;+MS!ml;+oidx zSn~riphn4+O6~mh2LC{+UAhTn=JY-9*56Dn!CWwR$!w3C3GNn9fuapFr1OzKrQDm8 z`29i!`2$+bpEHg_SA9q?=ILY~Gvswr1uAU)1d@oQ9<Lt8)Iqf8JI#|d{iw^Ms$9g2 zqz9S2o^-GqD7HxCB~YvvL0qCGDxcbmS<ZgNh0WVr*qX+`%YQ-1vF>S!Qcu!*rdTBG z^UIT#?}gdG(@!@V)A_(tl=T||4u5U@d9@^M{v0dru{?hAi}{`#0QxF%OtkT=x%!~z zIbzB5WcV)PZRB+3^uKX_dY6rQ3>LnkQ+j+fnQO_qR+<f9+IxvdFv$Z%7W&QzVqPcy z%+6Qq#7X!791?aqzd+28qFwTh5ZIf?jci7}D~DNk$&E`?T?NJ?t96Z8syF6=YM9bI zrNQQ;5nQQm;H^=*fUC}p79jOI77ybx2u2Iie>;qPx?IYVQ8(UB*c6do%4&`CjX6!v zjYH<C>Kow)E{Nhf2-ZY%3&pq6s$%~}Nyt#)z_}Tu1M){mekt&Fa5NDj@?iLT3vD5M z@<<H;X+dhN>&9k`s1Bam?ZOc!i{FchAzRdn0u`;sh4p*FiVmZgF(5Yd8jaz5G91+G zer|m1?mYoa)?|4QR4FRlWHkwiJLr_KY3jY&Wl&7$M#2u$fI1{+#(UzChQySt7h^t~ zf(NT99Q@dk>G)D*6(}L=d@%-v-4l$B&3gL+^WsWC-x<0*dE`va@Lc7Y_lq-wbV3*t zQ+Jsgp{Uu@bG|}8VGPeYBrez}a5w`s9xQagao8&@Fy6?oC18{3udw6PQ9zLicIGr6 za8~A!GqBypTUrIGzy|8qgjP+co98BDG1zIXc8(l2Gd6+gua`Ca78s<?A)SR@++>m% zvF*3&R+u0w24c(zfc%Q6vae3M_{wwBqvBo=YJ_m5t<o<5TU13ggS(o|U_n57=94p) zaczn-YT8GIuqbns`^eMh#O?=Gh!=%i8A>Fc>uBmH9?23e+nNCE5{w5=Wsz@nPPE(t z0s_h7&80+@LCYx<VJ0u4`O2z=6jCu#&|_+5UOoS@1bqOoQMwpT@9tRdNHnng;|S@Q zDe%^3I7$4GWh_lVZ`uDkWSX$!1b%7ji*|F#%ZCVj%Ccr-J>^J&LE@R-5M0LLd&Ck_ z8^N<n(ZNt!SBIMu`;&VgF*MPNBxZxwagc`qz)J^cX7^pJyYS9W9pp?2`h<Y4W>-bn zp&i(@;s@*#KuWp@6GX&c2UNqaGFoIOO-9gs351^|f$+0UrveAi(D)2Ng#S`aPtc0X z%WdB-7z)(WbUD4Ie_H;7?fKopn7~u;>)b6MIVOy*wG7q$oieV2nTfaOMgWuw83865 z*lR+bepnNOM0LMsdzL+?=9J&`eb6rsF2`Oi4o~B{tq;?B7V$o*Koh853pb^EPQy?I zmA@Uc;Bylt-2T$?F_2NS9Li-ZmDoh|-xHt>xJvTr37?7#U_=32&`@?a2~BuSB3^b4 zH(oU>HPk4c-1!+6o3%oyX_~%3zq7S$a}#Q2CN^^nmsR%q>69}__aTFKHES3tgej_~ zYu=HK>a^-VmbF_jJNh|{8xo+VdI7WTsoz&cF6@=24#4MU7|@aVzA<@cA{2f^PEx0R zU&2jpm0uFg7cUzgd21PrlmO6}vJd)WwZ*I848MxmA*ht~XRxUNrWgN*@417ETRKaQ zWyv5CFx&g5!C3UaLCAkOsr<#I7EA&&m?_@P|1Zy3@GBUy%y5|J1WZ0yfsKAt^y}w; z_tw9nA6szrvy@;XOoo~V#uFzOhkE~x5x_{W;53+DZhl0nA_VytaqI6#47}!-0{S1t znm-Zfe=z@;OmH{qzfJ)TJAtwQ-Pe}66u>sZ0l0QR1mWu6DaJnl^k6Xml;jml5O7!l zwz!c!`<oZ}kDmC$ExnjdUvK$9WDagD`%QfDf0NaHsKkJKiCd6i2>e4{t%hC60n_@@ zH8{z&5{|&UQ&@uZ?r6bjVw=qaqei#Gz}&GE5s$<iiSz(l^1mCv0Q8T2NI*RBzjaYC z=?R$UcNg`9TZu@;TSvdW$N*h^481i3wL)z~?r0SE&!p#-yVS?li?<(>Dg0VgzI#Tk zB1m@^e9rutHFhWopyYXr4fDueTRMPv8=SL69Iuix>-SAs+-H3;_!!TxU>bNC>}*XF zeb#KGGvFiOnb=%!ExD8C0=OK=pBwk$p!3~-6PB=1OXGG@b$Y21lfhe{9LwPQ_4JnN zcm9_9wo;_3mnj9Brc<YpEr3(b1Mn_jpU}?hc`IABGf;j*;ha4y%z6y#a??Th-cY!l zW3s|Phx$%C4;DMt@9$!OC;>2lqWd#QW4#&Cfk^SFfc?|<AvNihHaWl*^;&6k5%B^m zVYR8RUA-S}w|<kVal6Glo*i>G;rFBh2`BWej4Iz!m0dI#&ut3JC<0abD%V%=9M0Ax z09F+`8-3r(4@a(mswI`&8F_JOy%ukSvkJ7QwuCN{e+-S5c%#EfP1~qOu7AzmdrAG? zh|<&Fqz!ZcI3Q;nxmUbeUo9e5YJ8k`6m)gTYW?`9$(2vdwJg@dg5l6d`n@5fJI#uf z+89NbwhEE;zG>41t$0c#7g10aWS+jFD-9!Qe~S+OtMCCx$I=0u8@jHL7IiSi;YHYP zi;)gM*`x3^W`@$_p2*cg6OG2x;?Wh<OdDyy#NlKSS6I5FHFTGg9yzP<H)y`|n-VB_ zR<)e9x@>@tN6^0X{_4l)h2P&40SZ}_(}Sw6SiQ?Wq>(MhC-`_*4gOQsuIyIMWnG;L z0U+U_JXI=Z{1{UZ2PCNjmz~Sob3U!-A0JlET^ng;Vtl(do|oTGRw}E}v()WzO4?!4 z{dnL8^v20c^?q1H{dCJDzcE}3qR{bxaR9+wCNjd3^45O`90P644R|5{1o&<>HZuU) z`@~nt(f^uJ7oXEgR;6g6B0#Zl>gQ{dpVWV^2>xK|ubx=U6EHwP0p2Qb#^0ORV!uCJ zn?k?|$Fy2%vV2~PP%iRR_z0ldPtlI8x=XruR#WEmQUDoGt7(RokgoG+C4F%a=`t7{ zqs&ZYfGxSI33InZ=%sZIU`BV~<V*UEL6s%s<E=D*0yx%1yNEuJkVJzQCS!(RT!3A9 zH=30UsJ`#SzvVw&Z6-||%b4-i>mnNf<@AG5c7^Mu7#1t&WdbWO7pZZXFA8zqj#_2< zlJU|n6o7&T=P@L<019}Fcw<9gANl3k%Cfl?ea+sa#RjB>)w|unJJu@m6~&Lm@48ap z$Q$Uw0;Yq)H9i4VUkAdS{>Mr`BOnEt{PFV61z;NH-v^8pb};~Fjb9q~9>|wFI>Rt% zDAOkZ6wpxGvg<&u?*+e{8NhXcsw%veuRFQu*^wVYr}xS1%iNo&7uJ^6dGiO7luqlp zH?P}-T~ESLks$C^39<evUHTM)chKo&jj}KhZ#0-Li)y2;OqbpJ_9BRxu2R+)hnFx% z9hscB6CZs;rC{L{T8*i?**7KJpJ^75ebPfF@i=FGql^U0Gd}{>4S4z%PYtdi?yQb+ zLtIkrUo^>e!U;(nH5}vXX?{7()`sq1f3KQ5px73;S9G#^k}1>XHK~0n6f>dklLicI zG&GZ8P?Ly)lOR=;8k&S<Yu>`fBJo_pMaR$F&|(b$sBdmw)w8e^^BbTBh+MgNWB`@e zOS;{4|Dks|G?OX2C+sRygYw@07n*1`n7|3JRgSy4G$k;><J0!;+f3$%-&KSk0G|l_ zB+FaZ2>0S1skCE~ydfUm>eZz-Tdi-ScS^iXT)#JmP5@yl!DcIt^unMhIeK@_)E{8} z(8quraLj3<p&MQPirIgZ6>7o(fY~nl+tbEW|DnDWWrn1U;Exp@OfHm=n<WsFvuxBv z26jBMKj7v4Ho|1Jp3-sKjd*~0mFu$ucQ&1hEPNbQsjsI>0#Lhv!Zxu1RU5!O$3&;6 zm<}Z`WID%zLePg+R=sAv#A3%|s*HtPyvDEBiA`BhAD$N*hc!!=pr);Ik5D%36D|hG zuDRE{=}QrMqJ(UMi1o8WN=iBNF7(RT$iIuusS}$wPU8{}RT$Km*O;t_3BeX3JaxD| zD6rO%apX84{)SMVZx;6fM33HcFcamcHRanVm~k&)z8Ip^_{)ujC!Otc_<Z0=moF4L zZO4K+AWl$!cbp57RKb!qzLyI~SBrdNWx^6!U_%8xl`pq^f5WICM4lQ<UE!a`UX#er z+Wpol3NS*NZ+2+l@WrM<OO?@(`7vBxydyQkd}wo`>k^oYgIcQs#<4ZSQ~*NMvWjZj zd$21DB*rZs#Tf2igBapJ7y)3{kxl{U!P|^dC|~5~LWzM3bKC6&=b5U)Dh(E>I@(UO zVbo8SBfi)RlRv=p$}%)Vd^N=A;|Wk&JpqqaP5E;j@(+Hy-xNWV22Hh5Ue62kI>S%O zF$I#FXNX_EdGHr!&(5EG3qc+`;uOF1(NK?syP)Hos6{XtTqu9UKy<+2ULO1qVwq-v zC6x&HbHo5X9lx}&oo9Wj6KJEmSHm%K<^jhKGUFpI<@XN_K3b7S(~AAn-Ta-9usM?Q zm?p?&iBzLA;@Nr-->ML3#M3`gojJuX$ZyB)_^9`ZVLDy6!qOldop!;A816<!n4ps! zm@Gp>9Yk0F&LcBgeYIHjIIUZ5^h}%gnSkXEvz9=j{%KrnpB7BEWY@;h@vWZUF!ucC zgMZTg9=8A1KKc}bZJ(wW70QA^yvg7Nfuce{SJ>2S-m8XO{ur=1GDbQq<&M;N{@$a+ z91jWZ&EYvzcc6^K*a(kVt2qdX3o`}~N$5(8Th`ZP%vV^I`2lL0J25T=i@4K6gMQYR z@8Xw2<Mto{PO$7fYG21eg?TZhm`g7++5GqitX!V@cV(~)T+3SD6O&3?9p>{`9O-B8 z1T|$72u+6~jnToTTY$VTU$~{w#11+7d2d-62u}(&6N%>`Bu!Q6QrM(=$9VD$Oo_Wq z6PE7sN~-i}?q?pFtepjN;NL&o`wn)&{`mUVwm+(Ly5VzzYkR2sQk{giNe8U)XtHti zbhbq+{Jq<P?X9AP`Ol6JRtSh+upN4~C*YV%M$udK>kgfo-}&EBBher@ST?GQK_G@G zCjt*Zpqdg3*u9t)lExLQZWs>Wze)}B1<I1}F&4%)AXZpy)Gbtup*O)!<b%plvnvb* z{()Ti3)=-~@Td)h5Rb=D3CtBymCX_<yOb!WiF(Jl)Xr0qJblSU&d+Q)C_TJ}-`RSn zXH6$%ntPM)+npb2Vjr9xP=~)zsYsu(W2u{d<5DD#7sD^P7nTnTz+MuK4w=wq$b3)g zkvcuz1=xyIUwyC8{e85r!0mL#i2oK@O;(ROfX_|^k<{8`*W3f5@>a`;A-$L-i3FLD zj#b(&7~tiaT9krmgJq4s_^tMTSZJ95i0+f4iZzSN0#tPc7f^*ijh0JKg)4zSEKKsC z`#Zpu18esR8_P8@;l9#<VH<E^-vdWw7O%i+?hf)ZMw~#!u;)(i8m#e7`UN1~$~e}5 z+Q46;k^_evrACmUu>l=BRzDJ)%=GF2$Qd)zDy2piOBL#+wu5E!2_U>D{}w}m(Hr8e z_6iKx^Lvg`T@SWPUA$>uo%UL!vmjMHPLfKvwtd~4!D*ZoL3!<2`Z?EilY^1%rN!54 zxRLtMbM_=hud@qLf9a*;jpcHDVb|y~P=5%8qI)C?b;RcT)ay#ScGiT}f=(_?G0M<$ z9heX3R#-3JF4)(5%{s+vW~JAK)o7?L><<2&g##=sfXef56Gck|VNC)qY?K(OD_z$+ z;QZ4;>9q+46G2&u4J~2`z=^NE3Axn|Zgd3lg?)B2AH$ysL(jJX7QrHApx|YichAGb ze7RfAn0WvJ8u@etsA2Udn*Wo7A7pZ!T7m9y<gsz@9r~U_$cRYPy^cB5<#KN#Jr(rT z8nTHr#Jf5##+FlTi9Pm;t$|K?ID?83g;4f)1+?Vk7B&8kewE-tX)+$pqZ@|O9I&xw zC0xZl9LQla+3zD%4iX5sQdt$>nXsp~7^liWuZ3!<!WoZxz}zemC-(r@#oiVaMY6Yj zCfIK9|Hw*bo<O?Q;)T)(iGD@_20|fz3<TaJz*p%MD5<2opj;J1;>mCP+V!1NqCnUN z4CS_O=nuEHYp`oD-!0640=7UU`+}dg@zEC|m=_3@l~T;gJRbQBF%HSqspg596rXkg zA2oaylofyk0xZs{PV2{80x*WE&7aAW;bT^<?>Qm*j1WrqE*|GuljrJNdx8l>a2&CT z;Ls^2ESS1Nu!a-9O#Chx$2!uL3$o0Hz}yTCy=jiPlb<vm%}?#lG3}fas(N~G)UW49 zVE9&gm@ouDS4xus2ZNa9-!QEa>>nkK2A1kv9J5FyMXO$qNuUtyZK3#(hrlv4uT=?7 z!jO4+H(PsP7SkC^y-Y88dU`^B65wTg6vqZvO%~~qOLei*?022e4}3ljPUJqU2^!2m zO`~M~9TtD(0K(#oR)I8pI8k=6*#xmF?}oOJlptc|Qh+vwDh)C8yB?N?Cn0YkW}kao zKI3_;kW3;phEjB(B$Yb&hFb@Y7nM7T90Ot!<+)J=)pF?zv617+MppwRiFTrvgNVYW ze4s_JkLSUQr*=JLJ)q2cljxi2d3zc;9?HMh*W)ds|GMWf6rZ!qsX!F<_QrxHad7NW zi$=O`vV*RE__0l=@<YN1Ucr34?1oeh*~fU=HRN<Y47~S|DeJoUSa?`--GJQju)*Tz zQBr{ob4Y&4+M4-SAZ<*K$CwvN-&{^^TxTVj$Bu@dajIxSC;<g&*{F5&Vq??l5r=RY z0}L9)1ViSyPbZl?*sn)jQ#&!VZDwo6xNCf|@8J&`t0NEH`G8W~la_-K{q9o2e=v(y zz}U^r3BHL8<jMztSHk+L4N%Ovvp*v|!BjyYt4Q?Bxi;I;E0Eg^;4bGi0@<}1V9`-p zGJ&@YC;iH97tu=@M&|8ey9TX|o1bbBq|o}W$x}%nA=I%!adcH$#nFR49`d}^(e<2~ zX<$rob8oV+u=cYcBNYT7KA%Gi_R}mDae%eG2;?792U9D=95@>#rD{wfaE?Q@x50gD z_u5yH+pQ^A^qv~C3>XdVJe24aJkGWdLKq3UsO-`{Yu9G|Z81Fq@#tfI-H;>)G5ZD} zY$55Fxm4f0m4rxj2Uj5~_RZ9>U_+Q$jfQk5LQV;m-9eLl)HM?u@FWIGS4Lpf+3ab% z>Jd|Ni<snx<piw~(N`xFrX_1jy-a=<Y?`^!K2Uo7DMt8JhR|0_$^2;o>pzx@uA9MJ zI<WHL!eHfZKTJ_#U{>dh8{1KntwDkLTC7shBd35e5nLCHVUc!KI6eyz$Y1Aq2RK3o z-H$H`n!4p^9-<f9`b{zRgdwvJ!O~?%p-m|Xt*T;3{<usg^AwgS=2AKJmsmc=on@={ z4~cCX=L01Wwmpfve%7Yiuip8(-q;L(1T}NLxBl8H{_T4iO=oFGj}NMv0OhXrka9pi zA9-d4#1*Z}j~I-vX-$0FWyO_6hBhh!oru4J!hm2{?4GZw7Fz)MUcz?Xg@4_V5cR#Y zLCRN?Z41t6$ckpC|0sd}*SbZU<bq+ys<Ig~>wKV3n~~_UL!Y^+gzk+&ld(9rEg+gw zu|Z#QbRpVOmJlx~S?N4jL<Z$|6!{e&9+HB&?=0Z3_A5bX+3nwUyxu`zGUYmM&wjMv z8!&VQ$$1%V_wi`?R%9TKFS3S!jR8^uxj7p^6e+=cb{raK$%HHnLFHfknZL%DP)#$* zFF`W++*Bfq9PXW%+?RaSF#le6i6B0M*`*lH0rn*S$nij@^uBvKJkILpZ|SL-4SFNi zCPW0502LmM%5Yi?hr-Ob^Nt8#N3X-_b*XQIHU&Zv3cY}4g1JYcHm#A-9b%%-+}~yf z-V$JzDB5Dj$0+YH@Jm*Wk%0f<g5Ch&*-oole{!rhC-`M}b07EL0||@-0q8XFjsfoI zb_|dl0ddp-ZpZrnVv~^IikgkMCD_)&2@LPyf4cq)qy1;Jfhq%j3q#fE6C~MM41jqi zs;d4kp?3{THglQ$V#7d&dIiQN@7;zm|L5js!4K$4<0lm0ZVSFerN!QX|2fGT+^p1x zQKeWxL*OJVwJq-dj|E5+a1jI+8V*2Z5BnNiTz^H<{C|7o|Gj|K0l$^V=mzYdHsHq> z4RiW`2=4#A;060nqb$9V2?6CW`A`uj-2wbQ0k0?GkJ^$9MXZJqU5ndlpPB(X%QW5= zo%0rTDgD0CW3^ZWzj<Dddr^io9tOBrvRFfE?HN9R@BIP!xly5N$3+YP@^PTQS9}X5 z`u~XKQB`(t0Py1ctkdITB6l;`+g7Ro%s1tBK2PFxyH@;zTjhu$Og+zI!+8sS7-bse zU2)^q5)Z-P)K2o|E%$U@0mRUDZRh4mm%B*mz_b-QS#HE<Xcyo$djg30)9vl@d1G52 zr#3W!rn&<TAFPD)rVOzlUVp(ju%vvB7N~;8H~iihDzxU1`1rd|Wb|7%Ti=a2Da^|p zacxR%Vin<x{Z}l^FW`zPAFgsVkk?C2Wr$qGaCow+0F)>aupB<&NkU?fB@$4mr{7TA zx4flQSV<9~x$6G{=&;ERB6!jHTpfy6W*N_d9)gj>Yf+)G(#2(FedCKS-q*Q(;%;$H z3{z_o+n|(sW4!0|^q4-3voLgE4G8=77=EgyUvw3~`EMaIzT;H&T&z(CmlK}Xsl^KK z_&hIV7nI!;Zl(;ncn5MlDer{UxbzAitMu|NldC=e9<~HVi7l?rY-;Ua>kyL~6{%9h z=B;+RK6DrWW9MS7biL;TktEp0%Mvi&GQ6mQl0TkOK9{VVEk*LFV6M>;0YI^0hj$zS z3Ak9vYg!$UCNmY<=Lfh(x%#aXK5dWxsns{kI&LlXq3t{>$8Y}`_ne-4PC4S+L&ZDO z@(-KR_Qf(lmL|i0JS`vz&jOfSdb0t*9T?^XKwNT{<72xRX*HNWxs9gc3taTd&SUY% zhmL=`2Ov28?H&LyS@=)}+P?mRe4TtWxo@yNzVh-><EokI<9VD<jnfMkKwUn;=D!<* zz@9l#xO!31ki3iJcNA#U<?T!+{MFc~$?v7^`uUzlQOcLuGMDoTvl9_3Wcko5%PSHn z-7!oy+JkEkZ2%abY<<kB@(D6p8q6Q1FUCT6&N3a(t_S>lLti)Vj%A2_^y$!xUU#>@ z|4EEJS?BYH@xh1fU`gP^cQK=rG7zg^$PqhkXNhJhw0xP9Nb6@X>%6u7g|59shgyX( zBNl)?NC3q&MlRoFzBn=WcRv0iGO$nYZ`%5{wJ$p@iJh3ZzFj8=y%&?sa_j>c?NJT{ z+@w_HU$s)Bj=s&(vv|Xahz2DQsDDeX*rRMP`6>)Zg>-LDSssfW^%yJxUh7gpiI#^L zQ@0Aliw5FLDq{Q8S#$WGG7bq~0#hAKGGYr%4#5supcGH_KJOXx5rAZ7US5|9r;E#1 z^rZ6CDTYp)LA|7SNQUs)46rc|pZpWWtSmtFgMSJ21PDgJ_EHa2*@aH|7w`mb>jBJO z5HM^b2Pn|aE;MHnS4H1pws(Hf4sMrvlgT738NGLoN7Ww9;3B!*Bn}U)+&Sv90oc}! zKp0gOHIyUTp5!Xy{6%Z_#Wj`fl|?|uPQY$KB!0qI8f}ce7Q6M5H2xna7-Vt7+!s6z zX`NO`VY~lkMJK9OZi4od?JgEDd$+2LBa&pP_Fzu9_FhKmF*lFKJ!?-)V)!j&K4#E_ zzZX?*P48u8_i7-#FS<b}e5wO$qXv>UUp_$pDrf!d_I?=6zRByRE>{UF*c6x4Sjr9K z@E+hb+c~^shJGd)NO|8LA+~uG6T+n!yc(m}%f{+vkZI4h4Ifes4MFR8KWx6tUGjYV z*7&rHni(jtS;lbKbI?01w>m25^haRKt^hwQ1Hhv_L}Bwo>#fa3mN!`UODI|uGp_!p zKmV|xQ`{UdfnS6Rod|fU=%1+aGTES%%xP1A0bRH$HRf*6CFRT)+sE@NsSl*lmh-xv z39^D6Dz`@+0Vu!q^j3QwK_5q})ka3!mAiK6R8H^#XNlGI(=nfa6#xM2uOb}>$;>GD z6u+6-Ja~jA<KUHVr?erE3yV)_gIj~)T1y*M!t$vI4cMm8|8At%9(1%imN;b;1*Fr5 zs~Ex7(ERyp4-Uc}kpz4VlxlH~nj}{b!;N$8&Z;+HCyG;N{o@>5+Bz@y&g%ZiLJywo zF@|gM!pa{^V_*jLDVnEq?oiw>Ayv48Hjp#^ZL5oL?+2+eRYPUlT`9N|=li*5K6HsE zPvND`q=RM@WZl<0qvTl^yGvL%JMY0hzm6Q&=CfAAZ(rKUgf_^K%ZbG2ioOSdb3L-B zm-%M+uSKDZ?t+pya?vL>n4*Hh5Y+ooaZz6gc<(2cmmz)qYyAC5vf^NULx~`LseGx3 zgXIsr+I;L^s&9^$zcyO0Vtcl<zIQ(9;j}+Jad#Vaw%c*M^XZ>;l|6Zz=NDu$`W>6~ z&?Lx%fUi_95jp$CYb7hH$%uNDl~MlZUh<6D2}%tv>HS^EE?Gw6{4W!T4Avdjn`;+C z$DV$_Ld}<3ypEKQ4|AIb559KR&f_f8V`s7bfcGK2-wrW<JLxCDYKQA&rSbczV~NQy zVTQ|&{p$<QkIPfHOFO4hi0ox|4?CG(W|<N@X+OP=D_;&auRqeyh%pcnL`4PVU_uZN z=U{3VX#xk9usv{Hh&>;JIJ=r3<BOCd59$xI+m{@RFH(jQpS#_h#*g$`YrYM1Y7v^T z`1U{NH$m%sF+qZ>bqKdX#(K}L;jpED?|75oSR-i_xaQjd$0LUi{n>^%?KbTW-sg6k zwa?t@9qUWuhEZI+W|HGmsM_X!EXvBP%Q%!z04nA34C=2065HXePwi*D?Izu!mx}8f zaWu=MHK*TSFsO*$E+W7C1eHOXQ;c8AmNDw?JZ<Tl6~g{?g(i{rVr1cN%JgthEP>+9 zD3Ya`gmqbJAmaE|s#=@V(5PaYlL0I360?C%Rq?L$#O{bE(a%>qn4+)+ie^)C1;nxO zPAU+iB$n^8*y}uw<!Mo13X3n!iWs)=2S=5^xJjU#)DGs1@{~4B`c}TyVjGq<R##Q| zajmTJV<{_uVa(&pbP*?gGSA?T*DzzWH#5quLCuy;$R2g+SfB9@s`A$ul8A(a7n-eF zRi1D^`(uBY*AAyK6MB#v)Cc7fz{UlH!Un9QC08J8T;d5!jsE6)jehbiU4uSs{v70g zxYd@fIb$LtMu+858l$~?wctN)!G3T^(KJKzJwz?GraQ#1xsJMi^;xmqgDSpAefA)N z@#5eV#|QzoI2s1VZ>MBDRn+-#7JvR#mF@oYgnH`Fl48Yp_312;&wSR#Z7#eGBdI)d z-pN63){`mep^8s0Psv1jkoP+s1X_r$F5@HJnheP-Buf{slQ!=RpwrhL%JlkIY8)0` zW{GCpRJQZm%tuALnl}BfmhB&T5Bpz_*0=j6bv{9(&q-OceaV}7SzK`ax!59Vh)3i% z+rp~EIWXqpbz&BkYSYU;%iE_3C~;E+jJ=2W!o}UyymoHyZj+h{@~P-0+{~A)bM34c zi>t@dxPB}*pc(pN%)H95C8wR8RBM~Ac3y;5&&|NSo;K=Xqq%+jq+Hj`Zkb+#uPZ%_ zKOxbWR4-?EvN)sWiPa&-nv<DH%F~+XnROSTT0W+Ey22k>5^?(#74z^VA0C#)zy?Ly z-L5*Rpy{AdImfqH>_O$hX@3m*!cC~sPHFmc6x`*mr6~Jwjy~Qko|lP3(vUsQX3<2; zYbAHxXhas*4~zHrtihQ<RDLGzs^8S_VFI%eW}2cbxvVKbz5A-Qq78&5!z^}mu&|K6 zf(}GP{%Q_cG>-6)zTOy+w!Tb$0rL}#*EIM!_^ubXKS&1QTUDi=<TJ%I5Z7mADZ3C~ z2N_9(!2kO7_NR{NucZf9+*^`I!{`mg%U_`mVpi}%mc&=-O&>J8){+L$ENjlhSdYJB z?+q<4?vED`Dm<GYC=}?$#`g;Azi~_zq|j49?t;ffvD|R7(#+Fxma>b0d#mU>az=}3 zu@}BaAeNEF<0^X7zTELU!Qxx_(*_d!LGQa2(<OB;w3TwUii&8_m7ij`4pAUg30(!m z7>F5YqB<g1<ZnC$TM;&EjE@b_JUSH_zQyRVj<1dBRa_d=evn}x3@^r-XXp)FI!vg* zqWu)7DaE(s)iZ(}g0gfV?uo3tOs@*o-S!8iB}x;1_flglfJ7&gNr|g~>sf->)W+D= z@bFf`zlSN#jET1}hYVpBCRVu2taO%0%BBlu1c7el)Ci6=+l8$^;5hFW@4nfsOor(6 z_Gb@hdn_r5;+J1=Z@D|Kn?Dy#A{N%Taou)!Qke1h(eI1I%+W`58XCt?PQQC?c_%o= z`9UBIDui=@3pzWjvV$i3L-OqYN~;^2x|+FAF}$e8y8JfmjLrM$cX}TFZ{)g$XI-wG z!@JOTs}|nL5Y9=LKh$H|hA5uhk+_aDYqX#Ighnk_HApI!M}b8yVO(OYc-PJ2EJ+zj zFq5D*lpj7b7G_tAL;X|}<hUhx@!WF(>%*$c{LKV`ZunMv{IE(MNm`IV@$jvy<sQN% zPV&*#1M-!kM{#(Yi4zq%0R&=HkfK7g?EV{{`~7_O`}u3j%E@V&=$#Do=EGL)dbn2E z4fgO|*F3FY?o+0(Ii4IYsfod1uRUnA@;Ho?ZVa)E@Lwaa>YSRPI@kx^)Y)`hu*q%o zTmno4Eb1>SQ*C7e&Rk}+3mE;TQ!$OMd_cr15j`DQeCHyMky0^(J>OW=R<9Qlt6?^) zrFes?Ie&&{G3CX^r?c_vVWxrLz$b5G{2J*>Q&~mbphEwBUdH^QGQ-SrO2Yw0EB{&F zGLF-%^VaXBGfg}03pLb<vCqIrxBB|PyWx3zk28N2N!1j4?=+oRKknVbifs5h0rT|0 zr7MyyK%44Ev)}GuloQgWEl|LYf4^}tX#U&n{_7QbX6CxgX6O-@AkB@Hy3!5yDzlxy zU5V%OS`cXor#;@de=mx#O~Dbral(N-WD3rVSo5yaJ2oM&9iFS$i)@deU80x5uDf%Z zf6P-7@I1|-bwmK>DZUSvP?BvyI@>TWnz4<TU>pNQ9mkLGk-nQ9a}{#Yp-ofY)Pyz> zT&%JDKCOFdyYEe3VrnC$AYac=*o$B0bvFyx-klG4u<7&%)XnI01PyCgHNJSWEf&y0 zp;IR^eF@^EuYeZPnQuj7kRnp4P}}(FS-3junT%_9p<)>3Blc4oa!3cC#M%F~*E*-I zXr`VL=#WHJuTlx>{5Yi2`<Fax-go0B#5PVBP-oR>nI6pK&QtOxTL`qABiZnn^`Jb8 zKh-yz?tpxi%|X{qCbKK#aMq<j=-RRZFY;5)C%z$b7~I0&e2eD3Lra=&B*WvoJK27v z#Kh`2nhx5ETk^&Hqz?QD<DB~otG(vy59KHm{@OPWU&=U(SvEUDm|n|ONJrNYeN|;d zFub57q$>zQ=NHX;1JQXR#`~pV^crB`&5ouTKJfGmEQjj4S)bcTL)YPQDUW;-_6RpC zLItK$CU`34#8x=H*5h%SS6()G-u&{kFaRAz?UAZHn-H?o3YVClZ9awgNka7-Gj*?H zYB>?Mo*DGASc#U*&Xd_n5Z73fiA~6%fJeyoN|2lL&YG;WutBNb2=Q-^{u#5M&7~#! z&4rbu$+$+mDyW^P9^$L%=yZnfl^BZ#kTBjiV0qTQupaXalle}yh&td=3=BVATxA3f zZC4<Nca315c$xaI6L6fGdCd^s^gbWt^7Tzn9!LAnrz2fhsO$0kdMyt*7VQ4HeQhIy z$4lxRzN}~#cl$NF<!SMq=gH5udkO#S(-Nmjjx7@*Lw*51G~+gRg5;-5C%pM9FWr7D zD;L<OQR8J8_00FT>#zL_?$Qj0*8^eE<1k?SJKe0uY})#sapr9W7a&C61v}T-h~nNZ z_3-qgNIznSxIZ&P-Qa4v=_Ny3@VN8e#eDVH#NH+oSo_2$FAl_ZAP6s1)UIK|n%oj1 zB3Mx@pDfDZBOTPr{gl!Ylp_YJiiKCnonH_t@L)@IguLVe$%B$3cy0REs&=?9zE1S$ zE%YK|Ni2}il)Lf0p$IwpVqW+y1im4oi+v!|^LHQ~P$R01c`$Ce;IeWZVNUKpfrWYQ z5-{G!%9FbaR!aLwV2#Bc21R1s$ucITd2@?NXhBcHe^X$Ww?A9LgQ<x1N-K`so6m<0 z@}tcsH4@)AUq?i1)O5`lS(p;IUBuJPs>W;4q{>>@dX(9P=Mf8YI}V3O&2;>QEMJPF zIq8t0jd-upo4R>@I$I=!hZo!Jss%3V8Ann=3#<gmYmhvWD11%8eT&ks0Je2$7P{4n zx{(@9w3vz{1i1v*xtdhCd+7?@U@E<xv6lO0@rkjYsKz#}DMgy~dc^7<pE##N=P_z} z0l}u#rD4c^I1?-9rlx>Eq|_&I{{Af*x=FzvBIZD+^N0(eq-gMD>V1Ah!*kI+MsN4( z?z-m?bB3m3(y`znz#?sjJZF=MKL5z0%kA?GTQftGJ;d(AEZOioQm;%}<My;*hCY{t z`{jin6D@&Wve?b;#oPBgijgk(5kkf*_lJyD5CLu|3}+RvV&_3joBUy3FQnh%#PA>K zg7<bQ4{S(=-Jf`c@!Kdex`WHSJOhb0><NtGjVK<oAobHuiN9DJz%<8W>7!Q{Iug3N z1vYCMwyZai@*sbw@wknwLJZtysZrN6L#Us~4@KPqNq;Pv0JwRFbf#~R)1KO{zT+p_ zyKbMp+PQ%(sET3yvqCBo11qE=G4&M?)_60*g81~H@o5^QZ8pNapJvys`B$Z<U;<SA z=ba_mvOaofetcc#rz1T?Jj$xGKIy|VXN0s{AxC@ds`Qp6_#=rk8O%~Tz>P9`Iyn5? zp2Xz$2}$$Ia8H6$zLD=#{%p%A?+)Wo{uQp*dS>mNx0+&epdP77)2pU8U(W)81rQ?` zBx8x$eVVS_sHGZfsqiI2hsO5}56^Xv|NOmJHEffu)qVIR!#87YSSFF(-X+b8z(q2a zw0p#xB$;126DRgI5Jo;$s#dxpMxT#x4y>w;(2cKoys>{W%;a~wScY=YWr(a2EWMSP zh2F(xZR0$;K^&}*`IVHH+8uJmDujmCHN5&~1v`x<9@Y&Hb?a5|{qpV-fVerkhL@k< zEL2omn=}$DNcMjfRw1ArI;?VUcekP>6tBRtHaLtbHpv_@AgVNV<qcZAH7GHVk-0!u z+7x`>wlZb0^Z8d<=rK$L!du`6W>Ri5uAQHLe?2Zg31(2ma4u3?kVA<1XbSqi_V7!K zZ)v9Uv%n;%v}mYrmqap&8_iW_40815`Yyzi8_oD+rnz~HlpvL{T;_0zdbREOt{?uh z6;e{8l!Z>0`4s+skj1?I*dpsv5q`ll&-P-!kLz!jdZosg4VSSS`g#+};i$1y51S*# z-?c7xkoSU(rl|>~g%UBqq5=y!5#o(%JP}ehO%EJ6f&7`=ykWh}42vrjXe5Dn;HEZ2 z_s%HaD{k(X9uu3%;>3MHY=qZDAW&><z`9Kl$I`6^8JA=1!#JOBEhah!gA+b>+I!Di zZ*=w3593xM6_8%{m{R`Onq=>l-JqgOJHLA9J~rAO!{>Q!$(6eKTRO3KLKET$MVbue zv(KxtrNXhADs%=+I9P9*&*KMi&i;%<)C)WmTG7Q|HJI`o!hd)7fk<2^%5~9_Sr0lL zU!m|}f|5mpNBqX3<usum;Hq)y)&{|54#bmp`r7&}sV6m(4pY$T*;`Hr(W@3Ldo*~q zRMo{X`eXaTbsPtkQa`1uQ0n&`MLuz{_+yu&m<Copkj|n=XDOOZnx_-FvB@&{%?6_y zkISgn>ozSq4kUaFHePg@d$Sxj6}UjXx-7nURPBN&4|igm<kh0T^kHXb!Av{H9M5!Y z)-6RVd*mwkF#6VI@TR8y@YIuj{iTVtsU;+*I-c>TLkT09^A%C%@~CZCo5@&l4qky| zUd>z1eG_p}I9AUcDb>bhcws+|q<7T<2wVblu&k;C-F6sbQ;KFU*4RjNXY;<i4P21+ zpM`IDy;d^T3um?Lnsn07k8haS_|P!l!D02?Ww;1Q3{1OP9#oIw>-EXMZGb*Wsi3)J zuWb8%*PU8EE6M!_Q6LEg#sHd_Oc@3U<P%JUN(sl>%%4DsR2}TO9yFvryHMVwYAXuT zePG8!Lb?~ih?8Yyn@(b~zcIQZB`0ffT`|NLs&1%~x+u^=BOfMmUCJ@JHMP91?8yzl zUNEM8g;iaD6SP`Q{+21QSwTPVZGXt2ZWm%S-1-mSpHW9EKaJ2T(Non|G)V=j{IM5= zQ?GTll5GTWS2#4JlCs<P>I-@{s}WjW950{)rl}8eKKG{TJ@EP6yOb@W;1EWT84q#A zX6dT%H(TI~4(dVl_+uaJaR25gryAnAYZAAWHUzrPOc3@VT)b3qJWRQ8^z;o{S6C*- z$qcy#dGg7K46b>B`-Ax6tdI|Ap-De_#to#m$YB}QHvQ*OJ}*s;KXj!CSAZIXFP>1I z>Rm`5_Fui*;X59?x9<uGL03pcFq&tXlKHTXH`cjT+g#)M<~;2{&w`$n3Td#r2}YS8 z+$*17CHv~pkdVMS%YY|03P)F12;tO1WMxJBxn@TIr`V6_8MBh|45U*Ds_SdSdWI@| zuH|x>yOzK?`MAufo$7beXC9`M!?*MPZct_S6@fPrk%&0yM1{wPlN@v|w13w2IB@hE z*F_x<2dwRHK^t%M>MW+rg=@G?U4*8-DL9}S7pAs{)g7^FHWS;rp^h)Nx};;3*9WEU zve*!&@MINZX#`!#1EBCFwG+gofAYN&A<2<c5^?$IbZ>$TDyToG#wv?GW|(oU?RrS3 z>*!GpGo(~YM8X+^0Jr@-@gm7)Kr}Vj=Kgm|-uez<g0AXYK;=Z#&|f)(Q!*sp(%d<j z?Bj;LBpjW@^ssquO@k_knQo%*{&Zl(_fnD1%sCrPKKyml7uuqBFPFpvYZ~hVU{_jA z=REAW7cu{QE0WYWr6=NOit;6;BN%Ggh4m<c$>3PPP4Jbd4<6xy_|5oIuC0TZs2)fy z2sG3L;Gkypvaw7Tj@oiT?G|rlMFW|-AN?M?Tj4m@DXVq_<3U3cH%f+DC48ddVde;| zGc=+aJFc(vkkRZ|CCLZ5tP0c5@36aYt)}%p2cI<<9#9Py_Gz|@6k_iAkhZ4W1OPGT zFRsVNg5O7CJtC@zo$%$kp)q%|IijFMao8==WRDw@r+MZLz{qK)Y4yX~$}9F$svFsD z9AxIZ9X@DV+fS+JqEJ7fWn`|-cmqc?C6n!!oCCx<!Yh|G%sonj(T$I=%+(Qc7{MbZ z5&-%#<Ll$eIU$|ao-g9w5jrvMR>|C5U%gyCe>MxW;|u)sJ{K*9-xNhIXgB;o!uiW` zP8n%|**PYJvF>k^OofK)?q0%y783(IJl;%zD4RQJJw4&!v$A}Y-Fg|xN1+Gp!HU{s zvwp;BRT;5?RP2hsC&YJ%PVV%}WGa6CxdNIX$I9N{Tm$W2qm2|i+C*qD!I%gPc(gff z6pMtOH4uJ8ok0rtlsGG@?0{xG2Z}&4E8@JjF&avt6nJtt@PFP2VLbEODa)<dC$#r} z%V15!Dai7MTEQoV>{Gs`DTldlPYs@47vdIFkMp{4p;N(qqE{3mDqi`-5&wi=pZjrQ zJFrkdHhRZ3k5^wRMsE--v+AcXy;ftYlb|ha^$$nVlqsWi#?}orYznBre8rJFKJ4R! zW}P6H0LO}Vx>OO9Kzsh$9^Q@;NSzn15O;G@m_*Vuzo?6RDhTh&j<0BwFCv0__pNch z3jQ6;GqC<M9{%1ryf+7BzN&m(uN@Wip$*+Q-w~PQfluoav#H3Jp2!r^rh9=Ijuiyk zv4aGcWc)KGyiL{+s;4?y>R)M6DG;MJ5XkLfh2fi)qAXny$z($Wn&vJm5rHCbLeH;z zl<{QbSeDIrvXLW}VB9lZ8gsEW0uQ;E{Ecyb>dDCTcq_D}Gp)G}tDt^{&G)&sDd$sb zNTNXI*RB>Fa}E;1xa8?z&)6ZS>GR<L2BnZgn8Fymt~G_|J`|iOy76;~KsCcf5}fJq zF9h--x#S;?H37Ib$>Y_f6y#@ggT;`>np`)p@!!|zlox2nV#)Sda+m%{J6JcNKjAYq z5hg?%gP4v4@Dt1S9|qa?X^<{2uFEeP#VpLmlw;fi6tJu_?cFAX{IB`Q3mi}4taV;s zNqnd}-0I3^(PA@|RvA&%ZaYTl?^!!)c6t^d&5&kIGo~_SGRHym83nh)OquJ<+{n*D zFsj6`qUPI@@ZS{MP^7xTW^o{0r!6Psx`(g~R%*oM-i-xaTfdpgBp$E3yhNOiA0qq* zDh>-q#bG}bfiov}VImnc<9BG*cs9cc#&c&K9xVSv?zzP9(nt0Yyp}nZ3oXN;o9Ld; z-b&9fZDNea`9`2=E^_&N6$I5Gs#V!1EUG+KA<$-gjyJ-ywmb`Haxd*uY?C29M4-Sh z4n}Nz*+ki2E3<0iw~RQW<pAxhH<?lB-1p8fpKxr^RarU<`G`%zAefjVU{UHBoWL|! zCSe)VkyyRtWtKL#G}e`-@kM79>A!v2<<LW`6E}Mg^!r%V&o0J|xeiQuITr3t5~JrN z2rOp*a5@es0e5FbYqn#C4;m^AIj!c^JrFqFYDRe`tW6dOi6iXId&xO3G9JbgUQv;F z<gpJhH;J*K4Nd2VVuNv{iJ`;Sc&+a}nT{X6Vk9e*X|A{%LH4)Rjj-%6)A?rM1|kul zYnHFT!+pnoEmCs@t&zmN%&PhJzQz2+kf+MJAb^j<$OY33G^-TL)D8UxM{}HWvleCJ zXyENPv#N2jVcGWbi_og)61g~MlHsLNof-4g8rPstxm*pqC-Y50hho_>WV`}dVo7u^ zp%2)|UX0HU30b5-RLZ-Ysfh_2KE|(UQTxs9{7yytySem&>7gRSE)Zv<K}+$wR56Jx z&rUzpS>C@Bu92156Ire!X7KrBGYTxcPI!f@t8O^6uO1XuJ@a;`xVOFXE9}f(V!Kw# zj_p1lHhH+0WtB985+~wm%xO+^$fhO{XMWr+b(tXU;DwM<Ry!q+Dte_I_|Vd2ur_3J z`LF@noASD{fGZq++NiV{CLNQu9BUg=QLnWxBLn+t9sbFTnVCligJ?c{rOwWo5|&o8 z8quNL=K=#-_76CihlE%v6{y{FclNI`XO(bTr1OerlmNGYlDf*WvqwQSGWRw?gEu(( z75<!X)ky>}nq(@)yKDl^LQZ}*{#Y~uesd&mR+-PbbtVl_4sE>=k7jF9+OU*OGU!Z% z(2-B)L=OWw_U1n6FnWO?fFU0Y2L$Wi;NtR-fCF972A*CYeo*RbP3R+0^vuWu75C&o zb_~%r4_l>*`p9s&B}RpO84CfUMnjg5{*|5P4B_aNHwh$*j0N`-FqfQiIXyCB`s{@M z!;WI1osNANdOo%t=<urgyesG&jv=vRzf}wBd3e)dY+)_xTR!uh3O7{czbm^E8yZr; zmD%-v&9PE6mFGPF$Q|!I7*soNBzBqSLe}ae=NpgB5kR5WH=A>GMVm-nL=s<7beZ6u zt}=~*Yw;#k7BvVJBC^N?r7zTpSoZYf&gWtxT~`{ii>7hXa$}@cA@$FCwtk$^*Ls<! z9=m6<K3U#e;#l{C8P3`s4_MMJc=-D+-rXd<50m9&^8XcCZxXG*mlR}?$Y@4<L-sM& zwmRC?Xx*!sLM4|b-1nQ%H6-I+yGu7Me2RS$WDi;ewB8Q&x<vHEE?m~Xya3w2JZ6{p zO}*zw(YL_6MI)(HCmm48KWFoY&D?~Yy3IzxnvD6!GoS`P1M1v7eBc?pg0xLwLdf4Y zKwza9_BFF)!BPPwX?8I_pdJTN%-mHM)GiX{pf1q!X_g}oJM&UF`>W|*tGZ$i!+Pi9 zPyfiML-)lzM;Pqo%eE^AQYia{Hm76?!UriA6|oe$cf(A#yc`jD@NacgOQD4Iodvvn z-9W<a#~!c=wsZYZ@?0e@(oZ6a<sUETEnM8VUH*~l|5w#QIEkAW!|zTwsh4Q3a_J3+ zti@X?Qc;0g%16o7oY`^HYS@bUw@Rl0Oy6k8@-hmAUv<9QiP*DfBu>4m+vk*Qu1#_I z{q%JwN|wiV3jwF`D=A+ZiBU=S??pxdvspETRQj3sm_Ep$A+95l)ix*U(lpi(ciY5T z9okc<KSP!dQaDKrOQ=yBSfd|>1Yudd#zXU~#xd@WyIckfK-u4iJV;WS7=DSPLWmMQ zs)h84s36W5kLwL3(I<Y__zX%7i5^<Jx8Hd%h~su+aHUh2d1t18g9d6Lel;t-cLM<0 zI7@11S}Yc2?#ey-*0cQu7dB!79*-7-MZQJ3zfpfu<TLps;5uCB+76-R8~ivfw4_w+ z)9VLKM6Yvbt~HWeE1ZMx4di1gAqZP-8l;K?hT&cVTMHu7mh2x)?$2)X=jnsr5+Y(k zL+zqt+7)vphXv6B?SncT9&Gsc`{0W0pCecjV5}hI48&!_LfXCpzHy_2?Bm-%{qoN< z{`n{P6c$zwYgNTNOb0bs4LrcqTJqTc>#~2o)&~|`lmZFHegfY}7x-g(dt1W)-&2!> zfd(ZBt1qAdw*!1Ex#ac#_dIWK!@V;}_Opmk6~W)AI7RIT{>*=!O~eImH`lI`IM8ky zaJvtNKeYePY2f>$1KLoX6$QRtFa)?g5K(z>|MhkMXwOeDG_`dWTMTqdhX8Qu1xopc z#AhH=aM+Brh~qXPkc=V{$~}yQ2l?oVua&2A?w$AQ>{W)TQK_B3>Eu<!PW=OF_<H~I z&s*?!Kt`ou0gseQj7SFE_^lVur}s1ESs*wJdB7%kKU*}U#rJ$_AH!j<`q*Ko`TOOV zzzvl|oABlEP#)Fl6yw)cjvw$#&CZE|byk8X1s^`PQjDnhs|gn$AncBg=jWY{?^{O} z{wB*`3dqPFL{gUxq?<2gvAtSb^?J$E!d00FuooDgVvJXO`Q(o~IQt-wXAp!k9=S^R zJ+5d<)`?v}OsDiiq|qQw86C?b{b^mytC7pPwwH$U_u&0Ho$oij*25FJ(MWpcIqwx- zYwnqyM3??C;nEcvzRK<M)h^OtM*XY!AM!!x0uTD$Bxf&QmK&VnyVR$1cXeFCQvwlh zU*NdW|KkA$JA)tWX}%^j9c&K`q-_LXxai17pTRZ^X$6#%<Brz1X55@%V4N(o?+i|? z>l%+r+g~Qd4(m=NjVw4EoYmrXt#q2>lt+_Avp08XgpbsUJyp|@ER_<+DXYD>2+l`w zB-2mOiY+zYH~fH2hcbYfywM??)GEI<X8kcy(#nn)Wd=yMazFEb)!EIHOpOqXA<}z1 zh6yilAA8K4`(58sMuq(JyRJ>h6OgCQcTp{WkHlaScdb<9HeWG7C6`R=!TB9hjy&13 za`PKHgUF)M{TD@Pkz{v*^1LA5t?89pjqA#r19n^fZMrkXEGLC~a&UJ1uR%eCIs$J0 z$g59R!2i~K1=9fR54-y7xeZUt0|sP~-OaU?^#5b)tfR8px;2h8(jXup(j_4&CEYFE zAkrXR(%mJ}-67p6Al=<9AYIbZcfFo-?)}a;?q3YXcw_JN?zPsO&-^`eF}aWr+2}|t zS0%ffaoD6*715SF;dD7?k|iFKon_YFbdpMEi9=>mxZfH%)S7A(qaFCnMTnYSKah8W z@7FXro8cu!!UUT2$voa)5nF0Cs%xYU=WK^2WYy^#Xb<$pafn|YW(S1Q!(bxfO`Vy@ zMW-?GTyAc&<>^rm#1}x-9&vnm=|RTo^X%4P(p;0zprj7=fY&99-Oy`L3lILcPx-!^ za)Pmgs>a&1HL+CQtJ!RSbY2lPuiLnM)Pww!Lrw=QUX_N1mkjjduIxo6QH0LB(%pk< z)0?Adcac=*=Ed82$5WyNw*rCNH`9Yt)%uhk$4-Ro<e!^gc>o#cKc;)I7-Y_h+2omn zIg1zs87;m?!|FO^UklZZ5s!;NV%Ttwj*qM2guBZ~!iC0Rk;+*|T8vQi$(-SrR$B5z z@Uoc&VXf<SBg-i%9A%TE{He+_ya~0%hM=c3O|)IcIVBNqLqdLlW1hKn?Q~hZJE|6x z(NKI$hcW%MKm5r3?I=GR;2qBQukEJ{4qhDE_0G=eWt}<fZrj6J<iwgym<es}9k(eu zi3(LplMme4+z~d}?^_>sAjr}<?+)aJou4JeB-acivWw?x3*EB0_&u~<Q?nxz$sonu z(gzFj=wBZFZeM6&a=3eY7-)V;TO$Z(Dar2H-5X1$V(u}pdTQ+9j^c}Ch{c4!<+3-Y zc7ELA7F+#eGFECW2=;~NTbEQUYS|(i`a(&~)^t5V-v_%(wtJ^F6W5<7)HJ{Zyx<!) zdV97vc=K?WkU?{BxS+h#o$&Rnf|haGtMy)3lgSFBMYjNH+O;``&-vG6l<7*Jr*gg~ zZ_#R5ko2`R+CkQCsG+@Dg0$epRsyjXNjQ$xP(|Qdu&;?S70jr935i*%H}3SXJt<&~ zRV@{42&R&~tY{$!9lB@Lh%V~IY2-e-w;V9yuxx0e6#i#F`fK<44Ozv1Z>x*ZAi`on z+X8(A{&8VbTDeM5HewYNf@?)1I64mMs;rtk_Tv|R6gb$b2vb%EUgna!IV-Nx4h)Fx za=S?JaFk5p${XWjFqYm#DvL-mvMwH;RP7qnM5^^AYA`mLjeT(Q+Ttu7RW7+BWSl)( zsft`ldFu;$sz~Oo7H-&$xZ8gkMN8%_NubKfRJpkAKAYL)z)|q^93j9eVhH^W+b8@v zI28a>rM(CV0<7&Fhox5fm{s4kSd4P15GHx!1+~V2(EDD)@%TtbRwC&}iv&d<A9241 zpYTdUB)e^MxboKtK$mKfX-aC|clRJMhE<IDju2@>4a3fknS{pIRK3sQH2Gvb$c{yA z!`s)s9U50RaP9hI@Foy`MUTUZhUFDLrHW-z$gIiVnd8lrU2U0D(Z#gYcXij9nS#Yo z&O^mBPks~6-;G*_tYu#B2f33QEaxgY7}DN^R@c@Vsu`WYC1~!z*!jYl%8PolXoR^R z5ht`4^Iwg|U*SzO1+eZM>I#3ANP23_{ugxee+!m8C<M}{%haGKVIX3_uA}H1_YSIc z40#f*_<i!;#787D4RKVXz|~$%1<b78B<T8A+x)(zj&xyRmyS{qy<V~btD|&S>%&C* z(L=w8W|Zwf=t2Lsb`jUfG-)3Hbfj+a1J$&};__zdbs!S!Pr-L>A_04$Y_|b`gU$`} z>2s-)uw}^h64g*+mNabIu+(V`o*Bu=U%q}12QLICbZ>)G8)IR-*!nW(rL%uh^S<@H zqtb;3iw{$YdO7X1=}Nl}Gu`OwDUfracV=<kSv6Hab$#-|k>G~a?$h{Tc5~E+2Ybks zi=T37A<gwUEUjpjdX&KNNLa&uGsnI_PTg=~Zc~dvn%zSm_ThI^>*MXQ9{0te+Ct0o zVU)yL8I5M6DF(9OuE6f<uL=b#%8HHF$zb9mPt}__rm-9E8t26%th_z68?wNg11JV+ z-stBLCz@d{b#XHEulZHx(g_dDn^MhU21veWB1DSHOZ*3+$m`Ext0?{zruh600z&zT z^@vfIufRt#jB+3>$LhR0GP1E3@h#S@rLz2{DK9Afv2&dZn1z4utCU^~G5J^wh?XkH zE#TU|2bhyXsft+hRfJC6^yZMi?qLnd!><5h8nSi1@pia5xbh#!qF)DpeN81gibG=E z@xMqqE!(7OvhCMi_`Pxu1+&~cGGmC&XiS!8o8Wma)qR`W-!L}gu(yLrz+%nLn||$S zDD~O+w$3tBMDmN~W(WI`K_Ev53S*9eu;#=vk8@4Hi{Qjp5t@pCz|hGKv9o`O-`@W< z;GdM{u8$%R-|iTXJvyGrMYn^rnI8%T-$$oeSA{#jZTI~+=(u$}-joI}>1PCx*_w|x zA~f4jV2MjsF=R-UB9~OFSG~?CWq0OP_j}t*@q8a8Y=m{V+<b9NpI~a)Z+Q(gjd*_d z$1kDPqt{T)2;4NrtIdz*F0R#(C{>`C+3UP(zrm(`bwrHu{XS(jcrw|O0Ogwe-+Q(` zWEWy`u*-tf!l8UVi6*wE9o!{N?mbFciuwn;8CgsfbmZw>ArkEi3s$tr6_>?6Hg?2P zn6X67CQ=_Sw+Iszw`7MP@5CmVi4@ewI!-vV8Slp1j5B57O&8n7*>qfOU2qzAlqlYB zDYuyM?6W~r45?cAso>D6=dCTO@-4j+eA!`#jH@kCvJJlucVRfHHI_4-*4h`v52}Hm z?I#feO(WQY7s!25Y1N9O7yE)A;yIeZ;7XqVc&lx&MrPrh0>*^QQm<Y{l%u7*8a`q6 zVfeTizSDKTalO20nv-?${A!L+|8jn-7IwK>&+9><IY6lGh<!6uqsUHnPzO6cR_WGx zJpMh2lHON!VeICL$S#5(hMEL+>DS<UmhY4PRI{UN8qgNp@J7-|oi4%;?)|t0KL<|H z7BdK88mazO^!q0i61Wov(f)6aZI%y$ByTW-)%@{c?xV0i1=(;RV!@*Q$`6C>uvjU4 zV5AUkHPOvY7*w5bl?sW2i!X54SWxlx_DC;CpM-ueo`-uCN)e3ioZCH*y*g1?HjK1j zEGrD3n9%rwRUe)O9akfj6ZBtYe05_?K9c`;iMav87^5R&s=&|4BdCL<B$PIcJF>}C z1;Yo%+CNEy5*qG1ob^#KAGg~MPqEH<#ZVRJW&K+qy|p5dy2+_N<if?0KgOiWTC5~V zEvhoqA+n~v$5Um5<t<ClEtRMckU8tiNhMlSv-ZbVL$P3neK}Xc){1ttkk9_`gK7Y& zYyDfa2NHpqF|@|h(z<eGib=_0=XX^<X@@2mdu76h7u>%WxF7Lg8%MWzjnR-5)RmuB zEz(porhVb~XY~GBo(03enjpwi%fci`LV_8b;Ez8F+gy_5D(yb9OcxwN9{S$93}@-| z^C{guU4yjign+>?o%;10!w{eLn~a&)$(yQ)_kj9~t8gdA`!J}}miRmjlssJprBaox z+{N4GavqMSRfYP>7jkQ>-dTinB2s%HvRe7;i0{`TDa<Hca4nup&z|Bk<AU-)nn13s zy170>#SSm*@vWRr33)uX(^L?)4EKC#Q!1m8nnY^G_1c4tRS8G=V^&TsUo0oHQGb$} z7Hfyp(Pe2{-x1YEYjRM8o5dTyMVRe8d}cLF!YSw=^<Vje7bJdu+|m{U`2!WSpnPI! z5`P?E{>2zv(Xwi!VwYt2)A$5CdESs(6X<WTyQ6q8@2KnC-(y{*)FoDJjf@s7wbqrM ze{ou@j=?6;d@rd|qK#v|;E_34d)Dhe?n9*YRrYcG+&x51XC%owJCuoazl#~UR5;4K zqez|}mCcPyZ>5=h2m{G};k%WlR<Y6X+8P<V9jt;A(MA0?rM5rsXmi<Uva=v6VyN3I z?#mtP4U@4GPxPp?r}L<DAp)eGBxf-{Ytc^4MYE&G99;tAgp24YUo@o_CvwYVZ>ypY zy{1AIo?iY)r#89o!Yz>scqq%;uF!iX>dn}R4rBa~5-<8I|IRc2`XlfQQV{tyS{eQh z1CavOH<Zcm{oTW1%Fc<5M(rk^_e!l9J44DFO{BBg?3+U&25BY?l;f{7NpAI})YiB% z`?w=xGhFO(goF;GoFMT@iX>W)*Jn*o+<M`{xp+tygBAm9=ehka>jnf;khXu+nPFLv zoe)gtCDkbtaNc@EG1`OjnRoZp6Oh*Wqbr<^wa1b<5-juAX-@dh4;&Kk#KLjGhBZC= zs;IT`%jriXQzttKvLHeBa(1~&RgoYewrlM)V|9LWQQKze(+IylEaRQmWahZEIxCbn z6#l;j$AJDACRHqeAz_2p;(S$>-qt1g<$235_nk;49<R%1sFt(uv@`_07+eOFRVmeb z$Ff*UDP2$n-O9_!99@>z`0%G|lX_;#HjGtFU;J8#D3mq^TkQ+eCs($eku4D`{gAc& z{}MF;cOVf}|7MwFkpnUd2I8xZ@b2y>hyE&yv@W5_$xHZ?r2#WrkZcLZ{ee>IGgB5- z6G<I9%M(lrr2^?=PI5PuTnTk_A5LGf=^S~({EZ;~*`{|#OZ3Ys=STje08x*avd9jV zF#T*rb9>nC!Ny(^)`}uB=(V?6kTSD#wJbJV(Z`>*wk2<f#H!r*eu)Iy^Lq}6&tfSw zSeEsI>)ZYg+O4hu`^DC1E#6jonu=J>N*y`zlaHu&a2cfiV+;CT54N(++#RmR5Q;uv znP#wiZ1vRYM^(0=m~tnI&@GJC^t1}OBGnS?>sP4)BV-Yt*4TM2?}s@xOL<KL1Pl3` zvrH~r^d1^OM!Kw>C+@T}8C6@m9+o$tRK&KsBQ9D7t(W_9W9wZyb;e}xZ&rhM&K_tM zxE5DEwREn!YwL(DJPjwV(%{+1UVpg$G_0t}?u^C0yO1HJT)0&_X!l3)`rm|)1QC)F zAX2LYQXs-IK?{O?w%g9#l+cyT0&S1otW~t?`+!OJ0g``VG|AJ?0-n6ZeTf{4j<a3K z^<2>NaQmd{z9$#2W9h{mwVbCJ16ImR6p=RvXYSgV`1MTiqQzU4G+G1ovT_0n6XXbt z9Ty#x#svs%Y`I-+Udtly?`228r`4i{ZPR31)dpamHw?_0%WUqmRaN*KO@oZ0@1?}| zqNO!&7*u8VYW!)_nU%a}h=q9rPk(i(A(Xm{c(~^&qT*Cmi*l;H1DSFhALkG7J^mW2 zv|%(_E6_7W7TfZZWREW>v8e@opCEiLOT|EhR$yf{>>tt4bVjzYx(RGK2lZRk)_NG~ zj)S(xhrCicTr*@2jL_6Er~AvSSXwv6k%@AQB|Tp2(8vd~bZ=yvPC1W0ykd?>*WKa( zd~r{sajzP}`CEj2!hQExGV;T+>8*yJ>i^EWK0yZg)3&M@6(TGdL^!sG#>Qk?gSZZ$ z>qRpIL<W56mJQ@N<89s6{D#^dz6m;8uVzoZ!pc@6JSsyln=Ku&fTiPTc-|ykNN9+G zC4o1&IihpDGtlgLQPsm_v{$p1{NB0eVKU)eytYecT9gd@67TEBn=m^)nXe3m-=;UM zU92xef4&VC`&<|tZbsczrq?0_Y>-o`c9xwzBDt<nvjTmOqo6NZcTS7tZNLuo`H(;+ zZ#r!X?tOhiNXQjtqt}nR7m5WXh!4G8u#$xTxhsO_1_*+`eoD;3`jCi_>>ySPr01Z9 znP8J~p(QqkvuySe_31bsEdKON+Zj5sHeKJ7r!Qvh?KrvX0_h8#{(byZgS4J&-iL@! zkz!pzZEk}I=PN(grfhd|ZQ9Yubc?lV|6`r-Cc&76<VR)0iRWq<!Hba1$?wcrt8weX zS|O%to3@bSk$KJ`mK|py6Pi~g7nU6rAH&EI-a^zy*xzY+%vm)kOcB4tAac*PJ&j#v znRP5=pXt*FWwqBA5RBsg)}$^{NDL|*L#cxU>kE-^KYYR>&3**B#DHS{b4sZLx2dk@ zNBIW!Ksi$=OT6wdGvd<4h?0L_mSt-!_88u4BVxK^<IMX<kVa%h%f-kF{Ym!k6oWGg zgcG)&PmQ2}E(OtSIoWk4H2(FPzozOsH5lXWQI`Y-Wa!EO@TSUuZ#91?kpH;iKa1%P zR`AMel(=WlAS$Q;_)Hoog`$6p&i`o7x{y26b<cAG-w1$dL{_!8Khyqi1pZ@jhJv6b zP;#LTk{E~|z`Kyw`{T0yJzjrajRE`<`ndG0v>PCCg@;^ogv%FQ2r=+qAB#~AnG?G5 zZ78}J!M2d=u<gqe{D1BoaIxT(Lwa4g-@r$Rz=I!x%r9O0|Gjq%kVWX@k~2w@gYWwV z`OOSz|L-RL;}T%sLLS+>XOakTKADhl&rFuX5C5-AgzQf35MZvItDNPPAjvyS#HW82 ziJarMhc6Rvzoe)4C6m9aL~HVdh_EB+zvG{_{=x!+CB;q;HO$kf@2~kE1iW<H#IDml zN$>jAhXmWOsAQg-96dsmu{dcwntSXd8nC+&GzxUTsms5JqHCMHC{EY~l-pz{t>Vp8 z#N)$lg6G3p?hXB?SVgeSO|q{%?e5)PY5_-0_%;<N=1Yq2zVuik6tTU$n_hiAp5djP zN8fm==nIRqalo0o@n_t-s3GGkU(H~E1N(v4CzE7Ij`%hdaGaw3FXiMmR#vHg%A%AO z_uq4mpS<QTXew=1oKHs0$(;s~k*fW?;^?Myxk}C65YRj~8PDmB2DRQ{lg}V=1vA4Q zEwzfKS;S5pg5(uMoc5SBU*%&sc8*iLCUV(%VDhA2-kx-il%9|5Fqw~KQO9}apuw@! zy+pu@pyqh3CqVp-Cx#m(T|&N1xuYN+2#u@^&v`y`qos++<1ijUZdSC{ency!(&GUr zdmzDQcdazkFrRid{OC-*u)v?s*Op&6TA5D@Q>cE;kzGpk_3xFJWDl}N>{Y}j199^^ z543G4Q>U@%Vo|qpjs`*k>a*6RsDp`E7UMxtu<yTXdq(|>&yE%o%?G0Le^a>M2>|8k zSEyQnJl3P%jd4>J<b<gwV*<kF<;8dIw-@<y;Wps8bZ;k$aSAoA1Y>_?1-%&T+QbGe z33GBoe|K|7mj0|kd*8xZCQjINKFGDlf5<Bp^H4dr_o2aQgIgb$W{!K<gY$cfp@=B& z1D0|9LzASw^{nEI&~+gY1aw_@Zg+P+ja;v|9>j(Nd~qxdcP38G!_8lee&xM}TMyda zMA6?5E2#YGZy`(lLCCY8`f!RSf{Zg<btGNF+HqgLTd#b}x4rH04!edTbWkAYWVkVA zbv$^)T4B5@?yF0bAYbjX=>}GPRiNQ2XXz>}BQzL!)IKWR9CX&617YsmHH}Jn|4@$a z;&mC0xz;YFRNNaPpl7hpG#yzI-Q6CyJu%tDn?+(xKbY&8SzNN`Z)p+xb4w6(q4uo6 zER057Ow=fJZ(CdWl7V|KRNj9+x`oGVW-J|&b{8qPxj1~M{bRF&%x+RJEk#Ka7307^ zpFjl%usAfgC$|RY-6NK;6DWG-86J(XGiol%W>D?i8{(pu-zj6*haQ?-{21@u<@U=3 zf;LN<9w74Y<y*L&N*N7yCYrO(6YHIk(l%O)qq}qU9(t=MoyNPpO1--a$8EIr-%ajj zdfK-(wppo>Qr5zy3>bvt4;T-%?+?{{b>#sRgR`3;akbk+tVS~$GpeA8uKIT6Cg5ya zE+aE2reMtS0Vuz~FVv#uD!_G$ll_zS*n&EXbxa<*=UDE!P=Ky`QW3kEFes_#0h4pX z`Lw(J;i>i07-@nt!y;6QDVQskDKBBO&eZ!OE%qa?nrZPH6#U`A<(D9n7DDMtOp<eV z45qrYD%f`TxgIe?JM@@$#y=jAF`_7Bq?1T1{!&OQKQF8CC1^mGGc(BC<bHzZDri~4 zEGO6Ya7TWh%OssaioMyV7i{%-&!MAyk?%(DHzO+4IM}9}NjO_(x=g6l95G9}G3<C+ zTy`6DOj}&#ZL0~B)AVL`zhS>`_lBipFbIos{gt-vFms;NEn*tKo7B}sbZ^Gk(HG;@ z-$L4FgY%RA>{q5eLm*hPl1e_%=R0&Hf<jh5HB>u8cF)3tYat)1=e@ril!q;444<CL zY$`qCXjmcJWc*!IS6HpLzsox_+i#6F+0yLn&|KkgCX%Sr<{BVJhy0}`inr{@Yja}1 zog32v*F*XXTT(jp=L9*VjoXtc=ko^Fbi@9nS%a2uWfQ!Yu%P_&jOgZ^94|qpM!$C^ z<97Cdd}Hv6G@Ue&dMW|5$gM6MtSldensO6co~?gumE@Yf%TpXxW4%@!1#;Qcr%?=> z_A~rZd^gCPgm@9ttyjkZ;#tCO`Qdki-9CKJ_H_00CQ|@`d~J?k0k+?5n37mlqtQ*4 zgz=M9))SVf;ZbP!qclZ3n8%goZC_SC`G4vKv(=LXn~vT$Ez2ei;$SMPFBg2JNzlw_ zuqNO5V#b$3c6PmgIA6A{Hq3>^A9GH4v5kIl4I5Df3LbmcTkf}XGpyRfmGGU$^tBoB zSATD(PXfX2^yCfc1d$*~A7(JN|2)6B@Z?&=FGeq;qUoHQLs!$oY%53I9QV<05~MjM zd$c^?Ob~`&&V;>coexJlx{=VLI?-<ZTK0sC-9mz9`R4ZZ2$eef^vCgW*C@Ybt(oy6 zt{CkBLFwYoA4rV;F@vJ$D@!f*De<Gvr2*;X%yqAQ!ZIMmIMn3XU2<~Xk8r&I4LxBP z`1`CZg3b4Rm^jsdSGWRcZ@waX=2SB72y@|T{85tXbC@MUcpmg9&bUhC(TtF#Z<O7| z4NjB3-*%~8KkMrEWEfQ<pnm29^rT!Kn1zEpdA;N;I=tOkl@8<6zPcaw7^^lI0HZEq zFEHxTR_(otaJ7!1!mgy%EL`x3Pfqf-4NDhhJMnSXMJHLyBo2spHesrQj)?a;B<9x1 zuKi_~*!b+LU9OT@Du3N#1wB1hXAxrV^LZzNlOu%lzKuufCkoWTgf+{63uX-CM-2C_ z?Xx69Pg#-^rqY7#fp`_{t>wW%JEoY9m-cV-K_S=_`5UO~X^2*|@}Q?=hwzOoHCw_k zBy+~DYL@GW$>jKsw&kw6X(7l+4b3&k$2=WF^ACJKH<TBvff<)tF^_JcAidFV@O{e} z7=X`#B&aRZ7GI<&ZQ*iF)^vzQz2j}^;-$}tTAeccn2GP_w)0@?r+(!)tsP(Z!#r!R zjJ}K(I0mOJ357ziD7Rki429&dfy}T4jSKtYgGWM%c3Aa!akFEbZ1g1{H4E5}Wom&- zdM>h*|LQeIDEC^ORlZo7>wfK~?8flo(#8X*wTLuq;~!zaMF%8Nams}%dOGWvwO?&v zk~!P401mNw9N8*y2THE$agT~NAI!U;5dDdt{WuEO%vFxQI0RcCi|qRSkZGyiBOP9_ zjJrCC^x8)*R<)g)2!gcgrm=IovNdxM{y=p!rDOi4_CE9ETxR@>YC8WFe(>P@K-1n% zd&|tlr#I7P(<&D2qRO4&8M4vWcNd%U4^d>;SuP>OUCFe?I|*@)Y*-R^;B>Wp7k2)P z62S#D^#ZG|=KX2bd!$8;zsOaPF-Sn-LWycp4@_8JVu(#p|IFh&ti{3b7_CLaO$E~^ zK$kwBo~O!mALk=%8X~I<DQu3ka$(sU)3&jq#+dVx4=|_y>*m#1^qv#i<8j2)7Y_56 zg*}(8@cAyr__v?;BA@QDI^gSMML#2<nMd?Vkd8kIUql3q=gDfVEsn?iwF%?T7uH{F zXy0U%E0<*HtXzK7k}rsSIeu@=Ve@She%qq$Q+2Q_b07yn{mhK?ZJgY5M5><=+BJd* zl2E%ovIyF`mHM!iaL8=xQ(Tntgp%4ywUQQ=?fxe$8XWIu>@3^Ylna1MQx$!AKW;KX zPUGt9${35w#kUC^x@-x=*!k*B7+zcq$>|mI;=wa`G;0AW!5{dz459<y%HF75nbuSt zbmZe3ssR_R^3NKJ8iV_%vJY#?08S3}=TzMN;`7tbh@yl<Ak*i^lLwCi`iWdzdOgo( zoAiA2gTSqtF&AHdq{6W?QJoSUupW2~G5e@9+>2Q*TbD3@jvsTg7(e3^glDldy$y7I zDCV$`-<{6Kn4w!#U7=Q^D!UQU@^R5Za8eAC<Q%H~>V`nqvPP7$Q;R(Al<SCWvSVuy zk>DyopH;5z+<@!&NlIOqMBj(dSH^pc*U7X^^!r*mfFjj1WcL;oR{6g_IeWUF#z40+ zZ9lGIrj6s8RcM8VOGP5h?1X_%<0U-IFJbmWwN4C1v}!p6PhBM9U`ZK!2Xy?(UK)6M zo~_AI;okkCewRNv{q;#v8J^)c51p))w4SC9<6heDA=?YytXGv4S7{=z;D+BVUcB|H zhvU2Ypff7IPuB7)Tj?1}pZMZSl!eO^{6f8br3ts5HMvy`ohhRuCb!BKo0t585j~>L zm{yHgoMlAicn!E?_t$e?=S<99uI;RXNn#UCaFgsls4}44ZKHqpPf#&rk2YC31{+Pa z!RRr*!M9$5*O`1KzrOW)__ltK)kon%ek;35bnWX+I1w&;TRMv1b;P<P-%x&@s#^AS zJBpw9oi&iQrZC4x_4G0h8>kWLii%J00ls1M<RQ3ql)1OH>9|^s2>OLcEzfJF&lG<p z-58FLx#OrVB?~s4AAdZyCrqh-d2Fi%MEL74<{JzdE#0llA{&DB(y^(&5leYaPB@!( zuFr$Ko?ld5WmdLB6H!&rX7c^4$ek;{iHIHqy<p*3UJ^fZ)Hbzf(I;lHu3d;-c7}m7 ztCQEbp-{B)UP80}QjsMLvD*^5s9U_Bw0e*_vS=F`UI!&BdS{jJEUm-Y^Q}ROio0o| z)0${DTUe?yu?w|$&qc%;Mkl%#*sNe0%rGp*vJ<}Xb4hoNbPZ3-(YAe5igy|OqnOef zA$65Gkz_y|w#yfwm0V%*Hfk-ALF!T|V%-%L`mCX3vyZ`a&-FvH_x|MPx(jx>E%;;( zyH@qY77iDGD&2!7H<FE&3@*7C)3K4YpHYGMQAkHd`mFKlWf3F~@bxZ}0VGdoFYaf9 zAZU?#nW?G$Ln0G*g+cM4=4B1qK`xqhxzJwh^v^i6h+sFQE9HGpautk_D%kZ3>&d;Q z)W)92n=^QzvkI)5kTeTC<;*Lq9oT6^B;p?p=+oh~X7Q+T4H^7|elVBa^G0C*gnggy zT?}a<UfD;IZ$E?vEd0pneCMpn>GOUkK|#}(h{`TgU0I&w>L=IKiwAD*l!44Kbxvo_ zOs72$&j9{<jZS~-@PYl<?93&Ane_zJ{)9CO8hfKqL6vUoXzae=brv{hH?zShD1(br z`4MsKtwTj!pGd0`uMIE4icMZ`(+^lXtgELIMy<pRtF0pacMg*Tn%tJk4W>a`tYbAV zxwC@1LRuF0w|dKeCo1Mjl}5EaZb>HtBprGF1<@$i%li8`eq;<T91A~eulPiM=pRd< zYI^Q5yx*B|8=euPUVWe6W}<=U`|)KtlQ+ATArkN`J=~mqL_i~X2I)$$U0QUb4?!+& zG$!?U@njR+VjUBH<3kO$vZ(Ag(UH0Q6F2!n;wBc%2~<InbP|kn|D?tzovFS%$-Lj{ ztBlQZN;FsM^6^!%FdmHRNlftr071qva~T`}D4vO<SK2d0T~6SDPo{<9lCJl)*`;Bp zO|$arvR|5C?|<73+^5xkcfX5gjVp+iWC-8X@4>o!lKQ5xPG7^C*OCJC#n1NI-R_X_ zcd4dWw`+(DxPNT0(}?CysfBxb%1m7GtD5?=c|8u&lZaA2YZXV*cZu4*c_drIGJ`m_ zaK#y)nXB2d4AZr~F4^jLV=8on>nK;!${U(S-+x_#yJatygNuuc!5@E3=09&Sy_)N4 z*_^VRKL$S_C56fNF=X3LvBjtPS83bBMEvvTdXt}@EbO6ezAc5}(GNV8X@a;6cY8O* z;>py^Us==_vpHR>w&>qWEwi%5A+yml(0zS0_EmYd6n`j^r~!YZ(X!OAl!YRZ2!gQY zA&yZ(vW6`MRT*RGq^D%=3xT}J65F@vM@toDY}kD;CB>k8U`mD~LV?(Cw(o2)C{s-z zL-3rn1dq3rj&B=Te{1j_8MW;RgN4PFMZ-~2h}XMd6#br2arKYpbPPF9Fe*u`-eba% zVfIfFurN|KO!;v72R<*M3yUijlErBc`H8MyXSVzH!msy*Ils*d-ZMMeVkA%aWLR1` zvd<FW%*>8PoAvbe7K0T?tX|D+%JZYv^pP3<C%1&tE$dU<_TQu-$3$SsY>4x_tM|gq zmafQ6XZ<Fh`{yBBct3uXIay6)B>23u78kdG1j^PcWoG^I=HR%Et3te--#SD7a{p(` zUs<`p5TrCTWWdPq6#4`o;^fZk1_z`HTSO*teq9EqW_7M%ue!IU<fA{W1lH1KnY|6E z5*rOq4~52~+C5g^)j+oZ<mY?dZZD|sKIk1>6uBA<x_pzg6NhbY0X=)hw9O}K+nhR2 zYW2Qp_)DWSciX0XG&8Xpa$o*wN0k2NOZ6M#A0KYw7`P8`u^0oU2xFM4GdlC}4Dt5M z*6uQ9%XcGA>k)vTNu<N&2w}9@gEte!f-yp)iA!4m!HFvzQNQQIglVVJI$ySo2qLSq z!3RrbC8+yMtYidcqgglMW?-L<q`g`J#}H`{T08Z!l^VXE4q@M>Wj~nH(s<fr?rrO0 z1#>hvMp8qqo<Z7<g<=+K=NF)oR?r2TQE<&s9^`16ket`4F=>lW)s9i|#9~URCBvg3 zFkXgeGbSME_0?<Z-zO!Vt`*du1%MJ>_p35v`$}#Us_yy6N$Y3r_wzC`>KzQ64tMul zPDKc_f{9+mb`jmY$W=o!z<ig;x%b9ApKS6`-j$mdChb)J<hf#f$*6yl%khU7t$f=< z%qhKh8c_S<te*qp_|1I60%$B|n1Qw2qsP_zXzQ5|;d>oI4P@5a>X3}i6+SPYkn&Nd zo+!tIUoZIseGt#t(-Z{-doD~%y<$6}l>TDFF<_zbfZPb}Y?W)`E5HQQ(s@6PT5RJx zID!KGKbG3eUko_B%)nC1_+ii(pD)mHKyq`-{ydCVGbo}vPj;Ak1*OrRmm9cgC)D;N zWS;$=d|e<*dV{;;kIz;hgDlWMlpKxA46pv8q83Bj>xW{3ss<i~nt}jcIN{F^wq1nm zxA15e6+g}@;Pz{ZpPzD87Dw@PeEi*k?!YF;h>RyE5^n%vBWlNz%TPW`J>fo)+~KLH zw?zS)jgdQKLvwJ?4&P&^@|K5wNq!YK@p>Vby?{(OkyKHqyeH1_{Zn<8jtK-@{%;}T zDPte~co=HRFeKV#D)uDv$mRt{pDGA0Q|eyHyht&!iK+#)^FjCOVuBeAvfyN-26A*p zpD~`~iMHxG8GLT92!mX0ZFJI;NEK=Su!#Z3>S^Qhkh2R;7oGW8q=C*NbX6O$!2)W% zlJKayTS|!sXQ)#~=3KO=17*Q_%%5eMu@N=+sxup7+!50hx<7=;&y$#}ovdAo4VZp* zXp%Jb<VvGKxGM9^@j07MI-~1jx-g;Ks6fG8gIwaZeTJ=uv;9EXV5Ml{_1VYXx8z9{ z$hf}hVaB;XZ}XNO;aqq?T`)|#;`Ymzza?gjKP84mR+2+x=o4fgPRYdii^{ppKAl(@ z9>I>Tkb1p_wVB!4%VN#_@KUPm4`vm4C~wNKW5>A9zW|H86S}tg5UeGYDH~~#v8_?k zmV8_4n}EVqY<=U<?i7zoV(|jdgq@jm?tr#ZB_fUDy$(ZgXOvEzidy@bl^!~0|3-y) zN+%a?_ZuYE&$bo*tfgn~aFh8fUVZ35arwCW^Cv+Fi!n6N04seg-bVCW>GzbyI&o0q zHam$YHfkYgql@?dY-J6sSP(LlqQH3fO@9-0moP%}>ksVG-z*t=E^>n5LOFvn`8yn1 ziIu;&wA_+54|`?m5`FKr;ko5MwUVgA9I4b1q*T~7<7~zck`+Rq!_OB*F;~WL(u`Wr z6ag3}{*YJ%>*dXexvLD)gZ)CIejb|n`e4(C>_NV(yCpm7?8wZQ!35?_)m+n^iHTR( zH~kmlu}mAxZ;qKI_GeD!($an8u5Htp&bIkBwhNlM0%D;Mf6S2$HCVU4U1XlE*b-Hi zQMcb9XvlZ1Qs2C2FAq))?<>7@?BCv0`K!=IVgr$TZMKA5$q>8#coQK`Sj{(<4xIHN zJ0wF)*WaF4&6_lYrmC-KqGJ@U{Bgntb%!<kqaJ_?dMydBXS+%CN2ke2|D**6;lgyl z#es%Rb5Aj#o@|p^9*6m*2(17YteH~qLlgPXNQO%Ev&H2oKGWH^_8ej>cgatX)T`;T z%cKsz?1HT<x4P3+444K>EKJZ5ZBBRe35Z=+OM90jY|oXHs*wn|k$cdlpl_)30}D%t z%QXnBhC6&=EVw|;($%2%cU8sx4kfW7fsw}+?X_^?$v6LD<tOxZKgXv#$gDnI#cLk- zQVfAjo4O;s+((xfF_z#J7WMf6XXTq`Gn`AqpeiAp;p;ee`$~x6m+4evR89c?)3fJ4 zl83H{j;hV1Zl5lQQjwkm8!5XjLj9WtZfcc7GSbjJi0(a7sx{^KldZJ8hXST1s^i&O z&`yN}|D(;9Ku6Qu966yrVjtl-xH8lE0a&|0YztJf1riBKiUP;(WIY;e72w7GFvzEF z;QPr7Hu_!E%kddcPV05Qt0Sz@9375`T5Q{hYkFv(D3YfnuVj!l3S8$br(bZKpBtke z=H&>A#WET2GRD*h>-~8k#zB@I@nLyQW3(M^jKevE9~si%8Z6RH-aA2D1!K;gCMbI> zNUUBxYv&)=-L3i6!Gson)TqYTFS6O4{hCpA9?)E#_Et;-yq8t;@@0OxvzplH->@?@ z#O48Q@G=+A7mx=K10i)QcP?4Vauf}sVz$lhX>wmEV+Tq~Yb7be1)g=ot<i8yh9|e9 zko7^u^$Rv<$~@V}#0^b-I;@X;VX<EYB)?E9G1?H;nl@Y)^I3lp_$6SnbB$I0g=Ki0 z(8_%m;RFiWH>UjFZsA!rfF|)5J}}ViV1n}GC(baABN$zS6ehnO*(eUZSsiEv7-znQ zBSCS*ezgu1!`W}q_lPN0pQ&tio4$%%pS(On(FR7zQj;-w#3X2-DlLA?bntpNLop;k z6Mi@@rS&q4erc?XE<vCKhN3x+0J+!i{F9W~sEQidzLT1=N7hYeioCO`y%REPI$b#W zf}8X4(zvSK&g(Cdi7uK5a^N@oVbAHb9oQGW<4s#4U~~L}w?WOoudiTPWehIzT`2bf zQnF#ioN&i|a9=e^b7ocmUy46Q_5tZBE!!f~mdW%P^tdwm!BTbz6i?-c6+fHlJ1x=2 zJJ&~(CSBGlx&^^!qB}M|F9wUVBXv2i{6yz26bY2@t2d`S&TGml-Uv!0c42Px&bU3S zw3%A3{U&wQbwqf1=X1-UZH?*g!|-e9Y^$=Mal0?)`*l)+^qy&}jZx5d?walQ<DLCq zUk+WdQI14H$m3Ye#vb4IMrs#eGicNKts%-*y19-U`X-ghVIT^vBYb&pSO9t_rMYXL zDI47M`8J=y;M-PeY)ZM3ZJ3L>|9i0fllVuS9(298E#x=RP^I71J@4F3#78CEuuH(O zfmXHjn>KL;HjrhU=(m5>FMXgOC<{~);%*=bhCD{Fl0W{IKgR>7*pvj~c~wgG?w5Xr z<pV=#Fa7exe8RZWaj5B?E<!H1pVGlc!uxP~`CLA!AWW>DeLYi=>OI|CFu;OFC_(Wt zcJOaJqVLHH1!>drLll25Xf3!~HMi6G1ulBG0*NW+_5BA67<mE-rv&!9s>k+@q=gS% zRz@G{q70neSQ&;mLo-2u#gXuK<zRk$c;!u75Fv%OS!RwuZ1!&HXQpIkCVxP;n>QMA zmm-uQBy$yS$>o#ZI_*V}omOUz76-p*pi1(`9}yJrs(poC1%*`R%br{}>q9QnfwKL+ zzfx?MAL48+BHXMrqtC5z8iGw|RT+Rw2)O3eNOmE>#x_29`t+UwF-JHi0$U;yltj#> zIhyHGCb{3^DN#8#Fn&ATTKtASe8w4AJeyuhSYt(y(dmJxNDT}>shhu_+8OPKiq2f9 z=M2Z@;mvrbVF<yBA*R4kqkZ$aEtm4E|LQQ^6yHok;Fq_UPkH9xAV6hZ5kL02d5eJF z=GTi3R+y{jSmGT4Vkt>WrF804#!5FqBvY~-Ly?vuM|w4=(3njjB%<TQ7*K$XIF~#p zp5+<l9XV7h>jTm03>>gx7Usjw=nEQA6}a*(ep6lLxs!nn2`7fjoL0ZPFdw-8)>ani zIQdH(R&bA@9U9YsxPG175;gc#?m3Hopo<^f8)~O}_Wsg7zHN1caXBp8lr1`h*EVEB zH*y-erHa=C4{zBnplwGoZc%Wn-Eb<0t`PZNzoI*A8`sfl06`HeT7gLne?}YF7Ekbb zet<le>!W7B4^z+^()64y-kye}@=tOO<!W*kyQo?fGjAz-|4Dd*Aqg)h?F6MDNib%x zJ-`;>v<bnZjuk1fex`6A`g*zo#g~(7;etayCIZ!HV9KmhXTTrd`U}$=^<fwLeqqt* zL$!vHw|xS{V|oo)-2t#+OrM1yCQwqLyF;E52gXNN1^o=&=KadzgfylN*Kx)^TnuJl zC$*iO4t#m!v(iWU5jGl&T5_*cvu&iabPxCrT<sG9uv<PWHp>mA0nLK6e(S_CXHlU1 zL>s+cD){pD%MYxe%3Bus3YKixT)*d^iU`k;eBo6UovQjgiDssO!?f1INhF>4@@A<8 zVLsgP>_fjDqiV=UpN)_{2epiXMuZ1M^R*)v<MP3;FERQ<rU?x3)NdUhT1^s^^^o3p z?MIoqc(z_;wE9>593UJwCv!*x`g315NK`rbQh*y&;o_q+^yiDSS2GqxIZQlpF@RBn z1ROr!Aza1xM1<V{5)9zmtM=DmlEttVNd%72%I=HGPvk@EH@|Zq+x?*S{n|eir&fJ{ zS@S&INf_vGjHr8AwAe(~-o&7ejhIn^%f2hQDYa5G8tjcvm7SN)^?)j?eIH_^@`5St zEQh?E1?~3pb`;X&LQHyz%i(OF+uv6(Ck(D&u2WYN7j~T-0yBIsLDt8Giuo}HM>_Pv zJNm#|7|JRzj0$DEEc<e#eRDj`qP|62+p7%-x~{;Tv6oj<a&S~x$zC@Gj}!aS5UCXV zxt2$8sFR=33o2a1H@=xTtx`~lvjcxaj{*k}_u;`0#!`^dLVD6dzuzKVe_RYWi<L(T z7LRjNghHXCX@aV+WR?2JXMd@EuAW~a$V`PV@{0_LV;m6U(s-k3(~LjyRoE1FrT~-A z{0YG_9EEXj^;PNj^C*+4Cf}Z~>O3M9{(xUlpt?W_8Kebn&5<YPPqUT-p*U<sJ-;8c zy?`|OBf%hsz$1JkP2n0C!MqT-f%hQK=>LUBAg>$}&&*APl$Vhp<z=LB#gM<!^#1@t z03{;e@ULcq;1&;p{~7N8<>mhR`Mn{7@126R`<Vn|3L=kAP15=O+eh^Wu_57vgr%HY z3Sy*)T{w{Iq+_ML2Vl;>KM&YBUqS?fx9RTqO(Yn|kWWO;l#cm(^8WkJ18_|yd3};J z1GgLe0&+uW{EeFb;?4eCB=}Hx$hX4_#8W{S2tc{8B0uH&e?cHbWpHiCQ%TXDNdHN^ z{QWuq@yGWm7#3f1ZX+gNf$xw@{w^Qw@c%kx!MAC^Kz^zi$lb)47S}yA+z~GS-XJg< zNeaTEkih6HmA3a4SQ7ZLE^l&iA5AKK*)XoZoA<AuMxX}rB{jnIs1S_DM<A7As3K{l z(i4tSBJuNH-JTRWrIaB`|NWYLf&5t3Xx(JeapVO5<5JFx>!;(auM@H%z5X72P5WA$ zj~P)d7rkG~MSpRXO)f|D@mHk-(0rDx^lH}R>Pv+UNts4uUg0>xn#}hkC=d5ndJCXl zqXSxG@ZKv*UbOkMTfJCnb_;T{QWstI@(3PD<&@H}+swHGl~rJq%mizOrjgnZf&BL| zrK0I<Y)a|$4y9;s-fg4;ugB{kaKiNd+|@^yrxSEI#m%X0H@B34v0y*H=g^xd=YK4m z<(_c|CCSEE<SveVyy0w6MJ;HzSA*~7#Rz^bo$;R!Ot1xHL}!g<Zs-uZEFrA`>P+@s zHQ$;g2HXJVhIYQQzx3(|4eek~=&ToItf1kKpYs^_Ig5gm0*@HOPQM*mBuor&#n6Bj zeWJm<7Srg()jp+rST}~3=)J-#8p2;Ec|XKOSRjyCg61WK{Y>XA>nX#9V%3*3OjgCr zqXaWykzUu0BZC0OP2;&p;~OcEsCI(sG55v9x1?*yJ>$K{KRI%R7~<aIAoba=*fe=w zJiYj+K3;iD(w-cIRyU%@{fI4lupIgB^^8JOFAGh_X*i!AP{@jwtE2_AIM;P|mJ3-n zT~4|7I32#QGyd0O8{7sN0<UddF-F8Lc!-hhc7K$x=F?xEP6(s2+~T2yEd!hy+IT?w zp+a3ieIb6G!BAq$_g}6~l^Uc>UHyN00jzs%Uoz}z=R7m&OO1dZ)!Lkx2^U=6-5=%8 zDU#Jxx?YIb1(0ZtB^@4JShdSJsD4j@60!7k!|7IEt4__kXd@E-AHJ>+LtkzD=l}6} zbt`Tb0@Nm!#un>le+xBvp)ITq{v+^;0Uxh4`t?@Ce(rZ9jywKxy-9!anQm~-I8AX4 z#Tq2Tdd&%Vi_pM23klGtS+rcW6wit?B!(&vbUM(uj=6Q5pfGHCLwat0!NJaT2|x@s zs-Kp;#4yH;IWk;!TAMH@?>7%ZYCM)%_L!Z}HeL1_1rD;$CI7WpkibLMQuy}xeDIKY z1GCxXHFZjD-o#n?H?atwt30HZ1#{>gLj5`bP(wg!<!Vv!qIXve3AB1_w$Bfj>(aSV zk_F`PJi0ein>k260$Pm%;BXqvAx1;#VOX?Rl6n*4er{%eH{?Dv@1ITmy_Z=9Yc3Ch z{@e<DY1%bm^j#&JcEY7C*_(?yu5$k;i22g|895G7d~)FAft>19Vx@RdVEHsEy#TvP z6JPB0+HVer?U~hBc4tpldIQ`e-@$#r)9peio+BUp>aE~d9_<V&>#<di$6VuqQ`#)M z3^-=fdL}}=g=9G*ULk`7%V`lB=dEUPy|G+AqMi8<d9`u)+s$r9S1s^Fu8O-Hdku8i zXWqM9o5{BocJuDqm<cwAyyZPNJJ?CI;t$lNEZ>oS1K(;G%U7-~E@rw&v(!&%-$RE| zq&F?udv(m?XDrNr99#`sO@j8b6FrpnNFocMD|mA%>nbd?YC)8TtU+_VGKC$vC8o)x zeB{471FRE+b-lZdD*+SGo*@tqZYx(sP?=7v2qoS0)F)KS)iF@pV)UxFURK04q^-U9 zf^=Du#k<?Lk~ZZ$KsFFx@9pwx>0|_z&g>dR4VCt@HgIZA<w`Xmf5acmj~eIJB{eW^ z+{qUVI|LnQ3(fkVUa3ByCKNkaYlt7qz7$*u&V)3R|F}7W$g=mNENbT}Y`fHT<>Ra{ zHg$ENzxf~;-;+}e3bKM`XRG~__RVRT3LCd#Ep>nTU-2^dU;70cU^;-|QXf7{kOCvi z*mrB`SEOvsibha7_OKFTFGrb99*8>F=HZ=Xl+K3oGF>~ZA^`9wCmBOsAe`a(5yu5k zexd^o`Cmh)3};1C{k2ScNwFFBOXoTuU3l26mA&>@J`muzuT)#SDtQ$}v{10jBs;4Y zZc?CWJvi-JoKy4oBY8I#`U&06v}1{;5OzB`p0&eX(a%5O!~fb2y09U8GBq|s=QCJn zSRYQ|#QBrcIHqd7psTsy&QlbBG{k_T&@MX@(XQF5JY@Rly;@XI15=z-yXBv@iWysi zPyl$2?nHA{k#)G$&0Z|gGX|-4JW8&z#@xVcSt@*yZ1=TzwyxP_U^5Yk)lld^td=e1 z_pqqN)M3vGvddL->DuniF}-#;ZlY=&H_0@4ya;?`pikiGC=oCGcB*15ZlL)eIWdjp zrZZpf#PAFXoF%iL73{f~j#c8tl_@U1#W_lpBuVwQp!a=6qbCaKUzm3RH^{2aM(Vo@ zDIH&%Y8rIo%4L*j$<Z0hkHH?|@zhT^2cer<a;S^pzSf(r`i#tBN)Dsy2hKGe!wm*j z|K0CcJ!T1=0`2C&d*0NRv$`l^Ho?C&A@1^DX1z8@oZ&ePutSVsup!}q-@OW%{~<Gk z4oVMVY@W%DP}`?94Y5>zE3T!9<JD5cZh}Z<Ba3vIpeTYGcA-UWiJ)T<N4g`^elaEk zd?}N@zdP|f1$J2t+2qHq+wMdF9GUO?`dJ=hXF(H~K}?_<WCfqnO6`)!nkxd{TiK_P zX{F0#wnW*sR_bq4K!b4;QB}j)KGHK)I_K_Nt5lNVOUp%p5anDcH(0T15535H(xF5e zf2#S4YYWh&=5C1+R(fZ;VE|^I<#@q}3ds;2;k~nX4Kr|sCu4LoXG_cN%*X4x)<W0r za6Qv_ro}0^jp}MGp>ENr{7glCt-S-SpYTZ_OE5Rmm5QFj2{arX>~HUBgU4NI<@2d; z&jbI<S^5dmg7L&*-Du8NfSwdMORMuH#b`N3CN=Ay3Y~*GzNk@Lq!Q?T<L3W(c1Sz{ znp@VWU}?{v@~h3+jh}s5^NYDUTFHu>)*bJ-8lz>M43@LrE&OZ{Z-s=18iSMtq929h ze&pdn?Z_miV&yh273+l&ZUpDv`A)au_t2JwQl0gs?hRSKZ%ZHz@=Iowx2w|^Sw*F> z_WIN7GsTN}idl%!-6@AW75u46GMV^~N{)uM)StCN?6y^>_urwmTyB|T1f1TM>$Q}S zy;GeyN&tA7W<q6ifuwp5uH4v~p9%%r!Vm5N=*NRsC|H(&`HwZy7Ak(+dSU@3w2hW) zzUQzzJh>JL14mcKcHWJ{zC=RtUvBk6$YWKQuLMty2zc$2P^OLf-`gv^+yJOAB?|0| zu)ZRY5*<+fF}bEI_(_oesR%Z;saA`2-Ivyp7%SWX))QX|ui)P%`4GEeAIc=}m(dK+ zUn62~LE9olaASy;S>JdzZ~bn8q1lu}xb-1r)Oqq;kd)DXUS+cgDcS=T{|Bs2`G<jQ zG^^xj*;WkW1h1CNo<*6B>HJ4i*VeA$$l>fZkCmdmYDfH6AFvRuit`zhmH5Qf*^#Q} zIwTQNes8?mRNEgp;tPL|h4|`;kJrxLJT^iq)YlnsP~4#JnC^)Cv44n?r-S~VPOU$d zlNNAr_7(oyv<bNU>fbz9EOx-Oi2-pO!VEJyw~Bn|VsA_3l?5efG7+1*hkfCu<hxPa znWHaGSbmG&>HOD0)9l?DAang(Kz&|JE|fg$(dfoAjK9(wFvl`_#*A$Ms(A14*vjNy zadhSA9Vk3rX|7+Xq|~V5nlm&uBDT1a4+(UIBe`f<1hOAZs5N@e{F+0vXk*`7>U|2k zb^`4<5prNV;4BWE6)^`|hCg>f1(QRhF-p~ew&TXF&)*229>GIWE|GIn98dKibU~w$ z0#C85|DSr<-y!Y#Ysgl&+L#axwmKhyF>GUu0Diai#EDD-Av4SDwHG1dH6*8wSedWk z?<<Sj^s+rCag}D4u6cDA>L#QY2nNdZoj@-aHURLwQZ0MRe#keL7kO>avGW};*ux92 zKz$;iXFA0|&r?q6#&IH916$SOgIVVM;PvSWw*KYpFd^GlXN1FKO+C>I|GLxye{XmQ z?l6uWG@^&ic=L6#R?aX=sTCW^u$-UD##eePW1s`Yf>dFhf>Gy_^WFP}s}rijI%H=h zxr+1`i!FIhz-O}&Ufx-ZO(l0d9)JUF?_TFps)a05-S*!$|2s$!duLBi4mKYopG>kL z(DFd|w%$6sm(_?;-IMfip(Z(Amax?S+Ct&pDJ!055rEvn$6tqkuUPK)hx_hW*}dO9 z1$1*^{=}W{5q<&fJ<nvN@Gb4fEyaOUa&(hsk4>7S|8)_1(|2$+SSEKZ&L2l%lEDhu zKP_|W|58v<R!J-p&jf?3gy4_|X$e0pjGzR1f;wP3Tdih~uuCmd17q~;T-L!b;p_!- zcEC+?Dk5vmlNa(JjH+EZ5-FcGTd$SE_2uxx{nR3XV?*YadU{Cl?&Erwt0i!S<yEm0 zYIgEE@A3Gey_^V-xGlN3Uku2o?S46<5*!Mha=eRIV&TqZ)6qrC&!X<KhmN4}ywMaU z4}i)A*>-aOj#c|0v1%W+dLT6->~n})t&)?UDZRp+hf+RTDuxFmI_QmB-CMZAA7`%> z0Eosu4p&7gy!CNPNA&YKg>S2xXs9-uZ6Q@82IU#VV{y`%G0A~&+2C01z|l*;uwt?9 zjyHj7)W+Kol^^Iho)D*s$?>T4$hTZ0)^_~HJW%vu7lp%%Ogb@33YHzp^E-^I70~@B z$<9C~IZlDwN1wX^y8ZlYC2A2nnhJ>E2V%O`coq#bT7z^&kt>Y?nMWvjkgvpxruU^d zm{ZG+mv5=7dTCeXptD-%I*`toG<WAopF`JttJ-D|NqDU@W*_)A<}EA|SFR<@;Yui9 zb0?0b2{^>3%dEa13A^B~@BL;S%qwhIqg#$pv%w)|X$~HEHk&5h(|<>kzQ~ZI-Ga}E z%~v3k977p2Xr}KCs>=dBQ8RU8wzLQj+N>WWK~z8@5&_%j4g2KWBrGB8Z<k+)kiSra zbH|6Ea<42yF5m7^;7f6BPEDMu(n3550F*zO2rOHSl14R3B-dVEOrQqnn4!!z20vp& z4{a^h??18-*lqtzp?;x*MrFZ4xmDHRm0m$~z-C!QG2)H>r2w?*CK|#E)6BmUOjX*t zZ`CYIFJu-gddYeqZ9^b2sch3_0TxbCBkf^ocROCH$fF>AEiv0l5!8$qspCM?XTQn- zY;MPsf@(HrLp_|$i`Z##jG$?p^w$ZcA2)5M<mozxpZKaKVhSFLe!!x0Tf9rx$rmK! z0RGnMe=3OoXeQ|p5$J8&C;U1R3~9tJjgBFt*COBjp}c^jW(l{^ZZ~`2QNGcccTK(L zFaVuJU(#Mp<5w=|Z{7B{Pj6uW6_a_Ha_Sx=jeAP?RfYvWX_Xr3a1UKbw7vBq!fI{p zhnY_|KmD%0KAuy?QmU=kY;P!ueOz-myCHU1D_}hjwo`Ebc@l05Z6~kJh5G(JTLY|0 zL_u<)UnEN94kcRuEs-oiw8#c#`3Y%35_Jqj?hg9T)W!h5t1<f@w!Shfu59T#!QEXO z4-f(bcXxM5a1Rzja1R7`3ogMOg1c*QcXxM}?{x0H@142d{2QJ*Io)UXKDDb>ty*i3 zhgE<3Js=9{1ag|Hu#Hhe^JbAA0iVVHn!mgCTg9sHZv~){?`&mK7Ff*I+9Bel>i>f# zf!={Jn2%|EdyOEpn?I1!&SA|L*rS!7)FvY;iXVo$^e2?2Rn2`70C84~NmgUQ*eU!_ zY<T&o<f(rn4f!|!8sLvAk}U<)x&8uw!6AU5r(la%AwVb%JltjITnJhYe+3291L8p# zG(l_A6f29sjCd@)DgQt$pT;IV;yM9TUZ31?e^vne@f6@Ug;5{836?ARFSg~A0q($L zBue9h1+Ky0Szb2gtMXs2`1e=vxeWuSVs6hSmoG*q+^dd-&$ki9v2k(u!qKsTmlEVr z&A*r3PMU9)jg3uh-^=vNCW{V8<40jdsXgzw-RkVrAA9sX@H38F+!K?YG>_Epq@D3? zEMGLYlEE$70*}_X5-9T@+S%WMyw`v5v&epAFiQ4E-Vw_AOVQtc+kXR;ZBO8*dGM`_ zB?mZ}wLlD%760~67Yg=)@7d=RrK*S)s180}N*YuM{+_M>DSiYmMS(l+&XtV=b8V{N zpsMO7zW;aKtbbQY`e3{f`g;lhz_odRx7x$2Usz=S;)ecTA@yg-4vh<3O%Zv1sRh0p z6nHejXN{`h@3H9LKRwAS;MC`PHJT$Sv`Qup+%%x>B4_XK>+|O?KSbcd?=lKfe9%6- zKEN%}4|@fM({g|Rc&D6%lt{Zx5h2R-qp?=!{sMwn;F6E;L0LB#n`d|lk<8Qac#`q! zRXV*;Cs|iAVdnl)lZ&tXi+e1ILaZMkJ)C(_XZJ)v{J3pDaCvxf>D<wUKA>UK@B+8k za7Fs$c~ANDykOz_{zxU3(nb19Qlk<VW2tVvEKLMyrq@fel-8HcpLooMi(a7Yt~U~^ zwR&E;3Cu>y=aYLso8IO@JU`s%u??fBbaW{w0#p2{(SGiWdR~w6xQr=z#tONa5~Oa6 zW%q{RFRqs~(wCy|r4uL%^lomYH><>N8)z2GHlE;4q@>bi<<3ZD(2j@fN?yLNjEt{% z>10$T{>rcB%(djM2QPoS2(Wsq2DDt@VNCFby9kRi3clM20OXwL3uZQ<W2ucyds%%% z!w<%SQ9PRi3R!Jr#OPH~H>Y8f9A#&=znY)?V0GQY34U?D^q>t#>hntB6InWl%~^yF zQO-IR`KnHcJCdtbNux6C4qf1~q7>E7-*H32RURS!)m-w7BuoMD;4VLxN~GrFOMQZt zPT$gMHmH5ayGir%@j1EY#!9Z&bG9t%&4)(XxMKD6lx-W;p%BJe4@nmP%W_x4bJ#{- z;n@%P15%QpR~^+FzS62q($<BFtvRI=!*Z6py(cZXX7*b-8o}|G+9N0YD^hMwUeZA^ zRC(iU1bbCKCUZT7N7%ovpQ5VftBM^wOKb?p@1+e(Yo&Qf#fZAEO_zLN{bghXu{Y^7 zzg~VH*=apnZjMuu#`0bJl2Bx1=ZRFpjqmu*8FGkVEI=FZuj4vz+Gi_i#Z8@t^EyGE z`Ko8NBzl5;AU37_HFri@17*haw)p1t)iS$obL*7FO7b}_{SiwuCAcf}=k42MjwJ$i zd7Bsbw;-jDh;64HJXMbP@t_XbGiL8{?iPzZAn-F74{nPcR-1zckWx!K0@MUx=~RlC z;|iCVM&J5Idyd3kv057I%`~02=%(yD?h;zVCuUmjD}NNjsUhT^!-l-*n68$w7~4co zl6B8AbQm}twAVHqwKx{<Zt~ugX_OYL5;d#m9>Qvm3VAPZ^HZz4TtL8M9d8J4-dC4S zanolgMDZ26WoN`g%TKZk{&LDJCJkmYq0bSTRY8V;V88a1r`$5NK74<nPo?_)*Vc3{ zDZT@>&&{xRHTb+yz}Tcg4ws|30J7zWbUJo(F<bRglWsSj*K1WSbG&qVwcl4L<5WD` z`)Cw9kj9gn2~<_bg|!NXGT`N>f)!IH2+%V_J9cvezB7sZkUgA=NAY@L0L1jzwPhy0 zkDbO8ira$&$u{xZAxtP?B;ZUSpCtoUQ4u-64N9b$<(pjasI`(1l4w3TnR^VTf&V`L z?z3qJlJeW`qUkX>VGoNFvjy{q`#3*jcJ}_+w-UXhmnBWHAUh0#iqA26L&+SDAfbW0 zr*hg@HZz+Dw?hwn?2GjqmOQLArL^}u&QaXq3EN7ZU+?VJ^GMA<AI!%pNGVPo%yKla z(q}l9Z<?+BQrk;4B`$H>3CEXgWG@?QCH#sn$cX#SMu$b=*0U?o=sSERl2)aqA<_)9 zO$1#~KQW<(MNpOM9*c5%L5N*Hf^19JhYB^!!D;9pF@PEE_;`U)zGGwB<$kh>j`CeN zia5UyhAAM1pA*MI^GQly5(<`Ob|z4Nn>g5oPu6d)wy|cs5t18|-?-^YzFF}Qr62}` zRsufev>ozT#nIqgpgI6?x8K`A6al+qw-HLgfO5p-S=Qx2MS2x-I*l@~#cPkFeE~mY z$c~8tek@AKl#Z~2nY}^AUN}cX{SOGA;@~I8XfqDey|^WJ;`mTz-*)Db13zXq{Qd)= zbg5MM75`1nFE3+%fsynqfjEXxnyG5%!mx&BX@?$$t{}n)Wi#jbjbEiZYrb^p_R<K{ zd3x6x@AMu+hEt#8qI0A-3}g@;xt^6Bg!$wa7$rm8l=<R<)_dGyWes1&`7{vhpQ%op zHOHeadU4%T<f!PKA8CTpRR*?at}Hd+lX>znh+o#+zAv@j|MnV>Er{&0tow3@k@|Gy z7yl|a9Tc_S3i(wR9fh$#E|NCU?yQ6KtEdOQd<ri#Uz}C}s9`os<RwF1<GDq$V8U$N z!IM(`Z7(-yq;th9>$8ktVr&(RRsF2md$Rt|+dnp>P3WMhT8v0{JL*@e?lF5bkW3nP zZzEu<o>FNhOh>Pkh!9nx2xQne^cCohYIYom4a^WPI+Bq|s0UNBB~Xr@q_3=5!br0} zataRY9ro+(Ru#I*n()w9C!y}5PK3v|&e}J}<jR%+f<F`0Ue99*8N}tJexG&c5E?Ic zNE@DVtpz4G%*0Cd`92c}Ecf7U@K#y#F_cU=_EGAnH{_+0<?(+HYhjVrHk%8z`|A8s z3rK-7h5g2)Ri<@M2R$r`nQk|g7tu)=r={^aYL5JvlOBL8G>Me%(v(Z*N-k~W(kq`P zQn^NpDSQsVB7%(#VcRWMzHyUY+A$^!sc6UZlU`nfblYo9=Ag~emT~bAebz)Z>J2N} zL6Vg4@~w41v%b{|yW`4w0HhDg6z8<%E<aWmH5ut}=1%)szI?pS|Dgrzbqe=Y-Y7M& zsE6<nrAOJG)UkA3AWcyqV(^#U@kxVWD{8CItasfWlVPD}iyc7>lkdRSJf(@d=w7?w ze9x+fEC=pkfasY-Hi<ms&DPwx(ESw)&bU&6<6WXik*VBx;f5la6QXlXs^Z3aNJ%oh zF>atQt}bnMdRc!gF+GdEoIqjP#91u2d5_fydYT#0<4|w9nRABU+^TwY^}v2zT>AbF z;l7`X{ml|Zvu>7rp$pU=Uz(nz{pG6D*R`9K$p;~7>5{aF?aD5TFs5E}(KNfzA5Z4n zA!fHxz7Ln2-3dX&J2f7vp*l@3t9QN*s3TwUR!<(Vj<>FOrOs)pW7jpLJ$T>KCAMpa zxQE5idlfl-kD<AA!(DD2^kx^r7H}!7y436t-KL7LFP19#jXQo4#WX=C1mvqfae(;u z6GyLK2l#^*9geUAi7;|_O2#~yoY}AkG;1#8E%(M9z6Zv&-OpgYG+|D3_LC1X@&q2e zBym&UT1X@Qz>K93)v@IKXWqlMwlsM5A)@nErC?S6s)tTg_jx8?JEwd*q4(WLj4j%l z9zG`~b5Fl)qfaqrT7n;NZLDG&6%s5>;Ng8tCVE^aF&X<9?!#&Z$}tFOiV5I&IC3nP zYaD!k-)aY!<({yfh<^XAT5*gwmKjA(d85t#I{`1hC|jsJkpt4|M9{S#>9ZA+gFbF` z1*fH$k^OYx3#SRo!SyyN?mcU<Y-_;}%~|Mnn0cf)f|^{zxKBw+Hz{6~7nnLIl-Z{_ z$-fD&pSi8u2nf3Xp@RLI(;-Ev!trdrr?Oj}0=Oot>3sxHQK0@{1|$g2RMO;&FMk zx}%pbxMl?$X>o+rQ$~GN2L7yy8~Qw&bq4*7S-TrkAqC<^rL0~Ziu9-{o62;b!WvM{ zXZ?IW6V5c}OtEKo)_HICe-S&r{YAVH#?+ZxEfZE-CG?Pqz!K>?>prCJ@%n?CX`5#0 zN_}hf-qm+L$r-z)4=#fO`953p=kaOuYHFldv@hZB4DVNME~l;;Se=JIiM?8;k{(N6 z`V>bUyHqvvxrqBYw2aMuCFcs!O#9}vzx?ii;b-{9XXCxsEGRo{3zr!XR0CWXM7)fL zHBaQ+3R(lb#gIJ*v)CUTutsnr)9IHROQcPu-&lXB74D9>(IjO$(Y_?Mw!V)-j}RJk zEB-*Fh$D$poch}=bjRGo51HRq6anL|5(ULlW91iWFSLHzRdR*wOIJdQfafWnC2bGN zG}nZSKL;;F;sgk(94E<$q)yFZG29g8<jCL$THK|R&e}B>aVC3^*@WNJ0$-m8klvub zHg1S{l0{=oU8XG+U<HK%h8^sWsYP+48%B?X*3%!wpURhwmKbl<p$o?vl2zgyh{`{y ztW~>}CU2acgaBF*{6{ByX2tbv+D<d<m{0w^D`w6hz=yY_lugEBPf0TB3)2eUdQXIc zp7lc@<KA=ch50w_-<c4<Ut{DmJVaD{t`b;@3I|@ybi0r7PkndlwsUbfvP};}?H@lC zO^*DKBg~os3???}FY@t8pj<&GD)I+hX(BGV^tm(SdLv||0Z(|L)qUx95E!jq6Hb@2 zCp^|c&*ni589L4T8k3}IFmI|&pY`?2XkJWt9DH|*$4HAfb;H4xn>&!JAoAx%X8}b| z9R-@BAAZueAwr~$jiM_S#?3KC8W5YlByn<{Fctkij@Nzr`{oL7Rv0TWGX2WQW_f%$ zDDOd+`+D4}O*A-k=U));(N8(<+()!qL8V@mpYDm+L*COzd~KyG$d8C0sbx=-PsZ`+ zOcf|z_bI!lS}v8DMas|x70UV!U*qn3#i_!E_Rn|HSmI3hl<`mKosd08v~42lx(nS> z$;U^xELVuN&O)ZEL-x`J1<qa<U)s9!S!>h>*8J|mKs?=_{WWL}*&k`@({2p4+?Ozp zEQ_?tJrDV~_JnRNx|^-rMRw%>Dnd(SesYRa<bIXb>JiJw-&16%F(SPvCp$7;@7^xF zMdoR8f`BD57Or_kC%bk1fE2|mp{kuq@^>2^qpq{wyT@F4K6P}tNStq}?;_vbCLv29 zJm3lP;K&n{wi2~w_7G)%isWFIGq?P~>=cBH=9W*&jtDKIKAM~}5<5h;Z1n)3Z{PA6 zCE+B`t#m3yBWjbZ8|c5~HkAsnr->{~P7qpFt$ly|uD0|tT6fViG$Y<kXaG}ChoZJJ zYH|jX)K4Gj#8@<hHh#ERr#-@q*9v<Vu!{VU{OKdAO0-Ts#pe-~UF0o;Lxp?MYIG=C zVZfcss~0!0$6gx3Ryp0Vfj87P@^sFA>jFf1C3XcXKB7RxpWRE_=c{e)_Nl-5azK&0 z33h(x-SLKt0fuQ*VxGX!i+Gs|A`Qxf<uW0*<(s`Z3EHCfli~WqKW3!Wet!LglslB5 z6=Se+c$<)mvR}A^H!UZrM5KP_QDx|(D+7dIGbyPvYHSyBYxZi-jO7GaO{cT9S02bx zJ)|v99puq8We<ZzzHnaCLi&i&u>0;HBEeu0?uZ%G(&RKf+qjwNGQC0<IAKhC^qO5w z<$gkv=V8E2b^^;LMPOUEc(!6MM1fr^yN)Z*cDYXexs<+bhL(xTmpzG8G%_B((lzzo z0%Do+Cnw|O9-zn-B%lxYGWp=m*4S`)<fmDJN|{=QURli6#OdCY2bk*a*h^}8l%>h7 z9N&w^B$d&}jUU!ufZ^n7tT$<-Mvc{@#`x2bi~9m;0t+0)<H|aCwx3-$=+gz$)_l-R zn`{zCOsc{jpmsp1OwVWSNXky2V`fI>{9E8S$T!CZxDdKOh8za}a|q{0sI0mxmA$V{ z&X3?O*u%UcgBf-~3s4k7KaiT7hbAY!c=-NUrc}A?1pmcrC9j-hcIHCn6JIY&$c#Pk z*5|Q<!gqLvDSqKVzn7mKgWQ%0SUV}iFnWVg73Mk9A@P0Z0$Y%+38qG+m(;GAIz9ya zzW*_Szyh48!5~MVpeBZ?<1Z+}tz(CQy&rQn8$0G>uB)$TOI`<`cE~VLGN1km9AeZB z7p!SkT`h&er4(2u<(TyiOyX+7^(bYR0FJsb#?DX%8I!eI#ttjvyq=_lJprToYTE7| zgPZMYEE#-je?Nq86NuF<`j4;Cp@YaH^_e6G=}051gb0TkRuRvt+Lz#yO=6%}e^ts$ zI9cQ(%d#*-ki%ce|D0rtWbEDePADpP*D%C(7l4$2SH5tpS(v>fC@kE7U`zpV3O($& zzz&kZQNZ8x;cLCD#EKXed`e`lqVc%fGiCU$c8bD0Ff&MD>sj8;+zU1C{@gFlKn%en z<-l_}mQ2*;n?f+&5FS6g%WcE{lht!4wo0O9M#lULl-=y8LxGlZjHS^Jub;hyk_jzg zMV|&;WnEXI9owm0Rd;e-W6W(z6M5~TnS8Uc*}I8W!?5Yv>apu5^B44qhhTP(YSNfm zU~BR-54J6{*2NQi-pm%Oc9dJJBXdk%@ar}Qk_6oao|kL%6SytfNH%LGEJYpLcX$Sn z&jKqf8RIi_AF4RpJ9)(1dY()uN4@tL#e`_=o;fOLop3hmi|&wbWcN9#k(W`n<L$2t zEJ_N>A$?o~`#vP6o!>gjab@=PASa=>cCUmWq(oyV*W~D$<_#V{uR07;ira<Fps2j} zJz<V;Z!L2e9rDLtu<cBmv>$hu;X^oBjP*=qO$_Jqt@m9;MlB9{9<<0WF?cuE%e*ZR zdTxZ0Y|kC((B0L;yA#10X;_*yP0a$dv*B^9wwXzspM<yv7nYIfXd&()koD@6iUQ9b zyB%r_{O#RE1=|83;?rPgm0E0APz%^oC7;%qFOU64xCX9vP2EDJ;3HY&+>k4``TJgk z7(VTr4Q0#~F%;<3x<k15!wM((@O~4GUs}kNN=O_RdE|vc%Zhgwji8duXZE3)82dET z8xVS*3cJgh$F_cq*UV5*h*>DmG%PqiyTB4Yu@I3`CbjrcrAs|I_`8Gh1z`0Je`eD) zUg*=LP+zj6nA6*c+Y?{56WRRy$n(om3A6!aa$X5Tnw#+xfdR_idegO@*u{Mz=X_~g z=8udQ5xA-kPhA8nyZL}m&BS`<oo!+3FVZjaNnI{7Qg(?3{gY$&mbJ8d#<~HNxVroL zUQie`+v4B#cxj544d$L#-R_d4l+V~M`jUwx5iJS|-wr$eO3}?9azDjbXa%%4*PuEg zryymqi5hELZL!}e(l{uyv0eB}u;klSFke((<N_1+CrX_*zUDoVB=dromG$R^b@)|| z%dB#Od^z(zZC!$8w1`z69K$ycj7|q~sRccq2?qAB2?dFs)%T<L_T#h6;*3e%s5h!Q zajhmDe{wR!@a9f}^jIQf6NJ`LjyND=TuvpEw50HoUT5P<b_7{EEmOq7j>8d#7D>}V zh`w&gKy|jQS2yvQ2kF-Re$yzEQ^>(oIXUh*0poBn`_T;$Ij5kXdC{#sUkxjzo$DpJ zXb4rK>|=3~LR*CDhn4(&H=oZGUl)M<@N(7bBKF2+)Q@nTIahFW9+G`$)=%g@`@_-u zq|j~T9~_6cw1!yYut!rG?(yO4P4#+!^XOT^nVEa$;Db~Gv{Bl<NJ5y_bw$9Ckff|* zq52~U8%q`Gr|6Oa?8!}dLf0L+!ukSnZM1n%&CwH#1rr*f%>n(f*r^loSBgPRhH1(K z+a1}Y!(Mk|c&e@o&GL=*o71s{l;s@$PlLoPf*wru8qg^_Wl{-ex5!_0)*4pfMiR^s z`QNV^`QqbslG)FUKQlUhs<zITGz=5-7CpZu8Avw4-pbGNq~~i{f@9&8Kxc|DXbqBL zkp$!x0{P0V8K0dETWy{lDH@!MXRRyObUkZW)s*EK>4cHn8Av_GZa0qbNq|vS2lHh! z$}RVtNHqvd(`zSGPRYHGPbF^KI3rvaF58#xJIn-QW+`ey3-CK3R|!-Dk$xLTBkbxo zX}>%po}ID!9%5^+Qp)aJQRWJ5pOCJ#H?31L6wzwi)_bZg8&JxZf*sJ=O`hPHFuyrT z>YY$OZ&F!+@jo5_xOVB4C<V0d1V5w0@pKHTq8q>AclP4;U|N}b^*+ys4|SGBgflaA z&sNzFLXvp-&1Y`v!P}6x*@(0LD0%I6_mRii3?sM({*1SZA_iAbiy?CA&m6za-K=iI zRxgD;9tMuW7_9m-mUViE&&#=>s^4s$FeF4v$?th8!EC@*7n?-!K_+R;)T9V5woxP7 zwt=Y5528pds^(*RLtQ?+4}@n?iH0#L&xZ2K++v)Q-%7T7+up`_^O!tws^QU?ji|Rn z5yPJAN2o_wI4PWYzr}Ew_DhjR&6^b-uDdKw72&iNmU?jZoq7z3NOdza3vnn@r+9FE z{Wfx~)T>jN*U{QdnPU8k3>Ipo-tj1g*ow<-;o6UxRMwp)!pK<Ho9@=Cz#!Af7R3PU zSi({F4=!ojH-ZU<%d<x>Wau|>9EWrAQ>V~`$99r!P-3Ehs$?(jzMBAirT(A_Ac}kp zS%vD8qnt@{Q?q7wc0JXYY)&n_*pq>6`@u`F%E<GUeDF~R7T7t|XQ|t`lu`|oZ}u|P zk83<LmY#?Auvyd=;)f}Ce7^C{wnTB|L27A42B_#8qS3j|Iw!Y-X~qFbl$6~GhsKRD zP1FYNJOKAf{AGkz0xdGXy}`WD)Bdh$xDr`1+=;sZU1xclUlwjP>zjql1OsX8-UtiP z9t6NJ8nxv38hkJLc?R*_pRHHxTqCg0XkeON`OLpD%AdE&e!aAQ))b(EPCW?Wl70>t z3(S;_W|sniRD}@Dtc0s(ClmXmS9|k>bB)`*)(jR<Hqm{tVsR#46Ul5{EB}J<9>BK0 zD92rMpi0pIq4Zzma84;Ypc@8<0x|ZD$rt%;euU6cpxPHa689sy$Tt-=YB08n<%!TI z&Tx?_{Ye*8N8K_D<;r(V20e~V84$*<@SE<L(U4QJdVcuh8}!c#(sGc>uOC`xP4Hpa zImDLXDqW0VKhbjKg(0<OXA<Q@B&_L~gd7&0^Crm8X`h?M5`L09qQh1aze&%5(Sg|= zqA9{&4WVr!7al^>%{Zj<Zi+e=OAunKFqvC$6g<2VO*QbMU&HK&yZ(W;sC)dhB<oCJ ztr><Ctq+eOkokK<ARl%;qZAP}usHWMp=IQ~bQE`UO%YT-Ce(7uIUeg&2d9Gi73&`} zQB7-D$8i0VJ|s{zh+)k#tHTf9DkxFePLYF^FN@q3t?#V|Y}xnOubrLR?$veyRmlb8 z8^~_#p}e1`=}K^{r<1K^00-vWLY!kC1RCCmUt=%8o;$Bg2gW<LEr3aQXC#ub%*>)s z=-DIX;_}*!GH!6y>{(H>*$^M{R!6vZ#S<#>#PjaDF%nLT=Ui8vb*dKbeL{eITm3tG zFGX!gkfgml?61<#{MQtiMW59>l+#msC<j-i;;Ebk)z`Vc-vJvC&cOe@W*piG0_PVd z$QYlE*vaVR9Sivr@vmo#wIe2ry_V>d8>c=nK#cQBU+bel1Ds{a8;9sP3<nwb7XdHf z_`3`Ktr7);=-0|WTGB*zY4XtKs^3J~b!=TEblNs1!pm<s8W*^lW0ZSn)mCAEt;3?g zO{Bgx8Yh4rDX0%8ir5jVr#e_RVZ+44Y_bAX!HtM%TDKCt&U_dHB7wqGWiB7P3}EWy zGhO{9_`bXfXXbZlX-CbJrt<GW*d43LIn|lUyt5q+!s37-rrCsj4vzx%$;X2CFvubs zR-y#UXcu&Dx<rpugOO<*I&X(MTsVoB*qhoxgR>cS3uH08l=+8Ot6`}37F|E(NNPOl z7-SPFxcw)@oo2S#w5f&`KT5<i8Eq_Im(As~^e{+5=>c_BOd*nB*77n$?T8K0{ea2V zQ1urjYemnl4moZ_Koma0o@OMN7iOrwzc9}cTgX|!_bVi(>6?UyuJp6-mmn<JpQv`r z5HYdn6$Ut=E(CyMFqKJaXkVofto>2oA@vlY=e4E$0SMDH_lah)*8+$r3$d(fnTyjA zg4?XwB}}>A#6&;4)Y)m!%b!Kmh(FD4r(8Sfj3w4R`edV0dZrvs&cRp4^s3F9PF+bL z;Z4IT*$fv6!n5R`wid|}u8KtKGz*Zk4<26AXd&Rrvx7=ZU}ndC<@i(z1Nsr&_?!F_ z4xrB0nGN|nA<2M{Ys|NGhEM9xS%+?9bH>}#q1N%1{6xhu8h!R`{^)?Ki*q3&N+$#Y zpXN>SjK-i-aq=jFcXJoeIh-xpz9n3)Bg?TM{Sh8S1@4HGe+SyCQQs77z_wYh&;xP& z@E4HUw{iEaP9Xwb1^IW!NSGTPmtR4^vpmz;wsf_}j{p<XRp@3HCdo(FvpI#+j44vi zcM_-K2|`z9!Y4KJirkbGBr*w-E1mmBNlYIa#0V94e(=OFZ7^nEBPS1c-FDJpmvhDE zDKZ}*^wkg#K;k38`hLE^X<59^T!s>kMj&C+Dnu#dZjL3h=z`oaxR|kYkQ=5)>p)sw z5WJj~B9YYhiIqvGA{27KX}9hqL;{*<K*#4xYq<6iI-y8>!YdypBXd+>=zy-8_`MQO zn0F!WJiqg>_U~}wnb%jw9fxHXJFyhr%7N#2nC14q&#xV~x48i!{KH~o8#S37L|*%* zM>-v)s~6`fa+#$`qKyW_8Udg_Eq-rO^LnmgiBum@8|fh-Jo2AhgZ`qLN8RAR?eE)6 zpLAUysM*F&O7tbcPYbkx2wklvUY;3kO*+wlf#L<$Q)uiPR#_h9ilVi-=zcqMYz2we zp;*0#&Gai-6?V1?A-nYe1kftgoNHbOJPxtpic%6e=-XRjFxxnn-{%V06C7|f>2{Rj zQPv{RF(yX5SqRajXTC-oyjwkK=;%rwZo3Wu>{q)sDIGhI&JJZ?k6<{T5??>N{w9+E z;4~{^c@0WXpn2nvOpOC<ze;3x3xt<(McGC1aqdn_bf{P=dIDuveSzU|yaH(C6d2*D z5wXB1%a0*s-o_!t_QPMsFQv7xyJX(*Z{UKitvJCMIlw3~hB@cos!6M4;GZ0EM=_9J z<3bD&JEUwcKn@H!)W+)L^zmSE_x3?BiwL@?lK7mjA)M5Z?3x#f%@h8Ct;E?Ahy9Q{ z^1R7$fw0y=Ngha@K>1+if+{B!3vh~^`UqBm21muO75#olNw!m}znc(}UNs?d{J2U- zL~@sq-^O<r5|`^I{e*&nFdMqh*~x>idsvR|TZJN7^me3GwEX7#4<YKv!J$8^)B4LW z=Jd<jIu?yC@BGsmYy+=~c87d|K@0>op<LCQlfZtJoU1<kEnEgYxn6@9uAd$v&36HX z9~%ZoMi~Q8_i7*_TBNm+oShLLqeEG#xqMcDzUV04>K6JDn+UMP?6nl!?BgB~gj$it zb@{3wAJ}iSVw=}rMvT>rtrFTtIkI;hZM3u05_J{nf=-`r@20~DZ0$R{l>K+TB*ohP zxE7mH4}KoTK;VTj>JBgI#VLhOW)ew@^u{l0icfO9EiZuPLcqP*7Kh$DA_HCtm?)kW z4>!zLaSV|oiY%gDpj#QB!sAk;aO2l|-B6suuV0C<IEoX)WE4aZ>%$zwprNa?ZDE3o zVj%}Gh0eY|CRn{P<x*XVFIu`JQB%#{K+^(O#H_$fwAC)MMzYW<^g4~&y{xtvQL;#= zCEhvgHC)tE_Ivf~EMT+l!;xdY&PET^&*vm7$CaLB+o6Z9X10lB0!0|4NB0A=Z65?a zGemNQnYBq)o1R^~zM*LIU3-KV;REuy>9~dZ#RpuMucf-eWw^!EzY73nA+<|nkNnqq zbW=&hpAnrS0#EvLlwI}+(Y1<qn{CmKjwj9TOP;({H}qRbxryODkAoN5%ui&n)*m=6 z@Y)HfMmz@RJkqUaj3(q+>$WiaSa^X0@@PEqQ;E6Xf41?$DGQ(Kg5A!FjnN9jvU~y= zDF6;NmFzr*8}kQf#?8gHY`Fd>fXX^1lA^9A5`3C#S!SzSX3~2tWfecX<IDxl>rcVV zncD<uFc7>NE;x&3D!V(2e3B&RyT1UC7xwymn$=k)sb&#a_=wl2%<nGk?x~33g%;a* zW;v<k^nWaKUi`MRCPnT^Krh$wJ+qSZT0^X3>_F<mRW+$8sG3D0UlseM*`1_X`~_KV zoh*FHMH4ug8|;Q?6ME9@h^_sP(3{HbURAtCwWXcl?^e={C?1Dlv>n;%x6Ltez@M%E zhAYWZjvQ0*Ti~n>udaPNPA3iv1B*Tuw--|HXgXPV6PmnKxsyFs$EAO>Ev<18iTpvi z)nsE1NOhH9THc6AqwUR-LEh)6R>NFsC(_M-z#A7LH`zX22AqnY%M@Fwh}EL;Kq-g& zE|G!rIf0zRm>bpE9oCu)_aw=d>Tb+o&y`~q99b{&$~3ozsPN60!F?X&TGCq<tc)HN z4lF{=723bw#mjM4b(1)&Z2Cv>@ntJyeCdFB5n`RPSqY)Ou_&V=)$AC=y;gM6%H3Y6 zZLHNcdbR0*BWC4`#RQ1iy~f~6C<BS3G3gG>Sd@mbB>Ul)-tn|V;uzn*&p~ihbioxp zrLthRcV?7}pMO~7cZ&M4&=fexyTxUqm!+7t;NH##bxGU@=}sirrU_dgUXj8irA!<k zU}F@?QMS}epX?%y5etd=)Drsq(1=2=C&^Np#wRvH)|ib?q-~|wI<CE8$EV>#uBu)l z6REwniX0)h!+H*N9niW`G+lRlDLc$7js0xP;9@DAK$hBifB;|)Jq-)DTwTk7QSsJV zN&Ytmji{pRX$+1M1faoR+k+*|q#MsGBPq(H0s|YDY2FdiiD@3X0<aqb@ogL2P4mLq zz3<u&qx!CWd!c{INwE}bSjOU-N>ZN@1UZ!G_aj^eXTtQ))o*Q_u%uY8R1+~xclM*~ z*b018?YPCO-xT>R^J}3I7Sq%**|s>>CR$kD(r*E8D?)giCH&}kF7<i#AY#Slt^wUt z7F6knzq5Dp5If--9$~W=cchB81$g^9(w#Lisu#fEZaE|MXm5^i8WgBU*4@_OaV%#w z@PmH40iBA350TuR66C-b(n{t&;XFyHfO3M(og^6g`Qxeu%!n0$BRwaaA2J}!X@B<3 z_Xj7Qt(k4O{Ed-4F^>(eb=T5f13s+6ca3*XV(^pO;~&zt`L6BrcMMc+0IJrtjL<_E zE{CLb`<RcwCnzC$&bCmH^=Bn(*$fnm=)~8in6aDA%st{Ll;1vhOCwQcV$k+NZTBoP z@Ud%EY;*<vs##iY^D3f5ouKWwDV5-u44}$OaCay&65a5DOfmE}4??F97p)wt$ryfd z6gNmxT?jC=z~?dOfzJ4tujO_?BTv|O3z6u%DyJ5HfjoUCG^7%XcqOifk=OnyPlf-v z)HlIC!BP>u7Pj~d6hdAZf1fk~;%`xpj+hk`=^h|DcpGJiv3$*aUs*bP(rz2y!JMO$ zGQL#CdF^NJbhxf?*M<k+(`MA!>PAJD#TX|H0)KEkx8Lby6`fdK@a!#&g%PrwnW&F0 z&rp^8&n*f<^M~uCJJoyewN9wizK|8jIH5IAZYi?|UDp@VswG)ed`B}|f0%h-p{j2o zd7IQg=D&;{81I*OL;YJ#2=s{VtubR${d?EnTOVLr<|z|%D<~>;YB)%Cgnew{%9c`b zV+5+|!#&_OXmN1fuXY#m3*-I1#ZJl$$k@o{G<cjQ^HrH_2{X7D{(wLf_Yl19@Eo60 zq2aZnfuT4NywES`Q6xKoDDleJJIl<{ctW^LQHHjewEmn?D<S3TK4_uK-TI|QC1u~K z<tUX)8leF-QTY6Nw*AC>@H~QH@BTs2Di*hWytgGgHmiwT8Ibso1cuV*M8?O)WraYZ zVE&6Jw{RgD98~bDBGvg5-$Kf!tsGCfm?ve)7o3i_*^N7y#YB^-d8o%W1!Tz});(s` zBHtvtcJ2TVuKWSD%T8bG@`0}dW?xoNz8;QxniScI#Y+9=j(Y%Y@GH^ff4-eC8I&DJ zVCeNN4t|$Rd@^aV@?k;8V--7C6q;L&zVr~|Ibt+^7o%LfZMvj27x|}5V-E4RHeWE= z5;mwMmI02}9ms92Pp|5$I0?vJ@o+mEyd2G>$9<Z}ie)##vEHB$S7t-0eN~-kG!P|3 zAa+Tmkos=1knt)k7fz5EjeT`Ep9_X<lokKIN`tHxLs_rVl$CS%cLeJiT)pr@@}_OX zE3?@qnNm3CS!8!j2&Na^xmnpHay7{+!fV-^<L6j?`Y=B00870#I|;xSrsK#coTF(C z4eI)K6_{Z5SX^lSA_p!Bz~oKzPPt>i3CbrBAb@IheU51aUf92KZ2$GwF<5+HNuoUG z4e&offZ+n*umYThe`kh*;QYXb3*X&wrJ$jG%)tf6SRqdi^FLfDssQGGr9IJ=U_c3j zbq0j1&9Zg>;X*hF9N?B(ihbZ={zX{eUp1nFWYGWd%Kxcvf$S1bH~JrE=2u^XcNF9+ zonR~|60p&QTOf~K(mUWy`JWpL)`9g9>Mmm^RnP)SzzHkgy^{7n@L~n4!2DM8rRH%? zA<!mxk8U=I(*0Xp_3z8154bV2@Q|N>0P{XDz`PF(oqq5i|KG}k-he+53y%sDoFZca zmp($#O~x!WID~jE-mp=*lGE}fz7vU6yV)1e*j+5pdn1-~i3J2`CG~J)A_BO@I?pf^ z&kHUS*J~G7UW4_~d{x#GF;xSPt#c}FuWc+Ug}C_jZm+jVKWT>k>jvf%@EZ6e5`W!z zb_9q_U%^>)Qx4mYc}t|8f)ZY{3N#VK(_M1@#dCX89KRJGdoel58&4{&FFR}XMk_>C zisZb=!92y-##O@7uJNEwfl+|RwtqQOrmJQwgyQ;cq$n0Z0f{f1f_Z_o=-zk;BN&~9 zTFqiIo+JqSH*F$zt^EYS{aIt#-rPr*y||xLCC+&<|H}xalt4MmK_@0qYLeKP=<80V zRX^nXy4l8$J)_vW#h~0VKxa3O75h#_-F3U07mLP*mwq`t1^~1$_PMQ|UB!l}DsAfB z&lu<HjM<M~3~`@4&J?6?E?F+cy9r`}&b&H-EzQevlN8Xr%pT#_me?6d%gv@$F5qX7 zftHTVN{HZHKT`KVHZmOEeZDNFV{|EG<$fm~n()azJL@vBQYoIO-hF$Gz8y>jPPf)l z==I_i#2ro>vvvnK0B=qXsNUr6o4Ez{X;$x`&sEfYUT12^dEWBMVg*KxJJXtbPZ;QN zM@IyRvw6P<D-CNP>{<Q~JD2;|Q=j;aqfhRQ`E4p`W}pMp_Gyj)qhi@T_8V}XlILNY zmn)O<*B8_P%&RsSlT(%w0?mWa)cxkV5xqtyeRQzUt2CShqm5~IJ5B&ks$Vn{ioc(Y zcqPkUo}YdIEkmjF(hsE*0C~`_++}mLX+$yvZ<Ui8@|18><*?4K#7$bwRx-zu9ZZw{ z{H2!yjtWhNsHb$c@M3*MzE_LlVVp#)Cu0yjgk9;7(=Ii7uMQxc=1$Orqt63?oHX%G z0QqPCssfRMakHO}A~Xlw<N})qID)L~DtHN9KR#%8`Mx2T&wUqHwvZy-q%$LpkRst= z^XV!i$Q}TZ9?1zbhLW=3YPq--D)oFgV!o;SpwQU&B4z6&&v1UMC%$T3Hz@$JPb_+M zs~OhicNs@9_f;|ay59fM0;ufyrq#Q1@f<<$6@Qnk8FN?DfsO~ysABJfi^DYa+Lr2c zH{wo#{Z;K{&iiTt%pUHJ7QO`w;OH>!5o4S7F~l`@U1C~DSM282w7>1#I+-}f-zfN< zkjNjTLdbfq@|V}zClbu>kK98j2n3xf{`m!x>QBb+zq9NC?Q07uc*)qMI?q(W8Y``8 zMHxll*qYq0gVa08Jmb?=lFs3WGlVM)uC;lRNcZ{`WE!@c)n70sQ>%#~3ua)?tG)Yq zVmoGwwB>xKCdZp-ELGNz_(ra0P#u4}KdzN=tjW%IH<_3eQ!)!Mgg{rt14d(IF@6gD zczp@f$)?lOT)=K2w-3iH*a)<kW<%4+WyP_r5wgK9Q~r=A;sJykY@si3*&oIbrV6At zYSc-V<-w$H*p~ZH|K`?k3@SMrsweYq2yJ$c;^Q9#B@N>nLD8d5!RCqMt(ZQV`z#I> zu!+V?&O4j4nNO_6)*f=NzIDl_u30efaaK5ekPCE7XVQ=fnmb|fV(~1237y#Yt%C_- z%Ev5K$51wRs$8b{N<8%N$VWO|%1^AK9JaO3I-v<+7AZRc<CN9;(3<0v<n*({q)>AX znp?eu@FYdl@$DDS)I#i9$rhnX4HbYtdJFIjC&ufmzHEyUn?)N1mPZuj_i!s7Ldczr zeZHcItw5P3G8Mi|#633`UJCbl{Obe@hzK0giCm@j-@u8ILUjZ)8TA)t2A4}&p|LCO z94KBP%LU7RtQ0At_!$x}Pn;E0<*x&)oHnz3xyvRwB<Pwuk&l1*^*ZIAV3KKtcr+0q zu26F7gX5JXsxZo(TK>v7c{(pfKB*i3az+qmKL3=}QmVL<MPX37Abe$BWoyPh+frKd zGoP8Nz<kLh4mzO2CAta}%^6aHH3mJ&N5RP2W65!rdv3R=F+sqKH0g)a+ej<~Q&XlK z3ltl2M*(sb(LRBnK2(%H^EnR_xRQ6rOjiTr1LkZpk+QbUDkH(Q>4I@s%D!Q4NSQwn zu&UpARXAWaa}?v=RrZ@<^h$j*y-K02G8lVHr`<xJW-&fIu(E<FN%hV7J=Oi~GA6=B zdx_;AYUqK_23HaUanW>`lRP;O4wnEx^PI%%V{&^%Hl1TC6{5)kUs<H_4s$XDlYTu0 z@QX1FYk&^c$=EwOLNzJ&<qGI6h=7Hil7Py$3gP2^coVtu-y<0f1$-W&G*dcYLH$I9 zFc3@d8SL1uc?sQ`XUBHx1X#lb4tK)Z(ZwsFCPZ={;*cxEq^T+$8+GS<5*>t6L-*!W z0H$uV?D^3TiU4!P-`cSXb7(~~6=jJZG2el72^>g&9;Po*ISh)HGhFZ7RjQtrAaJ`1 zUhKcdG*yi86HShxT*zs~vb&JCTW1AoB<1(inOoMC9v|oI3Fm9l$Pt(Nxhk#otXG65 z;#k!ZwdG<PbRiEgjN)IylvVRYNa;yvK-%~0rG;k@HuA5B5t8Bt^Qg}K8E1Dhj}b7? zK7EUvgxuR@aHg=+Bp%Ycl6T{h?Gxsk(d`8|%Q6%^?71B#MdLg}+}gD;|JvkCrRc;V zDWrhySX<Q5f*nyfUaca>mYoTP=|k(H7^Yiw&adiZ<^E4sSJp~FYzS~zje-*g;&@)d z_y_vK1~n2pa^I%B#pM4wyaNXw??J487li;1F-G81PCnSOQ0Ys3WnDIPA^43ik>8iS zrRD3qCIf?dMN4#c;TMhQeK5|tDND=XFqLrtqAG|@1~FG!KOx+&KJDE|LTBc|))vY! zJ?m6UxeaJ+GFTUrX19lu@qOj*#IZVE;4uc8eqjC)EgR2k--e2PBOS5rA@!Pma^VD_ zq}|Q4&MM)86`^^kt<2Pxj%X)|vECwFOjR|5$`D9Q3wdQ@#UIvcrEPqo)GHwrH&Mx0 zbA9q*)dLiPg?>_LOtx2?PULo_I!_XW6|iHlv=uEA@y@e#7kzEL5!fs=quV(bZ@enT zRmJDjWD+Q8pT2BLETOih@~)8;1Qr9^SiT$9=<PSA5JE=&cK>9;HN;;?+4vDffq-3D z*PqxyDz6DN4bf9O->Thf?+vJyh?&vgQT}zH)h8018b%(I6r_OAj?m!fJCJreb+II| z+`PXkWGct;S~-&;hps8?PKhr7!UR6BJp*C$iyK;)gfDj+u$DY#rmoJ~e#YVhRSJn? zx+4_8vXn#}@HpPl#2=Lb&S4R*9QmSN^t6?0h#a4SxV->hON_!FaOEF*GSwH-k|ftw zmcu6kzUGzHc!)wSe~|B2$*YX+$R)CptVg#k#^Aj3UAnnMet&22!ogaof~kW#*-Nr| zhXFGuH>Kr9@Cb}t?3mL_LZfNHm1j+tFX+o$sY}4z!bX2i_bd;^kt12Nqe)?9-DCUc z#&K1b;R(g9eslvY+Ixd(i%nKJl4Oj)HzzkNGzDhO2(cnAJ`8GEWbNw70C)5_0--p{ zHe(h#iNqv<Otk{|B!{p=qUWbU+NqhXy>1Oe6%{s9s=p@XXg1&oV8cg!g$>m~2;QBZ z4_9U-rQO-6s(l&Pusqrzv3+`BdYSN#>Jfr!nC#`($Iq(0?V2xuZVbanlB39x<YoKg zp7uS`PPrB~k33O`F+{(5r2~h=IxK*+0($C^^_CVdxI7lmewnHwaozi=Z;2t`xh2Qj z9}EEGw_Lc+ZlggRvzv<uvuUEKvM88#{IkAPnNo;Ab}t6?y1Dfm5}1wTOCs_5=bHxX z5gL$@;6SJfU7Q_uy)kO>xkmQlf@-w|?1PCmr$rrgp*Cug;GEuo(}+Vj0Cd^VPpT%e zLe7n9a{1|x`<z_d)B&kAtG%-=-?aA!AVsUDBtITWV1Q*)JK`WF1C}~<A3e09&kw(; z;Qbyv4N@p1;b6e#>PDd+B;X7RaUfWM$m+pqV2xoK>L9vv8uxnu#bE2FXORKV8l~NT z11kuQR}x$WMefSXeQWCs0ArkrC%$&Jk&H}+FF;a^#bPSEb#)#e?S3C=igZ+<M*SeC z_X!>MZE9@eC?V7yl_Vrtn<;wev)=8~+xDV$U;^skcBH0whRzX$xmprMRiMv<q|3HO zhCxE}d&atS;^7`sspD|0mn0A16wYMB^dUplOmt!B0__*ate3XK0Nd7#f&prxz8+nb z*qsri;WUBzFe}ZauZ7;0JRBvN;N~>;H2K!b+U7kRq4AJ(%g>;CbVLr9j~k>Qo_p59 zpUoN_Avt{I4WQZu&r~lK15=KBRH;hqgLumij-Uqr;SYIJR(Jef;UM!d&rs4?!;LeD z1yfLpAl|n@fag{-{EpmcN;g73Oe}`!0jNeSzV774o2gD#PVm@b^bqozKR-^id&5=c zxBUqNw!QeRBKVfngWs3~$a3bJCh2w6vQtl^IR9Eq3qyk|-5i>15@3;iz>5r&F<C$v z!#z(dqER6sJ8R%7hJV!RasgBQ)_!4n8lYasWov$nATb$nE|*dRvU(3|_mSx&EPLgb zM&~tD6Ymy3jq~2DS6382{j{PtY1!H#^Puw)vv(S1Xb;A&`_N23jyNDLX8|-Q<+vVe zwxEtPd5LRdE?^Q6l}%l};H`FM!*FlYO__ja-a9tw@k;6U#bMEC_6QttT_1k`g68cL zV$n(WkE@PNNS3C_bD$W!z2AK|wA<^Ur&N1pzgzHj<V%&F1ezgz+z<x`YJn^@;05{( zuqi9z^@L$pPo+aIt}w?i1xY{9eZhEo57b`w@>1_R*hv?jd9FL3CDBb6!~6bK$?R|^ z^(ywvhcRv3!@dug%PNsN{=)y|uPS=~n}qo@`ARQtdJ<RrxX%9m6@VN9csgvq4LZB& z7AV+Qu;q5u>uR>TfU8Lc2L6gFjlLl&e{%d_au|dOPgAnA^&=Kz2NpHr?NTn{m)lN1 zRMq!F3GGZT-lkL4y~BV52A!cL0ow^r@Rx1+{1X%y0~j;-59;qgG`>&Qbn?WLaV>La z%tF<ug@r`3<^t>Y@_ND*hTYpTRYn_hu3n4!UsP&0{fcWWdY>joBnkkw`D1jT!Q-K_ z!W&3Y^mCtV&0h40DmwZfM<aG~mlgAC-;#$ouImO&V8QQnn^A8#{5Z^e9Jvmp8_&Y* zBOH06T)y7p`_Cn5u#wNcmpV*9C*SZ-1qcgsjW(e9Kt2eO*jc^<xBdha6Hzg&dBnJP z41WPM1oOcFPySWlqy$>vG8mABaZ^2_!yRjWJ;hP8;|qsZNs>eKTCn@qjB$=`zEvDq z8wsZ5s|g&ub%WIhzMPJF+Yv_-K=1<loeK1r_B;XxAprjJ$v%5SdE2>38g0=j`X#;j zpb4jV#QXT9D+k~TyguP9FQ?+X<bL?)28{-w1;gGjna*WB+c^cm4jo2K%GucuR~(8a zh})z6w$gk+@>|!3<aW#x=Fg~C>OAq<Q<a|3N2$La{Oac`X@A*wSp6>it}Q&;g=3L= zF4m;-nPM>G*Ui**L{%ZfadNq9K#fd*eg6D7!_Qt!DsVMQKEO-_%>5DoV;&?L)-ZCH zr_=p2qY{xVA$NTgSi72H!~zSy!j<2cA)Mj_dspD}X?`uP3$DU6wF0D1@OcBaRmKyp zVpXYy0Y4pEO~{yJ4jXeCoN~ln9Ek3BqnB&`)r1B(QCvvyGLkRYbZSwBwTJuD(0`~F z{|T*vePE%1KF2UsS+v0TKdXL8W>Rg_j<kecE^}=>k965v&!T+}4<&c}O{_YM_P78H zb(H&je??Bd!7?(YCKi9BaaR5v=7Y(RzkCH;q<x;(6nmOr)0+16;$0l&9L42?(~9}~ z*V&RqJ{t_SAs`At?%2X*K^GjWj&AMVJ`B@8`6#T@#9dt3%r^idHrgTv*eA)}$Q^Pf zP6v^{^kACMKou~)PZ3?gIbMxV20RMIEoQmCvG|9PWD3r!ZtrzqOct!Fdvd=HiK{k< zQ_$CAnwFP9DI@~XYf5zY;tt`B1oA?al83W{701_T4PnGY76X5NLqa1BEGhfb13vXR z(<M1th8WRGV4z5mFm<$ufq4EQC%y~96p*>oQ1+I9D{O|NJ_}y)300#O`@FJiBE5B0 z0R{X<UCr|CK<f^fN4c$DDjr5U;Vy)2z~H=LB7bjwmv?<p7&J>3AvhjDIBn`d!4n{$ zUEvL4*rTT1n<oAH50MWB^71eJCPmPkIsYVKI_Fz?$rhdKbMbi+T|&gC)#Gth6yCtT z0SzFuk~~SxK@w)Fjc{U&voUTn$^x@R^}Y5Ks$4&sc!IAzYL>|3KI?&J;n=9cdct3! zaR7t79sW4`a*Fo90sw#yZ#dxflfg+@WeSdWzK=F-VE!5+`x`^X0yQW=OJ548$j0n6 z%05nqLk!mOk=l+9BNQher(`@uj=HgH@Y8*)DRfNJS1@|<IO}jR={#bPD;Hv$Bx!m1 z!ZAcLr4eK7jyx_?J%3$Jd4+c3KUc5P`i3!_@9jv8WC(XhpC?f6o=#ZmKOP1#vA=)w zlT|OOcK5ES)_R_7sNYC>tJ=kZ7#20E1tJC3yLhSErDV$l`zCuNMXspoy_DJZ#);<l zNn$t=v@o5X;i|rN!JeO5r7p`Q>K0=DJ=29%L=jxf<zmPT%&<YSD6$E0B}=i=>P2N) zV`YG<7a7Yt#o{QXh2Za3oYxwub9#q$$IrQ_9N^k+&aO|fy2tx8?yrcbU>!Iz)m<e_ zYM{Z{LUn{Q8P$=IK_&ABrAg*CKiV}dp!@aSt;j2-I7jTeEsiIi;SN8EzZ5=1=em!R zGY=kc2H3w=U#ijzv1y$<eaWXH9W#Q*Se|GioGG*5F#v$a9OHuVI-7wdV7741wlxhD zX}>#zWid{_$Ok{_h7hLT#la(xSpeC%NP)kt;0Q>vWVgm~(*!nYRxC{+axr@K(DB*` zr%seKKZlCFW1I{>WP%(I09e?l`L56UW~RFe_q(ChLEl}53+oI|S472z|65Rj7Qw*5 zqO0saJ)9p0gn>-LcaFm~Ew`E(=m&{%U#ww+jcBf_Pt%n9z$jAuA;M0LpX<_GjL8uD zQh{L5d8Y1?xmGJMS4<B}<Xh#7$;ln{x259*hZoN4q)9C_I6ZLTw$kDTVoM)yjj8cr zH(rMIl)C1Nl)hKmO&NOvXSMJo<W!;z5RC$l-rhIfH}P)ytP<7*(L5Bd;*NC(Uc+Yp zepC|Je%J(n=l_Gk|L5cY^6%ycj@=U97+(t58zu4TJ9(@g$4LD)(q)Cp=G!l$%+W}E z=$szw2{m%CJYO)51t7*uhL^oIvqk@#3&F90zic&0)IA);7$Fz~cY2XN`2PTKK>HR4 zobWR=C=rkW(G%QVkkgP7`|tYxU#r3y?7n`ucI0MJ4>|<_i+(7S@y_HQEUAHc4nF`1 zecetK1l$G`4ZgqBt1YlO_WuXL`{V;yePw>HrZjl)zY)095f_o`A0&VQ9;OobiN?aC zJ^^(z7Wj!QdZjD>>51gQpP1^ZGEWZY#}9rYPf{u5e|n-_@DpW{LO^(fKI?)1yQPES z`#-pG1Cqy3zz33lOIierfS`k=!~)4_ZvOGoMGoM5Hrgu62mxnQ2C%+<b0r^Y!3mwp zPnziQn7#Lsy~{LE@19(I!b$Jb(mZGxyMu{o-sC)A^`3kMf|2yKF@fu4MXztPndp12 zs|%%Ie3o3cG~-Y)sP2FRlF-Cf#%C$5bg#X&Pjyx7AgIzO`|rbP_unxnJ--2>^T#*M ztxxxrx-rpt@;f(t%NWalZ&DcAN@WdICv(`2TCgdQ{!7CH-Fn`{wlD7YkKM;al~`Ui z?O@y&<_D91<u`(wU^Ns?r*u_aG&mlppU?)5n{B!TW0~mPZmj&^mZtfp-0ao1N9k!D zNy+Z&j9WecV5(LLUuDF$0DzV@pBX~+E{A{G$U|$!CVrG`!xWv|A5jYc!?s2J&g{5X zhRB*}M#yb<3rnY3_;ae*PbzwRVoq<AaHZ~yq<Vd8^O@}P)4bZJ|Hsx>M#Zsh-4Y~N z2o?eaclY3K!QI{6LxA8K+}+*X8ut*~-8HzolUIH2x%$2_`bQ7wQC(fNckQ+3nxfMY zO<xUk{jZf1&Cc{T6&2Nq?C+oKER6g6u1ym(m213cIuE^MQLO>5gxoC^OQj!=bT2La zYPk_%X49pMaa3j<elu-nlzM}Czit5klN>;9qzKdnB?@;$lesdo5ncV|x;HfPm5`#v zvQKNW692#%e5=8Cy!wQzq=^s(C^9e%8uqr4{%O(xo8p(WL1c3%UwWNVygD)Xf#geg z^>H|?&H;eIPlo64?vvfb>Xf<(XU)E;f$=Mw^>Mh?&V1&Zeu^VhhOd*yGWw@4#1AHw z$8gJMxf}pkv6n#~!whUFk#fMGisXDH^jG4wU44{`A(V;#Jpq5ndEqJf!RPmczqM5L zkh4~re}cF175q0FM;ZpK2ay2Aw3SBx*xc99z>z0H^Vr6xTWgBnT7{}copbK&(mhnK z21Fwh6a@6%SwIG@oKD|Je)!_o03~(_oxkSEIoMcHcQfWMrIs-mja{y_B%;gU3bbrI zlBnv?n1c!hH!lqNE`!C=pW)v5qVvB9_-oA<dWcFXwnuutv~Vso1yLKkW%OHB?Sx=M zV=_qOb7ZsRHjeFp94rJ_Gle=pyDR9<Tgrmjv`{w=6U#+3Yr4>GN5#@w!W^KRBG)Ud z9bNz?2^OGcSj}!&`KCtD!yDzjOkxs%bPyL33ip?}15hC;q0Jzb*e{k|%hp8##u)Z1 z{<_n|+81zZ_l&5fj5ZU^k6PCwh_3keVu5hZNh{!Xy`K-b-OAZR90SZz6j8VU7=yP@ z3jBY*`*bX{q*ODxA7!{*_t5;P$BP9_X2``|p34XDi;ZiMSQgGXo{r#cilA>LBg>u` zo?mfz3kKdcyz&1mhU_IaU-c?{km>bh`N_TqJ1sN|nINCvo4-hT{VO$mX1?s8_V#(~ zD_|Z0mv*2$)TapmPOkaw_NVNAC-mZQzdT42E2j%OU$aNX+xKVtBC6arAHmKO3MN0H zO)|M($`tpMIumTF0ZR-epe?9=<MTqM&tvw@dnoe>gIcQ${#PWr8aksY@$z8S-Z+)? znB~Z#EW+iR5dY^Kq3_Q@XDY3K3IOt|B2c_obFMs1V;a1j&Cg1ZG(<4zr?Baxtt6#- zyq&oqfwqn-kQ6S-a!XyA!BAMw>UavYU2Cd5LOLL9l`NdbVtJwqtaS-w3X~a&vtTK{ zHEnKma52;m+sB+`kccf()_ct=qVd+tUzf|u2TU>!oUDP{PcmLM^R-Jo>N$sRsq!l- z9=rLiC35+s+&*S}oTh|<AuSIYO`=mLr>tVWM3FsrX}=ZF<olMvK{Vs>{_IcSR>=ia ztC+)9-|SUnKju`%pOepkSrPK8*K{S`wprnhhr>)V?n}&6iNh8fe_%sJ(H=+KkK{U2 z?vdw_2V^Y;z;CFKr4x7l#J441gT3Rd@8*bllt~A(9Gj2;y_f{{)UMHu@DBi-Pb*m- z3lH7Q=kcRjlN-=4K=obhNMSl@$!K}b&uBvXL2p|8U?M&AfC&%l{PU<(MR0qK$?z-) z*!@%-bCXYRM_gTkob9=AnfN1Ls`bh!jkGCbQkG)3ex*8Z75P_M)ukZM`RA3|Tx{`I z_Psb-KKHUPSOK<@G>s0yE&94fbU!`1+t@Cy2F}Njz2_U2Po6V@CwvMs_3>zjnCk0q ze+RGI-wfY0BYq@XzRMqr))*I6I^>k^vECU80ldnz4;c-q4P?JE&M$S&;PQX9&(fSm zx4b!CaIjsD@5Sm(mMuSaGU!OQBJ86E?syFyEB^NPk1(}`4jC(vyB=*><3{0s0609a zPaJY6Kfj!Dk2o(Dd&K!(Gf57gTe_O%F8>g*ZQ0%jiAS8v5b5eRUC@m!4MGV6IV^*# zPvS2vw<oZc(mvi+)Jvnli9+}IF+QoG^3zGKmtFSjt)$Qnq_VJs2-rlopGU{!%n!Th zmA6h>u6K$E(AW}i*HrZmYKXo!dQ-HN>hf`NECwysIAGAFvCWa-9g70%$dbSP7S9NY zt}`a!unZtoOnG1+lZL8UckV@M4NsJyO1FF);8GtbsOhMBt0cN@vitMj^qA&2-k?GI zk@+f4FIVJ8b8Nu0S)8GQJLOI@)y_9Jg~?Uh$;>ithwMfv*gFoV^)W}&)yyx>mjMlF zke8$XX{OMMYTEz)Wd2Z?(1s%$X`oJju=Tdh-e!Im^x0(uLe>;-`CX>?`0kL^OWz>S zfJ9?t7Y2RIfWP&W#xa#O3V*8j+bw|w(T2NFVO|hF0H3XByQNXN_e6t+F8`M<y|#AA zSU5@i6M%Qmu|9>c_q?+(i`W;IkFZ*>LD`{FmbPiAw89b9QgxF|&(YIDPSuaG+~o*C zWqIp)_aLkD+4{L_4*4>LE1LyyZb(ug6L(XwoBRRDq9qxAdq<>})C4ek+Ot1eK7IVe zV+JDre2SKD&LS=z{!-W;sP)ig0lixi0)J^c>y2A59xGGicu1m+#pc~k?pHn%j-L64 ztZcTZJ<Eo~nvYZev=N_U&0heL=z6VM7vB;2U>NL^Q93u-_*mKx_66~59wa!PdwJg) zTJ1S+H3TV{7mWV?9%7=*OH}7$;~SA-Klq<OjB!G`G=V|c`)y*N%a4Dt8-=_pSA@Kl zB|EF>oC65FnHeUZFND8kcHu6N+}Al1;tf>)KtHQMpUWrb4YK@-39XlgP({=%wgJM9 zu@7J@k$ZBVOZtPsAO%<<G?HF=MEuP9pusH@!T$6~zEO$aO(8<Cal^_+n0qgZz+;>X ziIuJVO8WEt5N!UeHE6fHG7sCw{blTh&W{nW>#<x&enMBZn<rr6u+z0FYdn@jBwy3< zX6#(12JHZb9{bG$2TWC5QK<%=j6EKi3uScR9UG$@3Z%|7hn=2SX4Ku0>v4mritUVQ zRU9e^&_+1lx6*H?oNrz}#CNJ&Aaffh#{nz=S#Z~t0J+^@-1!h_muF|OtW6EE{#7{l zfT7Kvf};FPKY?ojLRvT$YsNe8ny`-5n~=4cwn}BC+P$%={O~DDcq9sDe?99-8Akj+ zom)FIh1(zK(!8{oVU33!89F5&bQuJ_DXB<NwBP_sQnF|)xVD_GFHmW_y@QHV^V|&+ zYRi06mhUUxM&_y47#`KjdJ|$mD8dI|upumHdjZA^ekWLQ>Dw|0mtEu)zz)g{z1i{V zQ-x{JBl{7{(31M27Q>nkv0i=7x6!^wsT(t`xg#08t-ukTvQA|qBV>50;x@nUK=L;H zXYrY47(i2U>T825?vvdvJ$K{f5)(T7JF%c41Pmr0<PlXInNQWK7ks*|k8R`5>!=o9 zLP^^L?zlqbpZ3ZtXeQ=tSEtpnuTE)7iex{&s)KB<rd%H%lT8S;(5Q;h#d$X%ieMJZ z@$U~=!s@%Q3;+hL(a15owWU7KlNp&!ozp0q&OBgzaMU5vO^5_kTSe?0u6hiW;)Ok$ zXo%ZyrXb;B$VT+F5Iy0husnm?FadW3vVt}FlduZX9!3U6FQzg5<+t6q6y_||KEKJ8 zT?v0dh5KOaigq^+DdFy@L-gWS3k6deyd5keDjnMw7LO2OMuh&v=DHtuy1i$qh$ynn zN#@I5BFC+Rk1OP}Fq}z~1xr-Iu=Zdy=8eKrl~9>qQ0)T`r*kd%GS(=eVy)jgPnCD% z>-uo*$Y5?kj1p0dGw|z}VP|@ZW9`s@VkEf<v>fdBZ8JL&blnorK-hK9wG`n+^(_eE zi_T<LHAOJb3M6T&8+*A(iesBNo|tr9?2G#qxl{3D;481;q^W2_FTA<TPtAu(1kqb6 ziB@3dsZL2;e17Bq@NNp#@gb2otwU|Q%f_ZJmAyd=Q6{k)!qxVMz|rC<P$gBnSKWUa z1=?;KWjjZu*^f&nVs5pblsMlch4NF4`7|$@0awlAww2Ac>&P|^nw4HD=jBOdMGL<^ z3-XkG$E86n)``ah$WSxYVy|zu1zAwsH(3ZSfYC@=o}DLcqzg#@vQAl&ssFuBarkpM zr(I7IO#<UneJ-h_q4RPJM$6e~^UyddVCcVduQ@%^a?i!SxBc=R-@ROejAs9Jdk_Ec zlLnCWC{mC2S~+~b5y%QN4*R<B^RPKu^MvP-cm4+XMwlaeVEJN*6Pd4f<AE6L8wBmU z?*pdU<zN;d`3c+;q5!g<p9iKC{UIY8nBS~#H$aggbcuak<e#YNnG&e+dH^l-<dUmW ziD?>s=x@rRZ9-J2LCl2!AwIZh=oGdlZqK@HBP)^$299U!#gZEGw#RXGS$QU*uD4$s zV|_vkX7HME*Q$N11$8vqQK^NZEg|5IOSkWVIc1%*0<>1u4@=F30bjb$B}QoKzDs)- zUy@Tea-Xq<-KixyN1-ifTSc%c9qnTo0e7}+a><i54%e$_k&+MHSZt7RZ?XEv9-pP+ zH&gaFwEnVAcpW5<;<Ex+3w_yfu`P@tg0Hn_3<il+16s;*63zBL@hGJ3UoqYMKRMx) z=QUR9luBr{dU6zRy|-2nMrQ$>uL(U_z%9+WdzIzux2Q(f>9Nvps7Ai*=Y-A!WCWeH zg`KSZ8q-L0M;=0!XcbQ!wb{~9Uh;N3iDiLVd+DUzR1rq@P3cefBnG#yP2HMHgSK(! z=%O6-naqlD=Fzzu8>7xxMVln!e}Fn%;mW0>U@3Uitdrc%O2~Im{`w7^Ky&~D2IY{( zeoIay;nj0VPzIy_qw3R;=vmB6FPHyKp+gi=3lB;V;KVq6F!JENb0+mUTZ6UzZB8Xt zl&x<;7C^2UJ3R7V_oyWMBxAwgJ^$WAdtalfjhx<HP9r~x6341PxsvW0mt~%`5ro|& zbct$|Z&78Cn2!th8|)9zP=indRBoKSEmC#v$22{*>yrc>)5ff2+uR1f8yXzAYXYy+ zKWF`=<BD{aJ(!n>dr4kS-iEV%*vK0LI_;ArOauz})-)O-Z^sExaCQ1e=UkzI7W7Mf z=O?I#bxKhuuItIWD+2Nnb+iu`^7B@CbF9$hZVr)V@NG@0R;lbgBU!9S505zC`PQQL zn2aL%Zq)1IhhB=uWLRBs01*oF$+?Tp+Xm@MBRnVrF6oj2|MN3;&ja4lBU#Vx?PdDK zqj=S=zi<>dOfbZflo{)W067WeYc7|($lm&0<HsirY?Hh)<(s#v3___JIt9*s)auoZ z(+3$_ORv$ZG=Q9de>P1(*2=_?JB8acTM=11kKIwz@5`d>*RRrD0gK$D+zOk_Dm-5o zb4WjcZ6$Er`})|U-7emK5J}eo#UAzPRVoxOn@;2j^@ECOm}-Zs`#1x1^>FSj>T zblW1yz8eseOb(-w?$&ZlR0re+epS~1xSJCpwTnU;cvyc{K}M0yH(OvzMHlkRvTfxv z3Z;%^{Plu1S4{1S9iY9w)Ss1zuFd!&GryR7(v>}|zEg9Yf~y_RSDDHxF+Z-?^Nw9b zIhF(t3})KJ|0XNLO;l5KiZ-o>wcf9>=qSy17vRyKX5DDt(c}&54{0vhd&<#;M;qbO zv-hk@g;k2zbObDJ*3P3eFJp-7jT~YmKb<igDm3EqmgToBQ6A90M<p5wjdxEgWjpE8 z*f%P&{KaZ8(3{&YdIJ@!AwL`FDJ%|J97Jez0}>N|4~y8GQ6VPqmK8ZE$k#k+RWM-H zRih7Z<gTyP-mJ!!<gsd4Q#d(&=_Amk$j*dim4z196RZvH*Cw=rMaOkM;P{O=g%Lo- zCv@(%3&;r#G3KSeBDoTlYcUFY<|h}b{LE22u))1d@9`vD)$_P<3hNb4ui9v6xM;DH z)Ns(1o=Q=)w|zM0^$19%7@%2zKT>TC@7F{Fy8wI#Y%=}0a<IR@E(|J1W;Hn%-Q>XC zU2t)kyx@y!_f_X3jW)p67N$5|IE`%|c=1HFS7=0|WU&=%<&%vNFTLlMU7ItJ9kxs$ zgt&M!e!%NHy%Z(Alr~Q6{^@VH>A%tHTwa4fnYU~>xc9%>0f~*|N%dQ?X%`?>CBNf@ zt_R%4o+uSjg$zPh@*CK!W#d40B8^7JJvZV0lV(z%O5T4#4%3S}voQj}O1L28*PaFV z3n>!bY#~u9i>cp)U71R7D<O2Rj}q=}Mlia&{M-Y|&>Zs5g;tI>G(jqHJZLKqz_PmV zB^Lc4Nb4h&1ZlBA4+_t+(bL!EARxYT8BNFyn3yUshwarCX2{We!unLCKutp`CjIkN z{=y27><2B8#8Ksa^fs5M*@v7VC`K4n4bTLL4(r2W;)HFwmDCt?dJ%b_Yia<`&w?+{ zHLBiq^edb-=%%vU0VmF^0JnbGgyl+Kr3;|M&|Vns?tQb3Bw^1TI^fIPyM2hNx7M;= z8Iq$T*{6sn36Tj^et8{UO{HbI3I|f(x5MGZ60w+6^7XynT}^5+hA<X=i~eGzS5Pbp zQO89d{&J@>MM)?QslnZL)2j4f8-Oj<fuvWxi68V)S~tFOQoJOLy%vt8$RU_n<5$LT zkfymV8Sg`>rre~gUAK*mkp+Lz#b|G{1Y6UNe=QL!EF_8%9CQ}%x%zux*%lfPX)-np zzuLK6*~%$>Z@5YTeo{pYU+wL)5peP)j~m$;VMd}ovU^fY1*CZDTr}c!Aq3R%A2Dx- zUjJNXpO~7)3d0kV1(C9HMl7TD+FU7Ct&r>-ChBhe1yaF*$?Awfy0Kiq(+U=-j!&-( zfXGA{B>L^JO6*saDGaqgzGkE4nr<U)ru_YgM{?_)71q@!GtLVOx(Us1fZ*r+k8EM4 z`XFj@XUGz@Z-gU+gHyF5f9fR(%XCujtMhS>9lBr9%dUu!1_q02Debc1(I|1WkLn)* z0dqJBdW>mi+aTj?5@2LsBUx%e5;fdQjx)suFf4e<$F`TLaxNhUsgqRmxeRW^7rgx< z8iA&_RqMB7lx~8a@B$oZ&}Vx8lV5;RzTC9mVw39Q+JQPN9KM8BB3(Aa@*XDfrUgq5 zv7j``t%7KBx9uk$Jz}5gF|g5qFD5?J<SW2ED2GHW!i&EA-N8oIhh-G5r}{FTpx~`9 zOedJzyHxq94c85kr*A_Xq89>JTDf>#aapR@lV{h#wtuCguzPnsGzGobw!h<Aax%8* zmle?NU69K11QzN<t_t-|jCPHJL^+^6?$Zky0zF70jz(ne?t;F1z$zrh@$5j-Lg<PB zF9+OzMVs?a?<i1#1x+CJUFmQ%gkKB(v1-qgv4;sV(p15&X3vWcTiyvE3s1LGRkb)1 zPx$Kgm&!gJo~7@d%^j<=D<9_#m9=yazNxVaGl_8WLo2-_zp&&x&(e8FP~u%m%YtIW z-e+4e@u98lsJQFuW!`_Z9tDcQ$aL{t^bC*=s|y!EW?<SFwIgDBj=3J(u^^OCXj3qs zU{ls=>D0aNmdMfl*0P>i02#+pI+@e8-7krDVa88QMqCkEu__qNv{3MyolOS;nki*g zf$|rIRYTX0J8l=`(FH%$V~1s>yJH~o=aU(YR4}Y<4f4j`*)vZRDPTm&R8C`UdtHfr zg6f+#5LvCK_`yts9l&Mec*<Z9IQo6~jnOFey5ks{7?xT2V}Hm3rw@MsHjk+g9qLqa zG$a!gIzB*iEG;VhO?>K$DS1tTZW_Gp)>QI^I09QBIi=U%b=2#W?N7`YJusP@EaPsK z^Pjm@Re97hfLZBwzF@?Vl8(BE3>q1Hv6W&0s3Ag9o<vBbvE<j5!n78@?qJ-E)0E{k zo_F@6H1QnI(^;1spe_-NPxNyGL)_}<<;?=~;^Qr{YIlJzz6GTec&78ct*#JIMZ9a! zP1(760FW;byLNGXM_#@NQ{xIGy3Q>q;%Oly+RxE{(3AUnaZKyZ&#T^a#A3uM7Q&k^ zv@7{#{lBAt-xxOqz{IM&P;PFY$0M*D!}dVTxMU-c*sUv=vYrU&@b?3k*<kIe$*Hxo zJ_Iu!+Z#`bhAU5w%OHfOrq9y@&3{qrH^B;;%{e1QJwd`YR4}uza%NL;RhEk*2HO_L zH1e(h($97W=?maKL@AeJs4921&=l%ytzHl@FDx`jB~WV&7Sh9g?)X;6WT7ewq>>Z+ z=tBjEVD>tx#iAV(LiBY#u~!tT^h)<bz0M6UznzgFyZY_0V^F(jV;AF`W+rQ6Ox#s3 z_q#zOtXPtKNatb=x}wLBJ$c+YO&a;DwpSI^kb-dFa^$CK3&5^3C-GIk=iS3s&El4? z{JHDo)P8XIgffb~7U2$^KmxhorLldPrlussHha+Jgqx_%J-;qUylg3(4{%CR&v=J( z)9pBCi5x=d|MX_>6~)vVKwJ$z@Y)p8a?DN%03I~cCrs={<!ok4o_=qfu)Kn%VE{cc zL-_5(J1Ufst80x~RAp370YykO^yuu%0>A#N71?AG20(6S*-rji?aA&WfN8TgD4X0o z`xBr6xxz4%5*i#k?-FuCwpwD@VM$rH;m1aXCEV$5<nRhVMl%|*j(o<hqnvHcqTH?< zcVF-kUVhQx7We|#8&w*aX}tgJ^QUW<^>a6(1ZJ@)aZ|-8ZF3o3DrMe|ISC}XAcq;N ziQN%K{@{iV${K}C5fLrCT7?NFvW3WRtGB06t(HJGhOnk%1dJdQu&z$08E0wnmFrqk z7%s?a0mW3Nw=99}_ZPQl>Xfc1+nCq^>E9|c6E=KbJs7k4!&BPj7<4(R`uTU$+Otm8 zxFI!?6ucLtJk=IML_Be*VVSqx8|LsUPtHiik3KBC#XtU(YE`178S+<ekx(1lG}d;L zJfwmXK?e6XxB(@!3G7kBJdbtTRq-f#FfABF={*6Du{{{Pv^>-koI@F|e77kRuRt$O z4{^KPnhSp$K<yRR8VQ})&`o}_)ANf;w(Ywl>p7H7*V^;rXfGn03Xst@eduU;m&5B| z=#OBuv0&_Y-l)6AmJMW@l-XtDT2BZZb~V*#HR%D_*{qtb?EV>4(MB5aup>b8NHbp* z6O^+T!95TEHui!H|F)M9{s#{VKu*VJCE5w_d4*Acw;-^#jOI%Yz#bBCHA=u^uSDF^ zcYkM$N4g>1^LGB&SEUzlZ2Q?Ry#c{e{0MF$kXLZ%0lq)^CdgS`Bw(3&jvER;R+)L2 zL~;%E`&0i<@(tRqtP(DoC2+(Q63M>tSd%p|<Z{s;52G;?sL6NX{B$HE@OU8Va7%f2 zEZGah5{A;0He7A9{)k&qjHojs&I4BN6)M`3e5zRu=%%(e`fVp@rUOY;n=1;u7m@4| z`tNvZDgn36#J@bk-+9l|5cztbW`BW4gU|8WdtK^FdUe=~>a>E5Wq}X4;;Cwv0PNMo zpZJywL(E#^!<Luc<?SL;AWcu%CdZBM(PWdY0&EeTs31TPaM-30*mx6Kz!&qrm$Ppe zdj}3$1kdjPPg!i?RsYy`U!=Oq1`u~BaGLE`>~Z(8w|(&6hZ?k@__T8A`%)We*I9A} z%Y<;FAs*|S{RNEMJaefr<b)hZSh-A+1UiPp$|PU(6V%)F9O8YXidw(j02eX~<3^67 zJsv<@LIZi30x;X0JD0nye5;(WZ;n+u{W2+dI>s{|f4H$0=l4_dSw1}4cg2qGiV?Z5 z57_+!+Cu_Edlmip9ex7eNdVyxcDAVy^d4<W4}e%W?vmerALxq(MxvKs)l_~)kngv( z&&#}eJ^OstxdNm9P;4TA`{C`u9G)U|wMM;+&0+PVP#2H^{13gm71T%o_RKNglQ}Qg zi!kFOj<ofA15$$0G}H4UUE1a?W05p?mT7gq%YJg#n`0`?+8+7{gT$ySAgaL1S{k0^ zw%0VtO@EMF&(h3V2~OPyYYXB4;+ppn{zOQC-)F-AgyMa*>)UaU=r5Z-G7R~)gdpWA zBfSVJJt(GHi$QIh^>+^DEXQns)FJ8Cc;L&Qj6N)Ph;MO>j<E>7&^V-lvbdZga->ov zJc7Fg3=OvA#jE$Ru@3-6`4GGd?>v^pZQ|WU?Ij3?SgZFB_mwHjljKDs-RjbyVKvyZ zNw@ck#$5(kMC^1ylFFu}%6#?(xVYlTMTo;iQPTlXUcx6JkBr@=l0b>%%}@D)s+2g2 zupLV_H#Ad5P#J(;|1;D;o{1!HN(SfmxX+Ghe+HreJ1>lH03h8m#v>=V=35+y$Jol1 zuEneVYbJzo8>y}NONz};&Y53;0fkkBXm582)G?l4?*&D2tk$-wbwN|<_(^vb$p7q3 zN<F~Q=W6?-h4+I=x%eBIs4CkPp`HDth0Pb9zUNd|r5jiXXCJ?2g=%9M<R|lvKB5^n z0AW1c?1KC*1cPyy)8W?$LW*Q>sM51Ah;kzQOCRs5WGa`P-l@e`OfZMUc8_e>==`Mn zxmnIV6qx%3v95qdkM2S<*Anl<IPl7e`|>IW=$8+XZ`Y|ZH-hFO#Eo(TRhTs}7OrEl z2U7ctMh_kgRW8RumOa9U_G&ywg?Oc?#}|Gs&D})<nJZxAz%(tuIf%eIZ<L<QtRJ^? z^8sf8+0#2?KFVG8PVyfDxEyf1IA@7PkwK8qjTr3R>jKq*&>xDohff?*1^Wc&L?PbU z1;IXq^x5aUyYJ{CAmpq$fn7wpE!!qK_3GvZ3tO~7_fC0N7W&ZVu`d*3nQjm&B&ZjL z4VUqo-TFW(Mi2SR_>~#x;+Df<J0%HsTrgg{(@h0Pf#|t#GbR~-{;DmNPAZomE1|e^ zI}k~hD7`gFB1r0;&A_1dEEPpzm;9hKk`=lEnR!7VmpxcX&no3Ao}A*EEnSS^M|z7f zQH2T2)Qt<#_rAejg7GLStE{0q-a|6nX52osaF!nW*1(Zm!ZY^Sh%{Gm!F=V720=Lq zxucU_t|qO`nTi|!Qavg{kJ+<ZRX~omQYVlJq47SuYiP*HuovH~mtDSc8HU})@Ad2< zm6#Qa!Ma_Bd>xI{3sozrfIstL3C_hyKK4U>1;kY%3RGi@Z<(E$RDtw}?UqWOVrOVa z9cQ7Clf95`Z;Ee!(GhyE!H<jPyZloCkR(EB2++6}Jkd4cHrUNntPG%mJa1a4aK_g0 z#7Y_mtIq|wt5iA(SD^<pV#3(Pg2O%KldCHTTLTvTOQYgx;6%ir1<0e07m~*^xi$@= z&IKD23VferwG0<m-6xzv#UCaM@NOIGBIH`@8+bGRq1pN$&K1D05%>dd@DVb@o}d&0 z*^JT44q1NHa?Rz#e~HG$s4*i@d|tj3E2~;hgzEoc{s1{M0JJcs6<LvfaEa!>2foR4 znTQJh$t;@wO3ZV;P3<2%FE}UgM(+kc_7PzIfJ3;1viS{ypH>dy;#IKgf=bf@_QN3F zrTq-V4o~j)yjGAbvo7m4P2#uA%LznKp4W$kFEuN~YnCr70#kF}!#&sk!Kv`e0$P8< zxt!Gza0iX@C2}HyA@nDV%gC%Vf03?4lgTc(k_Xzv{yv5OLdJko$Q9$4r1`(dM{{8E zQAMlTCknU#aIE+b?IiIZ|HlVWppSu1nu>{dfz85{!89}FTpHK^4p#k}BL)rb{rExV z8H00+ynv6UWC=(x|DSv+*dSmkBENU1(m?r+1*S9^96MQnJ$C<1H~WXG+yGoz8_XFf z>If2!fcv#0-14Ph6SIFZYyTOV^b?GKBMUP~;lTJ#fCZKl2Z$yAXY#ubIAo|9f@Bg* zssKWe3_g-oOBVjm@&0kxZm^VJKbOiCOrc@`ZtvBPXl=~@U!X03@_hq1&}~<%05zbY zPzJx3cWV5s`hQT#&cVk#KV~lh5u#LrKj`CjI#=;$AL!&O0L&`|VD=Hn)8(rJI|-Eg zvvuzSK=~BUc!3^IX>oa8+BWmw$0Sq){{X7{-=@HIhC}hKCK;iB_DTev#e)1^cdReB zN8|A~4@s1JG(RW-^5?VdW70BTKA2m~fBK?`t_m#NfSRmy0-Y-JnmcNMeB}yo&1TI- z2Ao6^zy5Ze9Xef4V+YzM3D=8aNBhr<)A650soicyYn8J_I70#cv}%hz4}AUE>(d$k z<F&uZx5r)uH$FSTn?K(8qaA>N0f;qBe1Gy38;QRl77|yGZOzaj4fI`enxgvd#Bpr- z0mg=`);xV4!JVfl(5@~5h#yldmgyfW<RgHGi@TiT>*Kb|Qbpg$h4A`xmStqV`y=*H z`nKrMlN)>}DoG#jLCoGBf!)m2{_f3Dyy$4&^DoKi=|#u2IJS6V+tn6PKrcsL-*{I4 z>jRB*x_oKojOc&e6hZ{>s6yyYC<oA*4g9}~Z1+pdPtzc5-|rzsa51EVO4Q6d$+j~% z7-dQYfi}2<Z){oFmaTR>q;NitPAPziL04w7le6BH@_xM%#V^$^Q~*CJoo6T#fPM>g zk*F=v@_{fM{GIYt)(|%Ol)UX&N=P^xQA(XU)C2&$$_m=f5_o!l@-_#q2QH^~*{Q^1 zbys!TjxyG)ZvwyG<*tZ}-X!(<dU+82k!LZ^qWB~_B*2Ix(8`jhgS#3L@7XP?((7?# zaldzy;F@z_)wEq~A}@;9mh~@nBOnv`b}){}HHORHjMy%C!*zQ@n0R}#C>!_WJYnHG z*%|u%x5HdhKZ;w4fC9~#JkOg&c0iU}x{wVZE6u8ustSky?OuQa10HcO*^-ICl-URs zP)9Z0cB`Kv6`CbKIk@?1XZ7_M;O2a8^MsOdyUDDL24};7YW~*gqYfn8{wy?4pciKX zXj-X!8g&5z^q&ve7H(ekbr$Ns(O0B%#sK<Dp=QK2GKyq4h>%G@Xug^l*Z+RHcg==1 zXVSAr8`2dmk|?_4Ni{~l4nyNDF?v2ZV>P-dka>`YoHRcFy?&ot@@@xEfm;0NaVk3; z#=Pt-3@NbhwcG|`@+@zSdGS+@TYv-^nf=8;fc%mL@bx8sNUGGQvenDzaJ_opjzMep z9Y$^Qw;IcQs5Kc>Rgly~kw`oaIDR@^njZ!+%y*m#-Lgt0DvIZSAeIWY4xvSbzu;fb zyLw$u;bw8!T?<QTUZ0-!T<1J<9d{qwOHbKz?erQRht2;oJf@g0ZDJGh{h{ZKxkJu< zbj<eG_oq4jslpzu$2HkaB@o6ZzT>4@1^DRpdKBKT2bgH=w%TRh-aH4($53>MQ5qeb z8x`;`F%118kfZ{Czh1?C{8IQU5rDsSOA+cr*LmB%(gCiVvmf<63u(Wq$FLH}#j#!$ z!UxoEVOVi!*LkaRd$*Kla;u~Uy^njY!mOiD=`aAXeH|pP(k5WY;HM~xA^@PW17_2h zqRc_-8_dPQY&Iv*uZd&~EmDQ8{_<UvE5bugtp|@rSN{>LPt@b>>hep-Ddr#-C&(P= zQ_Jdjf6-Ni*>#<r-4xclt5{l5YEQ(z*P|H!SV9Hj!F1lnhDyg%#4YsR0mROXxysT5 z7Bk0c%dfU)2X&iaOs;j#Kr1Y+v+57o4fow_4^nwfI!qlfmj(&;#s(&$=DB)ng{(Q$ z2?PZWxhxBfM-#?RCJ2Qx_&pW%V-l2Wx_OiS-0Lt6l34&E=)0Uz2H3knI?$hu-DL7r z7Su;!Z94TvAV3Inlu~%t46pCQX5cdX&4+(_0f<uvy4X^S-JQh%Egsyt-HFUI<8ODW z?+rJ8)D^{p3^4|;F0H1VyVQd+U2GqfO_7kNRn@EeggD#o+-(Xpat&ol=E~cJ-Ru4k zqnp0?;*>KQL@X+X6fk^fVSwP04kQrwf&SOaFA@T#E?Y}cHW>OjRiF#bl>N1Z9JXUU zXO*#T$93`zHEk%6mO!tP767NsvYL>`e=>QIaao#d;GDz;Cq*~(VX9F^`zb=~=#%<B zZ8{dco}K#6IToC~({hkxZ2lnbVFtf;^LN3PS$Z1#3jc&OD${rNNO%BE#@~ucB%2;2 zIeg;%8Z$YR+oz<#;~L;`F-UA<6X1{v$%K;lrCMH-ZB;?t;Uiw;@j9IntaF|A^91Z^ zkWYf1hV6r<hfXUXz>`WzMY_?%xf5*Fe=Ash6l9_uT7<QH9P?t_Z)Qb|A@Ab-QXs7W z<jT9xCyv)cnC;#QhRL@40difeq}Y}8N;P##W~xCClqi&wu)S>VzKAv{sS!M<WbW=y z52rSI`%y4!w6;|rn{?k>>xlJ#wVHK_?+E0u1uzSx9;+?5MAbAkhDmpf9xQ0)54|tK z{3-GBscb(z73B5gy~*#+h}|3P5^at6Vre<Ak`Lj;yf;kCMc1iyB>1eCd3&%l1phsc zNP&D0%4C@J)URJiH^!kMzc4z&qJGr<xwPI$dB|hv7Vnox@}>)Vr(<_Wvj3sEF9Xf@ zk*L1Q4Z-(-=$IpnYV2;S4izVT`L!gVzEV=$(M;5|Jd&`GOztM2tkNk1b>uupq(C!P zN~7Rp9i$V5nv1!DOZ3HL^ro%L&|*6D;gOHz!;;P|p@^|^_pTg1FYQ%7^5jVmCUsfS zAKKY(qMu*C*Ev^0v#581D%3^7zUL)+=Jbk+-F5F4=*{N2wR$-Dp6^s9gSwnZKYY(q zh9OJMNC2|NkF>gI&S<Knh%h+*#G>_uLs&F+2_08Mt$UN4sg{v2v@2+nS#nU6naWLl zG(!QUqhmDTgt6b)Z~uDr*AIHxt`yMceVo=!nRx1e(EyUh;}i6`Tt=PR7rm3Os$q`E z403yIm!F{6d*8e(xon73V%MKTXf)b^K_h(g&zJ88kw7=?_qkXWB3}VQlykquB1F?e zf!FGDzp{cV7@=;TIS`0dW&FYsw471I7j8dm_sBdf1e?1U8Bs>oF{B%8otQZ^2O0+5 zD_05IWh6}0|4^9^9vh7}SJFKUxMrT6n^cz(B}e$wU2fi7s#vFjdE~`n8*H+3#MXEy z=4_J#brd9{Ln}u!uL!*;rjvta@~!&&Gzz_6C+<?w>iPsdc__Qpo+RVXy3r=oZA2qh zn2*Yysmjv?M3z0JV>(Pd$XoeN1l(F%{c_o9J?)sb9o5FiSbV=~(X_a|1HT?*5QCO) zW{#|TMb%+?Jf}W1Nu0jy(%|{ik4xRrVUB`|@gr!+GFkO^L*eF(_g%JQo2iv{KiS;8 zHTX~&DjvT<RY`CvY&r*TfLDJW#9=;wbyRYX$li2HtkM13Y2ED;eN_wueDptG(C}{% zOa*JB)8E2F!}<Dr9C7xlku_5kc}ES04|rPM!gH4^6SfgT86@=C6c{Hg#jNq0%|I-6 zI3#=|k(@UtOmX~uOTC+Nu+Xa@EdK{nm&ULZgt2fxfkIUoi>++u#qXcvq>w?I%qXoD z0#ean<qcs3=7Mdbcyl>%jbM;L9NFF#gN6=63bQ0zg*M@rU}0^>URdFWY8iHnqZ6AT z4$ZlUNuCSmlNET=>QpF(59+Zd!&iQ_hg@X9WSw&wz{1wp@FscI$D^6C$~pT<&?F+O zKwok;Z$WNi#aer~Sqazp3vja-be}yscpL*_fod>5>Q&1-Iy=_eM)tuQ=5>dn5vvz% zQp<a<J=42CS*{|yQ40=C=I_A#l|W+}!<?XvV7(7r*>%;Agt~&LJL>j6<QxAJB0D;w z{qax*KHZjfk0IH=FKDzt7izZtM|-KhH*o?2=y2-YBXH#QL`O`vmE%IcPP7`o-46bA zcd?lNWhrwO#t>6MZm3Fh&gIvZ<|m!>W8G6vfAWI{6Qs%$QC$4f&akSZ5%!MkZ{J}` zZLz0ubyJx9M%#uAtnLxc_X#TeZi&XO^i@v%fJKb)_EdACejsAs$;@Y=>Q}V{t*s-= zNUF^5Q_5lw`@>7k=p*O$?rbtFSjMQHjg~eU7pS=(d+A($!SRpPoJo5o^qs^9Kjt_+ zb!05plO^gf{(McDX^g|zd-qWvCM;h0qdA6Cpxn_eWv2G?3p+NNq8_6Ds2frOq+J`H zg&azEbhSaoP(#RZQs$ONx8eIlNAXCX0JKbPrcr;SEU{4=<;FznknT#j5P}g7SJHn? z=Qn6@^pGEbm}y>49JF&6$Tw8Uc--fC0WQ|%i#@gPRCPN(7_+9c<rg)O`f(GikZkuk zh5PT6@b|89)t<pF{#l(r{Bv@-Jh<KR@Z^+A9hU#J{ne#&aDS27I-f^xU#4s^W4STk z>4AOqZSC1~Q|ZS8^2)`{Nyq`TzI4b&ts7l9Q>+h=M?Z5B=u_9dK@w5uhA6!f>hz+Z z`xfU+XqNCdRt{fGXn4JE>+@y)gc}dU>Ub?QNFbVy*;b7iMZ#~wdJ-3Y>T^90%6No3 zqK|j>?~^)(tUA0Zer|Sqh<$f7Ax=S3yjmIbans|Vzt`<2ZyetXxllQ~`bVLzAB0O( z9;G~)JU`S{pAZ{~P7aTZ5sj3I!$l-H%!vN;8rg!ck-H}+H`+Nn<Qt0Q3{EfQFcW(@ z7TbUhfgd-c{YGyOX_FW*UBx~*9h@onTo#@|Nm61j*G0S+;s<I!{9o*G%@drrA2XrU z#G6EJGa5uP-N@^1%$t7SSm_F94n<1<f@90sR<4?W8gm5G+m0%|R%*vDH%aZ>4dO@` zBAv-!eZwU~hQQ<K{pqXf>&so_Z_nmx;V%l|R^gh6!D>2Uqce$0r+uH4dZOCBjVo5F zzeh$oeVlD_+Xu+!R?ID0G$-`OArCHT?36A;30wz0bw5Y~Hs8-mnjbY@lr=~^L&I0( zoUUA#jC{f`yi*n<OLUbaHCVk~59zcwddIStuxa+sMF1Ez|9wTsl6CG+>KmGTVWJ#q z>CLLEqy4xwJJi<fQL>~91)KQTPMFlUenp*mlr4{=5cb`?kNNJfE#W`rNf(A|GzfJ! z5hKOs`e8%Ag4IUe#U|u!*k^a9kNiNnP-!HQVuk*b%W{~^4zgBEjq&Gw^vxAuBDPb( zoG^;=P2gdNrC(xnPN%?pk-Fve?8ti%7;t0`vHfI%J%7TO=42Ys_2$N$(=yq2qr&sy z^zA|&6r37}mq9lyDSS;u#^XtPT$GLTO&41!TEw_{G|ycT&Y7Wq5@g8)iziF#OuC(( zC%mvr4I{f%8bg+hC>KcnJij!OipPs1mZ82iDihE$UKh#z0XTe6Voq~4ltmVq$>Juw z(pgF*rn^Er9L18#n6e6#^Y*SdI$yC8lhVSpQic5lI`isx-yJU*o11Z>B%02DPh$sY zFs~{Y^8Wit28{_U4c(y3bR_tDZeJhN5klf?D_tcm{WsLi5L}o|i7J*0nrdkq&%vY6 zQY7&<M{3kZwStd4jYGQ0U8W9sG_wT8(H^^txVuIHB;{ZAGoY-h)It+t147V;Bgf~= z8>QiJP&jxn;KXu}XrwZe^x_(z`nW#-a3^9m!RM}4LnO3cTxm5bYhWIF=ozN&n}L>! zcW9ctC2H|Y2`LDq->gI++D-q;xmxR-;xz=6@|uu{B1FN$8DcBrlomLFKTE2t%sLl? zxySq#FMdkKBUlutee^0S|4QSbYErmTx+vIm%<JmQ6T_C#UIw=}iQz?<eOs<4XlChg z`pYl2QQx51>8jA9!y~idu(owIRowvPg%MNEWVc%W;db~*gI(vuaV6}#>6haSwx%GD z&Kt*!^i2~Cemad%<;$aS*!qRwhFg(0Mc$mmP?|6N^l4?Uh}1P7n2uHwf(A!dv1E%M zy2<L~FQzoxoaZBN{<Q}K-gFE5_-UM%lbH!Y0!0q64J%7xkWA7n;AQ?%JY%p=x@x-o zC|7?(R*U%wb<{ZYdPbuEoGO+??@gnO{$j1$sq$heG-=b1T*3ym!68tA@hmb4M*Rf- z#en`Bc8?1)G=48gWsRMlMI~k#4RIpJA}W*${gU`cs&Pn^rL=s=Q@__cEN(I8odJ`U zk46q=E_OfQ0?X5HnSAJKO_y{;g{d*8-;tNE*@m#I_5<U4e$2TnS>S2R4|_>JT}g#) zc-T9=(bjiXIpa15pCu%<&%I%N-0gt=yC=x>q#$63*&<XzRUi4$duIz~6ydbAP(Zq3 zuFYQ<4Ne`S#|`<T&TT&8F8<SGZ&}?uXI7Nn3kc+Ly4qT{Dqw(ZZsyDCuVVN(GOpQ} z_AXoXM;Ll=pNZufCW(&rT+K-|-L-+e<Bi9RjViVtX6Z;(L@V-)#K#qo(SMBWXn}JS zV2$Rn8cc}>YDhS&Zdz(aRJkFI=uk=i(^Fp*=%mj4L8TU}ty+A)<)w$|0;fNd?{h;f z-o5#KH%$TaLPBGrN!HaHg5Iy~WiA%%Z}0sJE1WgXl>>6hrQ{}4ghDpyfTl&vn4Y&P z;ETGd;DjT-)wid+T`%=CL1>v++GbM8XY~NBaPdzm!?po9p5;wmH*yb0hZw$PTX`EB z+xBG4wx^FQa}gUBF2YXpng2C7eC2?_u~9{2CIIOxLRbvL;;jQ0EDy{fi8VFdB=Xg> z*5Jn@_0@=B<JNG=+b=GjZ>@MmauRnCXxRG|r{u{}^h5A{9gAee)u#^)8}TD_quSE` zy^9nFu7^A$Vt~pPmIUE<;H|re)kDEoD*rHZ;}l$i$#Y^)+O39B{6#r{$x9CGio@$Y zt!*LLmnXic^l}OcTAQL}G?TzX8!p^r)o4?s2g~ZUCH?2&?ZJmn2W^Q<5Mupx__F)6 z4DQS6J8n`Cf^-&<)veg)&|>~u_5?TxrZJRn8NXP3alCbu25}npMKGc$<Tjg`s@zC@ zda=ZzW4<(}zumg)+y4DM=0Copiv)Ob;a{8+!$%9eLkI5kPk}|ee?Qp%E`9|@!2i0E zUL=8!_HhRm*p<3XjDN2%|J-u$1N)5;xsQH734PUtfGba{VJ;i@A7A#@4jeKC*n11; z@<wJrLq-8_yR=v$lmGWahIqhf)z}Vl8lZj82eya2iq9S#|KDl(umk^c6%MfhHui?0 zz_F5v2fhBa$^X|Id@SH#$nWU5$={&a3j(MOB4s)8f0PXWIQ?@5;CDS+@p61ZgL4Mg z5S~9c`TzGf!a;)nqmRl7g9b+kyoEHc;ZXkH4+U2Mr}b+tVXsYMV-lD>x8_a58QhIK zK!AM=;AAOLN-@ost8jn2;C^>HIL`k{NR=s0CucUYwLDr=d{bf+1gwW5d`yoo*HvI3 zN+OQWC_Fl}!pce2v#5(H`ECyglK#EbTo()ADzO}c+f2233-JHGIM9&5&-JaA#07>H zGVnIaoDPgG@g625C9P8v$+*4q63IJ1sp)@zoNyi~LwaE(VSJw(fyOT=w9XX_Q*pM| zPD3}`L7q~(7mL5W`<-<g(M(RlJdaxOZ`45TPGpX<p;G0JYv(t372n@UBN6^kJzu5M zObeb{zU*OA?eE*6obhr0<DT710$0AOcOes6oRABA-?G{Bbvz~Vdvk7u$=%E`KvI(! zBVOFD+J(pmR176ayS1QCVm2tYr%MmNxE7nfRVtUSOJ;iCAeCYC@igi1N+ca<jFQTx zbIgInJkgY_Fjl>CiIZjC9Tn5wbtH8=&JP$xQAD7|zqY9|F$=P1^uFzWr}W2qvA@>( zQT^a=01F98HSVALyJ#Rp3K-?uM}v#eKApgT-!=G=n&hPtTOdsa_(y#I`xIsIIZ6GO z<`RU+hQ$444HR$!(ptg$F{#4v*p?Ib!K1T87jvh(b#F4;w(swqUeAdN=crp=?y3K( z)@zrnXnzf<C4Hjf`b8<}ty6}Tk*{P{Bg)w7_K^FnL`5^xy=DUn^*W>a5}0fMJhgim z;68fQyu*ct@BIY%h9r4&=TCkp(oJ&cVjP<=Ij75EKsu+FNTvsuKX{j;%UJ)gv%%N= z+<&%&K<&9}wi?Q<#-JB##D5YKZ!smgd)$n-q|<TU`s{jujEy!W&`J>aE;IXSJo$t( z1e3o5#967`Mi`v(ZM&*0R#!uZ_eR<@=tx!A$!`zu#iFs6P?CI{IxPBVzOuz>P$WFJ zjv^Ai@4~p{T`VBvm59G*D&yw4n$JQW0PrZ%@!=|o2d>d?BjtcNRm;Y9y`=yP53{#V zDE+gxRAFQ$LT_fA*!v-%)<qDhn_~Sm5LSgTdB^|o&8^wxtn#@ufUC(k6-yyd1+ZhD zy>b7l+-$Sf?kHBf>$T>F-`lOH(g}}7HdRg})IzP^sTYMGPvfl}ZSATX{kuf5)pA$f zER$eSGv{pDvNK)nG%6*bHQ%wE(e5e4dejqRM0Et00Vi=JCUR4WRICGXZl$`f@wJ&s z<Lhn=|12N3^SPKr?C0jK%UsQ)r^;%eKFOWuC<yfpYPVP?shmT2I($Be<1*E)jh7op zK^MEZ+YhOiliI~?&O0`=I>U5730O+If4{_;ht!eQX!}~xA9}A<8Oq=e;SqVfOy^>| zK9#PRFH15BfK<oHj}al(S-@>nwq1V|%T!&*@yL>X*gM04JW^=l=D)IjuXJ+BQD}z4 zdH+-P1V7`mt7MXmy2QVMv~CgLk;_;?z9$6vDhE_TEO4?qmhBV|2p#kxv7|9)jb65s zqPW%65z9kO6DTjW7WBV8Hcc#6?czzL;}?_%Z_2!fdOI?pnKGrj5v@R`c)ueQ?<z`E zM^k9C=#wqzwfb|e_*CAUWhpbXG#c-Sp_>5c*&J&+PeDLi1604nVbPSvjX8ILH|Aoe zRjUeYcd=z0N(#mrh0wdY6CAbSbcL6ShTZ(t?2FjeuCmANiCd(d`<$2ESiY@A&mxA| zy(Tsz^WtZR({SOmzhMZ@nMMBRyxwr<gwUn3>*g9SA?xB>l`zC6XDluWJ*~f?(~`;B zIsk>WT}3Rcc#4|a4me6~uYZR_#PGpwSeI;ht}4DjNo-h#3@aJZp;%=-Mv_v}w3HZ^ zM2lvn+Uex02A@&6@VdIcssRaI4Qs7en~tO2U%2;y)2F*ZHO=eK=*Pfz_}ObWf5viE zW;cq!u=EBWQGvckswgt{taHhD)`(B%i*#dvfSMKP>4?E($fr`V&{TNwX;hVrg8+0I zw6O>P*AfZ4RApn!=+kHJ1T#rGT6~TRE(DZioY%a~dS<Y%m0O^PT3GE0wNX_a<Com1 zT-=|u<X%Q?s}#e9Q3|#RSNZmXU*9_nd<-X`)|#V_L%7j2?_Qezq+fZDXE_7vY*Q?c zFDh*N^6vp52&{D)^0Xurf{<^72{)iwqJMA%BzY+?)vJ!BW|wP>S<xV=XC{<i=9H@5 z5Xek1oX#eosJ0m*wG&p0V6WzZa92CUbUj<+e|xtlH*9Hkes3U*X6vowVZfMDWykd5 zCP9C!pT7ega!YRYMv%B7e3(i!*P5eEfI$zmD64Rxo}EfmLO~+jBFaSBLP&w6sm9By zm=xStr&f{(b6K~3RMra2j^N$rhYo+GvpQ%7(~uQuh6&WIZM2!yT-YrSFuD3Ldim5Z zx@^$4R0gR*pknAU+P3Pu;=7JaWRVbSSG^fkdz8Cl$Ish!1O^6I(|79wnJ0Qf^Ej%5 z%R@4Z<HJA5Y#z<(Z_;T{``}FXivkQ|q?&^TaiK|qO;Nh4fmSRpL(r{<Sx7kgyjNZw z_7-0~K~=A8FdxdbBHu&&bG1#}E__lQ8&se8^F92;XCdE$oFdRt`w3zSb9LG&&bGhF z%H&Jy&ehJ<&AnTAS{dy}mcE75C*ovm-#!&T2$E#q%_pZcDpa<1kCFf|nswj+B~tLk zhBTqJ)Q=D@kl`aeHJ#f!CY?@20&0UjL+?QDPU(Q2x9+4BjKll&kU$;cP>}gq$-<N9 zNGrRfiu`(eUG29eb5ESz=(~jZf{Ep`K{69oivSf;(XJH*k430^zD@TZFMh1Q&br@# z>^e0Yd)#!D{a({<1Sk|<A}($l-^`i#1NqXBov1$}-08~^TEo*f^>0ckcCYt*YUy#x z%11&I%cxNhHU3pVi4!is1C{74#_qFtS0@~FzCN4zEwwKWHUe~j!m8%s6>1oNQhyxT z{2u?=li(L;i&H{KkEb2C6wdO-$S0LfBLcee9Htc**v5*Y)s~is2l4eN{?jcD!GPxM z@<u|4E~IxbhD!5GCiRr8Zi`8=>dlctBk)$5I;~wd>E|_AjZ|<hBr^;8@TQm}gnmT# zKzmQGmNskRZp0u5@RP-%6-0JAHB9bK_#pVSDfBRvMmC|0seb0cV;bF{>3b7ev_&77 z^4AC$V>tQA%|6>h+FN_+l%^Ghvk3Z;X|yo&D^ZJLWuMS3P&U0vRQz+G)aHyJFLZfC zw1v~wR`052N=Jr&-p>|TRlaKKa`dU;!$}?fwaR7&KkOCTCr~lfk98`Vn;EZqOp;_G zf9hK}gq0h>6}oWzM0z82p}`Q9E+0E*U{({`0MT|N2Yv4#;N$)~dLPd;&kg%duizwa z=}!Bn6F%n#G9a**U1rXMM<&eCo-+c6z4FxI<QW^-r+z3|tt@JL(1p=4n;T5yMajhy z2tFyNJtRa+Gc0c)oWOp+o60WMs{e6z9n0<x9!S$}PDTcLH?0Z6Jh-LWT}0YoBnxpS zCIUOxxauti=zN{$o4WC+YsSB8Z^%rbhW61DlJ$Xx3>D}CVC0VKFirA%(5egy4TM!G zfC)?3nlR1DYDDJHD$sjZ*IQ0|ha0OUs@*~wAm)zV&+2J0JfHTWKY0c(!*p*IepL&= z3cy@@Wl6M{=V%&{!I{xnQ)B8InzJ<Sl+lux!CP1rWOOvo=IcV(*^R(~MC1r>)eTx( z`hf@rvBx~0%eF!UwFRCj)km7q%+k+7LiJ5C?I+fAC4TM<tArK&tRB>a_&n~Tg8RsB z*bm~L<(j1|_$9e#o$u~!J!xAO(;S;?r)Fhx>vOdx;9hN`v`KeDzhE{*tbDLlO5iw( z%Uc1yhrJ5NDK6CWQ}iu`-nbF^Z#Q8fqp1qlI^-OX{4fKmB(hFtgsLgJ%fJ6X&Ud?t zER^!>b%AN?t6VwU=sBJk7=;Dy`fi3!_>b?<M|^uPon}v_Ddj8iqb-lMuqfjdRwuRZ zZM#j?QNmii)X9f>vQS5IuW*d|{PK!C4VSPRhm@(`<9T^+ENLIVSnk&LlWg@k6>AD1 z#%72~Uo9kJO8l#H`PViOF;L|<t~jAR9CW(=+a!i2@l-?IuNg;n&jq^VWeN5uBZ2j0 zRXW(C?Uj-36m-!5_4_~yxF_myHriZjA-Z}4|BtV~4y&qL`#@nu5D+D$q@}wAln!YS z>F(~5kd*Fj5Ksw`Saf#_EJ~ysq`Mo=So?85`tA2S=f8PfYhH7Wnd82HHF~mwp1(7M zXU8)Aranlfu+R?{(nbH-sC)Db1HN2aQ($B_8fkS$s{tFzlKVu;=*oN?HK;~@eNPD4 z1f7-aMX~|IU|mGUiXLONN7u{#0VET<5yIKVgxda<gO%rd5^=kJS~~0rkg?5$U0dUD zht>c~S_AV}xa9nkqR&-t7Wi$)XK(@3FAkW8eE88W)OL2mw1!>H;?2cglq4Nnc=nD# zKM7OjMWVFz8ZSc?D|-oIJ=^8o86=Z|o778CznH6<JKh2C-AGssQJUq*7!8l>wTtDW zE%x|zwd7ZKiRXs|%RBtTeoEd$UTQ&NmoCUQW$A%(^1f@H(!3*Gv%}jQ?$^RDU~zyy zlVz(Qta>VZz6PKI5mFUolP)6kF`e(2X3XOdtshQyhB6tOw;V1@$c0^ce1UjU2y?EQ zo#(yThQTR%H)1U8W8_b`&k}^eLZ2k2wLVkYZ2B<>eGE^3nhwN>$5Ke=8hk1mH3;PL zodl~gY;0$&KR9T1Xxfh*a0p?D#*1o3HW63O{jr#yYEUy(Czg4Awl>dGov;v882XBw z0=cBRy|$o|jrH@#S*HHRu6&EO%^ekIMb(>6jn$SmuOF-0_dp%{hml10REo$GlzG|Q zLu7qZnB7|{oGNnzJ>O}wWsxVpua|>rv`am<e6q!VozXHiTw{~Qxf@wR7$CTAg|P<V z+xqK48-jiCkFkH9OOlo#b<njw#$O?Q7VtQOwDzzWb3;mq*WT-bGtn{+O}D0%to(DS z9=q>klWp^uO*8~P22^|BtdbuPGC<Mc3CHwL_15uzQI}Kq)CH7l)fsUkTIjMmHdv{# ziEsCy4h5A$4@6nYyOV`UAT!qt(q%Ln$+Hdi+QRt_MAp@8;O#Z=s>LjZ0Fa%ipc!ki z7IJxD+#g}KyqThB#!S`I4jQhcx}7@`@h8>>aeAwL#4I+!!AXIxrl*Hj8@f4iaTXX& z<)@q(1GDO}wFO*PZ8e^0gLBuM$@a<Fr072}+=mD(r5yKij@FcXqYhDfh@l+Be_aVY z*p=u~Uim1|;*AWh#CErB6|7D!bX5<z4SRxcV00Os;jpzTOPL!{{h9?!%VWJkj<OR= zZd3V%eXQOSHV|er%J~^mO-;fA(AaAKho^e0!SxSuxz!RT7@^@3nB0;G&4aPpXJJW8 z`;ie&9^~}_lPOqqStemu_;6Qmbn~Z|28E43SE;M#lUWkO9b}Q}jK|+<J5dSS!k05z zCiVCph~y+Y7#He08hfDo;&++OAQOPn5^zzkf$B%8#ClfheSwTI(Z~6p_1twD^cC!$ zBGJAq#=V96{2HGGsfMMVXr@_Mmp<rA&x1{BDU&32osN-0#;0vr?|NQavDQ-e3bO`? za|tcG;VUP}v~X?x>L+HgiY~mLXhLIWO;~JjOs1Jz{yoCeeeKjbJ-FRN_eA!M_YAV( z?q-Cb2|T;57s0++$9mdKarx~NvqG)N?o*3`@-=RDyJxv4!yPU33RPvGxbDwi`y0rh z2VHM#UD-!m<a25tHOgc%q%#<MgQt&FcC)nANue|2dbqQCqR}8sNZS#|)@kJSTBX&4 z&RGK?VdtQ<J1(>o-VYmNrkd&6S;MsS#TkC<<4mq%o#!ajfm=!ExG1;@=g64fqj;Ks z$fhH(8xjSUw*L5aTBKR#IPz76kgZ<NWTtco13t;4xB<ehw+rM0T|t5Cw<t*r)IAAX zv>8Dk_G#vjOoGn5irJ#(2D4LiS?Sa&vT@3Y<MkdMFgJ)8@(p?*=q80vX2vOsqgnc( zEr}=hK1)7qw#s(HWV)@<r0)nN7;E~x%LBUPZ=`pFy%-5X*ohu$0u+=j>@xP_QPq~a zVS0f#6FFn<ZhMon!mK#tkND_Xp+<8ZYF(aX{g{CVleHcGu*Y7IQ(evRi(Yeff3$q} zmY-h0UO>4Xr&5ss+YjOR4wp`Cd+PAo0v8<UP;vDY{QS)j5E!}!c{Qk*PB+?}#`|;T zf9XcUm<c}K?Yg=+ynt1i=_{j1LAz6B0JHh;9OUZ*Y6Lws(E%TLxFDEx#<le|l-A`S zK!@fy5p-SpF<cPMuFG~U!7&N=W$_j#ql(u)m1>iz5!rdYZ*ZyK5NZbr*rjN4dxlqC zwwM|^`Qsj?w7a1c_P*WsMWxTt{VDwl9&*221?e@X;==P<&vv@4_sJso*y7e^5+aBw z<vesIxMdPH>POyTtZGq4-%~|2Jvi2%QUyrGv(9eHq-A=%kUX<QyS^E%!!EO$!n<LM z><TtMokKivy62&~>#Y)0g+)I1xmPU}fBcpEm}K0^5AC5lV#76wT*BJI4B2L?_2yc; z?G~LKCKN6e#gbg*XVnQXL00|@@oW*c%?CBOBo%TcgGRCny?&6kyfgH*9`uYy0L+sL zV=I17q}p3k;S5v@orQhY5uHYj$0{uB!V$=qdlp49vG+}-S$KcK-*6$Lhu7yFo%KKa zw9}#8EGAI^)JySsY9n>+!F5#9xwQ*@#4Q>&YP{mzr|mZrcPFD~hFwE%FQnDGCjD*+ zwnR+jfAEeI8ZL2lO?$F=N-$4W8fR5KZg-h^igh+*^WFmTiX_JT>2?rJD<hlG<b{qG z06%}@v_66qKo^3p8evG~(?W^P;ydXl_*hk@#z9E%KxyGea4f0*^B|LwwpxI+TO(s~ z>F@?wLiiT~2$}Rl;&^vA%ansywQF5<5Vz!#X-bv%cHx&>j+VRS7KY%Nypez2NXAk* z=AErPxQxss{oE0;rM=3pd+9ui<GZR%p(hceATX{>Am_9@_UP=BZ4JH~IaAG6_c+Jd z3l%N)6y9`g5%T&yAzdy5)riFOoMg;c5|aaCw3aWx-bW=Lbo4U936Fs?EeO@I>AHo# z`<UOJIPU`S{=;h*s`kNuiJ?{**lb?Rv?d03D^99{VscweNGNYbey>`_2pKTc*&;Fx z;{EzJOOs^|WAMmnb!r&{$t&_`)V+gTDGHX=TV%FPs;s>?=hG3>W8Pn8`!vr;ii#+2 ztUU<P{=jm)NB{yRhF3lXD9i$ol(D-QsmZ~ih1^^E)|yl1kj3d6o`3r`y3zj5MOXUV z_sK2GN17;oAkcz$EbYb)cQEyGATQiRzk57pTyz<E7$<L!_wN}{>@R?c$9+Hi0v?mp z=MKP;r`|Zc<UZnJ3*T>Nr;J~y-l`H$;%%dedGM<>U-Y?d8y>9?_Arr3;LzzzB~4$~ zF>2LE5naod(X}#&QuMG~yFDi){4@4*n0zs+W#S`%V>JLAD~dv5`{xn)@3Z2H2Y%vv z-0SfW0RJ#_KYHPB_TZn7LW?aJ>Ctq~Vv7VR4Q#dqi$iDs`@#6VL0SVNGax6wtR!0I zVf&w9FM%=q`LO)6?eB}vbssspr5!w?+xkCN%m42#Utl;_e8~Y7xGpr90I?OdYl!&Y z<47zl0x<V<U+qW|T+-)Hp*1M~_ej^E-(hdoeU2Ip?+c?3K54vp6ZpSJ@{I$(ENH0? zgu-7Zfc?F-@gu|jcPELK=iuT9w_<@Y1%XO(Qse_v{9pL?-;LKW!ejklc@~VR35;)* zpxCVX_Gk&A_H+%qCz_)k@MP3eH>D3SgXw(Iyyv4~$otge6qx*z*V%F7+SUPNUG_v% z)^|kU4jO7|H8`)7eSCQx_seTTIV6)*KYPcpuJwQD0o2*~n={>4j!StIau@#D66sxw z&jkJ*64*Fr2J`G~JtrYPGCaA@ho{hibZ#<Thz0|Q)Jw3C6{gU+KEGee>9RYMc-wE; z3t;sGNR}yhtRNh_^KXcgDvwORmu8;kT~Pc7cV*)s=v5pvs+xk|lw=uv(gyA7bcFB5 z#I}6+)2*L~Pq1mOL@IFLN=bqDAiH~q!**Nm0w{@)(_iX_H4+mPgc-V3#rO$q$6LsX zWCP`13`f+#7W%d@fl4f#El!dfksHsk_ud>I{@k~3F#H<4yz4PYZm18I5dZsugFU^) z;Bh;8ghc=pTpj2>AAqFCHwxENQH*(>A%sNQfdBHu9>AAnMw@0EyauV59Xb_Vj|$59 zW%H34yr5dp_s#fyl<)JY3#)vbF`<q;&I{r|gAk;3yW9ZQX1nQ@nDY&IU=Z@=+Cvbu zTg=#7)$wP*ceRlWKxO_EKFKl0kCaidJ*Iv9iN+({QseUMl$H<=R7YMf*XGAh%xOTR zb&N~Fn0-wb+Micm5bjnIyp`S%6me0wRw)>p4d}eMTEmCEK$9s^7XR(nslc6?5A9p{ za?QJ$_$utM?~1)QMSI6Ns8{9&YT_66YpRD{>A4DJQr12Q&@M{HgwDx#J$G)b=EU64 z&z_{7NL{t9IkvN3qKr;vXx(T_azVi8g+wQr?peo#j`d0l0MLLuz-=XQy+AMU;6ohO zZdz01?DQ$158`3ERYzu=c)-_))fCy5qrs0w8OE<?&<MUhLyFTw`15h;cn&^{2Ezu9 z`4kI?FiGb$;BWvGEMBiRYBMb&=4{Pm8cBi;%GdJd+vn(oWxt^0C9d4R;r6OSspx$# zkDQTa!WhS|k<z`?BT`*fgSFus{&;W9Ty}azcifVgj><80YvtS&Pxq#nmPJIefq;K% z^v8^&>mC#SXi-jfU%YaCJ-*L<g9ke6WF7AwWztO^t#DPpe%VH}4^aD9sR%yvP$jsn z&wbw4Oey7Su`f2*H&4xu`dU3fH9zm>6}<B+UyIxO%^MPd=PrA53GeXBU-xwmWv)KV z&<kn2mo)L%V&}`V=FO$OX34=z?Bi694Soq{k0vpGP0m63*6qW9`AT}yF@m=B;Ro(3 zyXic!Fj2Nl5z(Wug>iC3N}(^WEuJ`9`b@kpD#;wocu55iRYtjNPmLd@1G#G}GXtZh z!GlEk3DF;!8yJWJ--Hgq7w%t&(b?b|B<dFeMe5brBSU_OL1K=IVb7tKPVJW_Q`PvD z*%1pgvcL(LW?PrendH&|=PN{~DSP!g%^5`&K+u-FcX^GE==tNVLZgm6@z#0_gH~8a zy{8`IV@K&8&-KR?e=zaA3b$O}7bT!HKa9^S>%HtOH(n`3{=sJD-|%*wu^@x`|MQvk zocAiyX&da*%%R_?T%RvX9QRev>{6~&Qz&raN)hl1u1qX>{762L`d|FK-M6SOcq4Jb z9|b5CV*;?+i#T7l_Ibw<`z5Fy6pAz!prKGbj$^b;l=`m>QWibTdN_NLdq80^cBRnm zqm^c}0zWakghkcnbJ0F!ry7&HlHXALRHY3}1N~AT;yv^wj(F{tS9+yr#y4i=B8Za3 z#FR(;boK4<kzZdyV@q3+&DC2<7OHYpcc)!uM?VP!ess*89IYKHsRk<wdu0Y*-A4wx zt!;&6OGzy91}mprU6!p@2fr{5JE_B9Mg!C<WJnDlArhB^PA$2FuUtPOq?2E{^!@ai zeW&L+f_gT-H=}xp%VYtH$<pz!Pc|V8&(b6(H6jS7c4kXOQAX^LIdk@Tj71S<;s97~ zAFOZEq-tlW#c%l0MyJrj>D%Yinb4GF<M!BBKUh5gMyJ7Un_76`n)=gfRyj1W$e?aa zoV=?w7V~OnuCboYd*&?>d;eUC#ANSfy`I6HklB{P7GIC*z7zH!OXLC<9Kw7L;DkR9 zju$wAV7J%izj*`*yLTjQDO-af)=k=>{`78cs`%gERsl*F&yO3}{@jNYRzIe&-{m2h zpkvP!Ps=C>B<M6LeL-OOBm*sx+RPD1=uNH$EQ=4vM3|JT+3;19;6wT((C4o7d}Wt7 zSta^)VANy0$E8_0ZA4|<IchjVxV9*IZq*S0bY`!D<-{bmUTCSutT<IX=K)fGW6v?+ zP}OrmzG_2%e%Sy(oLyX~&PT(L@h_9=>fF{9HomV`Jy)D)k5AnlBtyi-&f9BPy`BaW zpuZyJC(Xq!bSK?!RsN~2HNxe3(5<^_GJ`f>q#~~4mQ_>sk(;_eDV$O5BaG5uKQiF9 zNqa5IB4UaZm?>S0V$kfvF@`LR9m{6~#N&ihN9m-yM7<rA^Je;JZ6;}t&Wb*h9vXRY ze*MVnHi7Bej%!oGFnS!pbc;7U@P4;MZEO5XZqsslZ)Lo);eBKIi*gT0;ad0y6SPa) z(-$@$kP_v0a!i#@RRM&BVzM^)F?KGZz)V_FO(ylQxD2wTc0+W->ai&1(+Lv7k+g$( zRDh|)^N9zkYhux;T0K+N4<wV9h&=lqha1OJ2v{mEG>A$|KBaW`@57rS4W2q2bvYpe zrTDE5T5@HZSIFAmuP_iz_@<CBJqmgaCnN&{znuHxyyIDhAr7pK71oH6aWKkfjcN=} zr)6<B#8=IrVC~IlqDpb|RTHE=)3#ZwSbH#(ru5zpUjq6GL(N10^TfiCmC-oP;Zf9l zUrQ9fys!>;kR@1#GZtuHD5{@n@^FX&J*^3TyxOqmwpE|(Pqh=$)_V+|;gr~A_Rnus zcAWWE`p@XrZGwKgLcB(j%+ENLgkMaT+E;eSC4aYAvv~~JywSH2)AJqul+HH+(|${4 zQl?d&LRJ@KwUyx6P49Q_2dO=*)>eGnUh*xh(w+SD(dMHB8UWP=q?owvc)hBStH)`3 z@<TBt7ZV#MOWP1IO!32B0lb2RoNY9K8o#Nzk!WZOz7S>LeyZxL4u<vfHqWfGJEUAM zu<r8?fdvY;`%HyjeRBdUpGH`zjGv`RlpFTDLV<1L8pkg!5f#mtN6~t%HTu&C<#Tbk zbrr|{b~6P+qSdAMKwjh5tB(Nqydk{(Tfw@o3ntHN7wea7m`Gx9rBW>H%u^<05dlyb zy#5>ulA^Z|$3OdxcX;XhfNW-O;iCQ}WfP!U-!pj&?Sz_MhHSg8oidth-^%)YLTGK( zd06?afWaTNr{rRb+^GF^LnWlLZ*#(Ow#&n)xqlI$syUV~q5+$zHXco?AL=mc=-fD6 z?pZFgJ1<JdX-2uHT>N!<5T8S^%2q1}0rxoWmH4h1qSaYatrJ9Ds-C$`0d>qJ*abU| zlFm0pJG)bDArS>v#WZP_Qc<iYHvvV-WORdWbq-}}wo29S%BF{;Y9rVGQ?+*2%cB<p z<O30E$0;yb@90kt5&&q(YPzZ8{tT&Im0}^NJPt!T7|}vDfwf>-;fRm4vKDnh;Bd{= zoScb@*wp!x<Q4#UR*Tl&TUv2BuLIUgrM1K7WC+&_obcamA_<-y|7?LwRE$1po*p)t zWvFU%vca=6gC_7t@^KHD*A>>o@fz~`z)LLL!5bv#ty+m1yuP>Qw!CW&ICPznpkquV z*<4b+OG`?rtr`jBc!b2S4m-WZK4yL`H_k3EX8c}1lrYnCiF)Cg{#NK`kpBzx4$gw{ z&(+@|?Op(Kfw$&Iios?&W8mq|3>8)z_e&&`;3JMYclw=Nvt4n*ytdvis5Ytcw6|<r zHFIw^r+Xae{VUa5pHo&UKYa~dPlSD1T}h9V4Q43nBQkQdpULtW_yH_`95dvt?To1` zLB|hw<=W}COI83nrdm{slOsjxR?03>*JK{IscF^qE`>&hE2q-8LNBiTPC?zUX7r$o z2+I&Pt?G5yNU-C>J_)%_BpWdnKLWNHX>DwZVl(wqj-*NRVPa&i5`k?MTgavU-<q(N zENal44Hw@R7lbPf0WA@Bcke~OygCaz0UT?_DGdr<N20GKVXq_IL+lw!S!d~yF|QXk zJ`e^dzcz~^&=V9c=aAW?Hp1A|N4s;gLK9`WTx()@VUc=)O{7eOlV^V0ce|D+cg`0( zzwB><&$eESmS#bZ8h{~YX2Akwyv>g;7dsN2RetKpGv->9hRvOLtt$5kU;KwY9S_j$ zO3`H79wGVgb@h)<Drln~H%r0SiMP1B^EUFh*hp`k%@3WSX4F%)?b!9eSFZzF_-We_ zgpt4);I>X(3N?n$FW?LB-pWLKbuiN_HB^Afu*9TCAv+#MH>c<{VnET|pX?6p77oFP z7Q?Iv`Q7mW^_TsQ|36y&eKgoxY)JI-5oRzWXreDjL1H3;fgV}nQv<paLIKt0mg!=i zOfKe|eCPD??qOkqxKi{Kq%EiIWxKQx+R?)Cx3v3R)2QRyBnDC}<!2#|SS&;blVWt$ zJXjC<Yt!ooZ@IaGMF?;PhwW#g6EG(6vlNK~^h`+hfC<xhAn$f69Rr^Z0}ZO^lg2dT zgt1N>!XU%C8$yob4Z+e(b~7s4Rw$^z!Ox;kCTnd4E)*Z04cdga_jZ&0Oo}nFvhTiX z^B8HF@z9>>V+r9X%5@U7ZpTXP4`u1R527GG<;QP_zVZ8O5?Ppso#ff2-8%|&E3o;V zESb!59(VvHDY1`G5}9+8x5GZitx5+nWN2T7U#<N%HOk=n;z`#$+`_Nl4-EfM;w5=v zq_M$n9D{v5Q#b64solE0+kDqsGe?HWY{P3lR9lQa-kWjn{+L9`7m-;3@4F%_yw{2A zEhMX_0P8oNt09YhgV0tC)Zh+RVFf)z0u{Q5pM$1pV}(AQvvGO0dwUb->OI83#3$FK z)litbWcH&7EVb{^Qs;O42rHpX3oD`qFFZbnCRM3-N;_6T*zJFQ#Cn7$Ep&{f-3qPI zHvxF#0HMHhbw`1{=%Kb6W<!v6H%h_s;bi$s$l@XJT7D__GjSj2jos*>9HEdDgubY8 zm<lVs_BZ>^kmAh7OZ{JHh4lU%z&M}+;{jREee_@z6-ujBeeY9b6yk;s@;6U)O3-%a zXs?=Xri@9wRnv@5zk8J}mvj7ZihpEdgEc!erZ&3Jkk?!DUhW35j{w$`Syap2wDxDN zKQdXzk$e(8bC0!9hgL-jlj==jH%c`Ww3{(N`@a5_yq>DeN~LgK0nx;Touc4Ue(;Hb z9_U~yC`o5bk8l_ms%_@t2a^UzSo<{sBI4Ij8u4>#QzZFiS|7vdWm1hEXOUU8j_!Zd z$O7cU2Dhz&y=T_g1$pIM0znmMYDZZ|(ADyv*l5a8<9v_jOZw${OZm!ayTtj*zn;3I z5Dd>1&z}#;gXT&7@yNmWanRi@X)ejq{NmL^u9f(2BS;J*;0ot2-*YD9R7c62Q%m-2 z@@Qg6=hOh6GX-Rl-f5C7ZB{a=_7f~&R+K+YXY>bem}qZ)t9>{|ummF*+cyQ-VVAG~ zs^<A>oM8L~)-g#KYAFiyCHj-M`)qeVH1{MbbICB+;5Q$>@ea=mIHjb!5nyRHUnTrb zbh0~@>JG&yEsO@}@Z)le{we8@P!3r~M*bGYfecElF-}kcDGSfC1udLY(vK&`DhpIF zl;?taepVvgZ$vY_{^k3hj`d&4BA9@mtDYfsQ3$UPrrQ?w%Y#0-Nol#ZzKk07{rEnx zAgc$+z1O{?N*yT_Belkgs9c+R9*3ZmM7DcGYmYvq`1~Pl8Nex=$~IA;3IEBI2xLvt z8i5gZW-hxwk$prdPoF+Iy+G{k$3mtd?VSW5<UIXM5I=fUq2c+{o|xi$`Gs(cDE4sj zGo7d+U1P;ug=3{-(1?ll;IBzQJX+VQmi4}jrTq-f5&!T&gRvo{$<9B4sBq_*3rSv_ z;6(8OKnVYZ>00h!uu3z>Vz2~hGBSLj-f12VLdvZTeRan#x^mr*FYOzMyXMF5Vqq#R z2A61z5oMwI5>jEG{TK>!Hr0M&S0Z@q8eJ;c47Rp%Qnu#{4fey~LoKoa0?;R8CV1|# zMm<C`2K~`l++2=#?fk;O{LTCs;trO&&o&OI_M06W*hYGQG3-}Yr2wUR{!jjH2)k!T z2Zqp9Oe?#3FSQ3TqLwd@65}H0!X1#8Ri2y4d7)z&nvncm?4kt%5Du|Dp)vLzat#%D z?>LeW6)b1)-Q_e$H_DCNNe5k{(}nFU)FCAfpXqb7HZBFei}M{f`MtM+=n>D{6p@Nh zF1e94sj?(mZwbUibw7?DyAkJ6!!&@DWTODpis{<x#5Xv9H%xb6!i3(92a{9y=mDT` zn+!t5VC=?GNC9w%G3EobgJIS~8l#6#td&GJoz@uxq!Ytp_-}IJ^fJ<Ty5cUK65HLI z;((;pbwy<xLgHN*WUP^Q2_fs!ENIGl{@5NMKd%6;bevf_<8Tl@6pa1%>3WHR+Fq9t z5m5wA2*}|lMRW|{hwrp0pMC%8JJ>B3y-y~<=N9FsxiUwj313=FESK@zE&XD0ww0I5 zC2w)y{q{msta6AHt&-l29$m(2V0fCq?zPm9ykA5>mNH;NAS3e<_s_W^y#v!8oF~F7 zFH9r`*rsLUON~E2|HD88Cx8i}v)6bnTXG==2Gf2YI%@rU%>Qi90dRMfk+zEbK4vg7 zIF^C$^Q-?#&Ho(I^%FQ4-`j?5@SL{5su60^Yl%O+{_n#sc*CZr`Bb~t6zEW}iEy;( zVDrC!iB19grAOG|83j5iZ1cOYee(a=DGi|4Vf_VzwQ()&utI2&eGTgmU;N)!#RK+n z7uU0T14n_*2nPQpAd2hny7M0ZmiICEiTI&0zaJPpkcyi~?En86do^tAJ7BF_3iNn* z-#0AvE_)f**UM<JOCdiG-*qwEFE!(`)k<Qgj_hf^n~Nd5cq9SdMBd1&fgMel;qP#K zRDg2J@jc9IwKA2^=diRd7kibXO6le6e|ZZq=AQm$TZur1Ntk4r`0DX$Skk~970`1( zd5QWpk|+@#8SA1V>aM|62y*jYl+CE0c!U4Yr2oOe)vx}B=ayJ`?o?&(CyQY)^GUs< zQtVd`VIXTYZ0>OUbDQ?1!I@q=sK2DaMACp0q6TW{64f>y&X|rU%9zD4&b?Q2JO9lB z&{&FThAcpeGJB3&`HB{0waVMGYMf5Tb&b@n0yU7@Wr^Cqn#EYl401XEGoeC}d2oE5 zlJpr@zNYl62A8C8tujfNmy3_QDjiC|qt>KREi6nuFfOb>3cUm7m>*@^ma(+%OdG5= zk*Xq1j%!uNwmOQ$?!4dL5%DP^o7gwj4Ju{{Uh%(A4PzdMvmZQX6Q~Hl<8Dw?yg2#+ zmLhSAnZHxH<QAy<cQQxf2WRJNEBlHZbKewpcG>_qd{|vTy}`yuzY`=AjzOP>Soz^> zcQ)ZvIn?r?K)4YncKHG>me1BJDk^QJIc$5fN&OKZmW!X#Q%*VvFo(pyJnz|-EVEf; zFb5*5UyN#MXF{#p{8G9dQNaCZnK8jLObgSTi@1XY=PdZX7&KyGw_5#$X&dz&8N3U$ zoh6W%r?wnowOi`T2=VFWD`ScznPV0ac_y_YX8mh6--&qg<+<KpHwt_n*==b`p8!XQ z3<eT+>tnqIi>F9T!QLNv=)vMy@!J=N(qH$#=@gm63;>5DYsSz}##ovIL)5dEgK55# zp1e)LB>mFObUg~Wv9K_QiOWm<-{ABQmnqEmn<>zyIi{vZCNQbBH9bzuS2nQr(@4_g zHLo1k%RPEIYrmCLW_PNgwVCQk#Hmhq=wZJG1M2aG-SxjOW2-bEf%YMcrhwp+fN5Ki z9+)aO6oXugMCu>>j3C)!gvsn7Q(=`CpMyj_5RjV$i@!TL1SMG;YY`^?M{S(p+X*>$ z3ggs@*jMg0nFue#kIp}5{>p4FmR2!FNCep#L2DU6jQ!^3u-xMXuMEt2K&x3$>h(}- zUkke<*Y_<aMYd0d=Jy5UI(!3VvVl=-r<>5a$XDdK_@UIT+nH(JeWecsHl4eqWWDV! z{U&r_>T!c??VoLOb<ja6ASeB(A$pB8F3<8kmOZLPYNIB!I8mcW6$jg~|0xn{GcjLY zm#2{W>qb&wF6T@ljI6eujRkOc-`-d6gzQlA)otj=%%>|}hqp}m9OBzm?^4P_xt6+D zP?{~rUzj%Gm-A7Bd6R3_lJ)fXfZRc+GpMg{j867497wijYP@6zGWowTHZBZIWOwWF z?Y-E@i0O{|S~6wO*s|WGt4i5e7U-Y1x=%==&35(&VTZ|@w_cg(EZoCw$g+@e*_*MS zh*Qhd#V&^r!Q5y>Z5?{5I#12stt6MLytIVHjXGAVNPT?9>KHjA@4VAKm9AtzO{{o@ zJ1=?sUaZvXaO3e<lTey2>aOBEK+Cbs-6~0DLzX9)&m)ZikS0FNmhaxh;=teai?`)j zvx{0c^@#CMvy1cRnd_<?CCiu}w5XFInP@G03K>0#Bg-z%zX@G)d`@AQ9^vTSdCH&a z<r?;6H>*EF^@le=@cDoV{hhBda(}mflT=0hX26rud+|%%CNQ-~UgdXEuAwkX2Yrrv zH~ZT(zM6>C4W@eOCg0+%%PqR-TOb39Pp%lZ2r^Z&921(OQJ%|pbb;&p1OhsGfCl;1 zi=I^a%!*AOg;*-NZ(9+{L5&z0>>BFtYv}9_rYjtT{H663aV=x4I-v(;`aOj7x%kiO zFrlO$pd?&M-N$7vD%n=X-zM|JL}m*53zELKg)E+$XngQRg8$s6xba()?tB909dEEw zARU2|etc1NLT~)df#kIUoZ^BQl9=<sA&}U!{npi+=SE=@RK%X;k2~f~*BgdGAMGs< znJ&8YhNi}EpIb$A<Czq!$|TC+Lf@^Uk-#`Q%4k=Gaa-+rEwC2c){sC)4ooC3wDuvC zoJ*xM&XGyh9W^m&j)4sQPQ~km`~tr@bID^%9=ng(UUcb@)6q7oR)766<9>w=7OS#f z9=DVQTb*&@-*Gb;VxKYc@Dx&>K0D<*XG-HAvxvf~w_Do0s`w4sYa~y@)bkcZKt0bg z{+D___$M$md~FW<uccx3@=%j98>}kG_9nzv!d3rK(^}D01aQt`rqQKe!!7%@SUnp1 zCfm#!OB<9$N|erLc<IdF^^+H%S&YE6?Xs)$8$`(8((gpF6x+_!!CEsvAjFli>X>21 zR@h?gT6Eu;`5dBnrm`Ug1H%WBF8<#iDhw#hy)mU!V|()<iCZlKV8Lp@G4KVK&SUYR zwbQRn1QO;6JMW4lU{md#8GcYvsgmrF=I8XVOfGVFSR-<Gp1in8Tk5b`M@G}R1@3`5 z4Vl?5dFD?Up%^rJw5KW?p;MRENp1)80HG6nA!5bQFe7s97cdMYU@Nt_D7`!h0!q7Y zqh19QovyX?{`yh|pE~rlk=QZI=E2Fz41&%u4&U4(6LiZl=1YmF9Ep_PIy5B45;32I zX^;u9$*q<d0OACS-1E8iNTaBwnb5^Sc-EFOsT+Cn=khPc1rlPN%lO9l4^h|+pCDj7 z_@dfNof%6~W&HyVS*=2z`MkuyujXgg+*PXQHw!09%{g2S_F&n8t^52p8b5y;dVh6b zTv^o+RPVZB$Yf44Mb^sjy=_n3XtOnaq5JMf{vJ&T7FXmi+P4y#EIV_j0W$Iij9uZ` zk4k>=<d5ycH1=-CcwDOLqQ*-2bTLK89}3gCWZ64)WK+0{meN(ja`c|qqo_FS<$39i z0>{)6`l<N1{Fx|^E&_Y|^s83C$*5$ku5{@G1n=+%U-03j)v{qRFfb&a@)s=6rz3(u zRm(FI;D`KV4mgaeqPGeuuZF2#D|om(a}J?7{?z!@z@MtZ=Uz7BN<3KlfI3(gkC|am zO6B4$vS9vgmZXM65u{@q#67J;^zAv+IzPAH(r9P-7^&byekTs&i>lmWb&jN56?GY` zNt2d=K6q|)2Ug?xNdMFJThUW}qToG}-n<-LwZzJEB0oXfre;_y5#MH0@$^Yb+4t09 zh}{9Ax|>LTLs9eA1Ct`VVPEBcCq#+`5?Gbk&29G-wtf?QxZWuYd;bn$%zhif_T!Uc zawR`{^?wj=OrZ|D6JJ}7+dX(D#qV?>k7Mnup$T``o-5PL<9$&5#uN5Um+dUwUi<sI z7Jk!XBf9|rEY6;;yEHV2ZB;HB_kW*=LR?r!b+S`vsKN1e%q}1_X{=i0yI;=aKVnSx z+D$fLZHW~F4I}%=(iI@7X8kPQ_(&ss9cbKF>Oye0Z>6k!Y3`$NAz@jr$t8_Ce$BTJ z$Um6%zUb0xLE7~fjYkS}-VMdD1B*(+q&UN$Q$pHJa7$XuHa`)=>GvYVeV?m73Uk^N z<!`JSU`ub|-dSe#t046?LZ)F43dk?Deybv}Q}M}Q>KO{5N0q>!mJ5Gfn7&EZV<D8b zMA}SRA)Co#)~U;qH{b8ZhqNEFL-%~k66pbFq{4P@yBi?nW-+G6Wy~jaz_(XBjVs1N zuK~%gB7Lz;JXiY-Cy-34HDctcNB6!qdtw-*Ne+Og1x9&g7Q#P}pO$BqGS8?>rLNU* zPX*)(m8+WL%2l+VJspdsfq`$Gnjh~P_7L$Wb87trgFJn1eD<)KRC7IQElL?uUtIfa z;~zbC3=erP*wV$_qA|4FWp!G65xJ0p-muT(yJx$eeNpy)6w_MB;;x9*(9z|^mHT0x z?JS4>U#4wd2Uu5efoXS-0=)qCC_GGR06AV8ov^GJ<C~MJDm;4~MpfqX0qev;C8pF~ zg)9e=1boQU9eX=jJ85B3v^iUSnMqkvN$%(g$_C&a>LU>0>Di1*-$jxKc!e(@(q{s7 z#=CJbJy$hgff%GZ*<#r9e5)%}aJOT85Faxom2(ul!JRSjo1oCpDwv{pozbT|)VkB} zRb1dmo;=B@KZJ^-mTnpUV=2p#s+Z5g!|vO=6`K(EJLS@9J&XW6l@h?K81qqDeMw%6 zxH3v}ke1Hdf|58g=KJCoCUB24tSqcrUUDWz*@^mdMA7cTvpW)rcRUsOSe@1zfhUYz zSPpF0xyxTJo+r1mm4RH7%F$=5X3SzN<tPj_s0T>TW6M##+TU<owTzziAenU1dytN% zxl{<eafB-_rLVrM&>afFNJT#zV3KKnRv}&V1LHi+$!A)h$M`11VC-3&hKJ^MZF78q z6M)V-l&9Bs@o)5|Gn12pig-LjJ)Mubp7>PKdu4%!zKDG#)mdD*p%du4A9c7PnS!(o zrhd1%>Lw0rULr&oKjVxhYCg+*(r5YSg+`A9FLWn`?5QYdA@O0U5-b=U>%No;wCJ=t z%8A@1AK2`3wHg=FPG7d2*lxD$PR*977k^H~#)mwShiGxX<>arS{p|p#ZKwkl65|vO zk><RmfKeSi#?g~@wyA)kqhRyPAMH3$nB`=QMpZ~IM`g>V*R8P3SuUMFJIR@nV+VtI zvl26BCBY!4P)%|*^=KFgcL0ap?ur0t{v92I24PVusqaX#xArlcg+ky3cBb@eRZ_de zSgyoF%y!~e((50`j66IWVv3T-{!buE7mey{wesUCwA-*&ldcb0DFxaT4%Er5sgDni zz1}98B&RvTj2`~vs*@_<({cZia6GNb&AK`&jttdYCYLY1zfw)}l(cI%758ms0R!aJ z88Q$t<WGTT7sAo0&Bs8_qeQPCqK8frL;Ir4mk&SAz3bxrZr2uM?mcm?KnIy$!HN(z z?qKap6q6cMX>m2)FL=04H?3g;RC9~4Y>dPxotZAGKR06kE$H~&^J*?wFoRFv;Xq!M zg^;T+B`!l%)15_}jv!H*L}#?GQvfwEH7KF0Y%_(_VV!i>WF-`hN|@fUpS;wmcaR(> zvvWUGlAB;wU!$7-NUus`NS3s=GXC!EyBVXdqY@yM??z^YE-0AI2=_0L)lo>tD0Cx> zm`Et1uOTPuqSv($BG`B74KFkCkWJxX^Z>FYk;TkU@c8s_FMV3->6Ip1excX*v7iaX z?;0V#rmG%{l<AaM_zzftC23avtkUc5N#m{G8l=?a+hBw%wsI@9!%^f1V*PZ^d-TG= z7LZ%MqmnCq1Z|?@CuY-2Wdq?ZjkcwxC=CZy*|OdOA_d61#MrG+wOTT!R34P_3DRVI zX`T}2InBIwmzT9t3m=l*SZhs(svB#tx@Jb}F`?low3ni7e$ZEk5&+dxIt;DUh<-<z z-0JpJC7UL_?Q8x^uZxpfE^nv+PPjbNWLM`{+SSBg>N{d8ph`oF<IIFT{&v18o;|<< zkYE0NkrHDPHzr^>T`$nB;c45>6NqEU8%HK8(k4DgiX(wsrfgsbfVzh#$#$$|85psW z3C4-5^d8a|*?->2o(Yo825OGe#rIBLi`%q}bZsjIYv;966PL{;t1~`4tx$M8@e)@< z^EHp+3#+BFNx!<~)I}?J_MiAkZ0_OoKgl_t&0LV0y{Aup)xE$rCL!@_2q+fbSz>Ru zMvl(gmk@S{k|(FA>E*GvAzcO`y!G`Dd0P~Q!2qo3pC7oCjst&#nx6hc)WEFSDYO%a zl*|tPTGK2{C*R*xQXiIDs^6?XpK|z5=U<4#l=2;p$K6PP5wb{vxgUk-sq|1=&i<in z=1aD~l=O##|Kd)6NfUkJ{v*pqevQ+l8k!hiecQJiOew^`)<ID)Syxa+Xiu3e^dg1r zQ(Bdb<&<NI$i4ShWw&`{f^W8`dqRti85l`JWk`M~+RQe5i-iw&LFe8+OFP%fD-UBw z4~c$p&wxm)f59Q}umpj9xh^QlOpTCC67JEGF=HwoB#vPHakdnapKGw1e!{Z~2P6l9 zDpk!_`!`(NHLd_(I=O+|)?h)0a3=4f5lZaOd=yF~B)6R5E$6Fja&&an92u4Dt2hf@ zYI0~g$*15IY&w_Zrv7Ly7&tb=h04e<t!1MFD>5x;QpV1v#yi~dMv#5i$lpv2z0cIJ zX)-hD+9s4CW*v`CzNgMh&tYYKV#3g~ebryH(pG0Rv^}a)$l?C>D({g=V0Aftwg8gJ z`J1^~PfcoeyGUl!`-;@N@RTkm5dU0Qd>=FieUR)+nr{cucZoDwaVDQ>*3e@2RsDhX zLTcahw7p5n1WMc#OER@-MyL!MTCfqiS@XG3jG&qyC6N~{IenKfh9qG!B|Do&1AqYk z2@Tzcsbk&ak1wn+gFRugxGNCx@Z+l_N24&hD0<Y!$6mHt;~h3_Q-i%4OlTEEn+cXh zC<zA|OR_e)=6nMOGA4ZA5^r9UPmu)vNrSk$%v1+UU2(Qt#p5_q&RZo&>DQ|IY}Uf@ zZi|)G6%r-3Yn(T+VH0|*#zg%jReFJ@!ZSYK{SF${YVxvW(#68$&`TEDM(rnzu*Vr8 zR`WF)ntPv#3_2E#FBf*x%l4*=wI3#l&z~CToTqEE4X;8&vM}d9m*V6eVtZ(_nE+`V z5no}q&sqR2ml!+b;PV+4#ZxBrH!%Fq^ce!zQ;<z5kM)H@OIJT8PQ)0O7e)e+2MDed z1*G&4hg_^GwwWL#2ps~aM(-rZ#?S}rXs`n5mv*k{N^tLw;Fm_O#J+2W+quaMS$Xao zMK0MBSISO2hOhkHNk8g<5UYV*^WCs~y@y-R)$m`L9vmaKTIDS{gAgcdd89sJm(FyS zRmfa|-a~r;-C;3)U*dv;G^i%fQ%!uDh$A_#GbLW-4R@mS3PK;*1FjF>*M#Vf@}T(l zevCKUSUEsp(cxN?dul5$jdFZ?u}PqPEi@yhw%RI7u=|VQ&+M>J46D1hdhN6*&_BX% z>I1Kacg)EwP%<@cxM6b3+z>2dt=qTrAd9Hcf-1VUp3-WLN9&OO00|)lPd~>0rFjXG zh|hvTl?gfRr58sK#r?}KVFL4cxYNb_+Mf-dEV3Eok|7I%2nE6=uXG+K;b>2GDuZNv zmmj}F3*3}|Vs`j`yEyZe0*Z@C-ZU3P;md-BrAHpKd}(fj?$>W3Ji(7k=KLo74LV4S zF8qUkeLJ>3^}&JHMmDM0oA7DmM#YDJsfb=(2-@f|jW4$aPV`2T6|9MG@7;%2=g07$ zPa$yieK3ZKuq+Blabdrb&4>Ve|Mna@*E-}1o9Y+sw|+c?GnVot%%tb7Weo50f7T6L z`y`jYHYT1uxX{oxX6V;5=5$oEWwcv>>k7W|miH7v_X$Aj;6@^2>}|*m(0A{W7->El zHiC3akzy6lQf=kb<C*Yv(Ip8OI~j*!SjY@{16*lelv7Pk=x&E8rpscGc)6c<(CY<G zG>Bj_^oBW4W*wI69$o65+zH+F2M93hM6Z-n(MW$M8k-aKs0LadN#X?g5gm_7b~H!g z;L^ZKKjk7?^ci-e{?Cjn&)z-4x^BPUL~_)HI`$=1GT!HuN`>@v9cY&hl1rbed@Nag zS9ShkaOE%g`*i@olf2cDBmLkF5Mg){G`1xGiFy2cI6kwNha%CkxY{Su0x})$ECszM zaF$UZCrNDO6wwQ<E9hLSCw@g`rpA7jA8l#SI>871{>XAi$a0mo5|iL3e*|<))DF4m zA!}T8AxsJ>2AHE_x=ZT=$3TRow-g(t!TZ!C^S#YrzCeR0c9SW6s9kdaww4isHJKNP zOG*||G<w$n%o~kCe3Wu_H9hT!hU&<D9BVHviEZ2$lER(Tyo`$`dEZSu@XoloUxL?| z4=+`tLjC5j=0?H)==&)&D;3ep8=}Qq9H&j8baf~6)bJG-ET_0h7tP@D_Fs`c4KPmt zfs~a9n4=Zp;S^ZHzV(!oU?Ne$0e&<F2*V}zBvC5DN(G?(MtDUa+YdOHMZNJ9;=9lp zj_fiTN*$acTi0eHGyk2)Bw9#d6l0L`0-glvDJ*-+eJqvzi33~-&h}_YtkFP*rp6h` z4sJQeB!R3IXEUDC(}x(aV{w<&5(_lwf|{TB8+OJ~<F(9xKf8i(Xla1*3zGdfNbWNs z3xMNo9q$4D??|L%a0YuT^lB4XVO#B~oLzn$NJum!-zn8r^>rT@b&K*gy`VE8HfSjQ zqn7_C7-o;Y;~?g*xS0RuwEW&r@GZ=i5Z~7;8S`NtMq*V<xBL7ZCHl{Pc%Vyrj12P{ z5`!Yw1kH~v^Y2{Tf98z8$)mqK8~>%5ep9mxi!g)8e#%h@1^OR;>VJpz0d_e_Q9AWw z1*2h`6evCBe@^Y6y#-mok#L{|eYG(k?qC)LZ_74}|2y`Vu%F=lNnnGgferpCGta+l z$iF1r-?ZI+B8*p%Ed5(a^v_?mkdp%%E#~u@pCqXY?8?z6rN{lR7sUtWuwW<|3(KPP z5d^dw!;tDw=eaEa+h7@s8T~g66tWpVM8lH5UH*@s!i3wHnF3u99s=Tts8R1IYIWU| zm#8JqTf}aGW`xparY4rkQ!Hw*1`t8lr(4XiAZ;HeLy5AsKbUKD2NFgBiBM{Vz7e4R zp1Ku!@e-u%+}^kX76YTM+5RR~i`PZvN4C2dCiQ#XHYb&TFK7oWt<zxK&9RDNKMxt0 zwVi=1Z}MRB*~Q9`*DJ^CJYdw5gel))GD-@`R4C;jN%JHfh@+tH9u2ep@r#zC&++}r zO7)!mm9OjJI@u(^{A|2oPE+2MxCY=W)Vk2MpKNZ#WcJqeC>zpQvAdNjRe)vN1#z#T z>DsDM!r~Yla9x#k=;gaE%G%d}GMmxe#VRIVE;q)3gmZWt1_Q#7+u!EB&s&j1=kkD+ zON6m<s$tEhCoybJ8a?M$Z(!;|NtoS>WD;2idc{>t8^xEQR2P}?O~-Xvqgt!^@@t{) zb<FmiP5rggNvIL@^kakKg-Fkl0$D-C*SPh$s2f&L$Wj_SRbZaNKh9!ls=#NH%H6Jg zzOrX-wW{&18*lcCq>J#b;W)^nd_(co!O{I%kK2zr-9UB~uR8A~7seGB@apn%_PDa~ zedMd#nj`wk?j}wWa>BkbD+*eLuRsk)2lz>K9NA}|y|`kf0_qrl*`G`=K(5`_A{kOR zHEtE)>}$~oG@FV3$6|7Mnyi!dKOtB3WSf`L7U^z3>5IMQr6hn9ov*61(-84Ae?EZf z(nTL6W}zt^J8~r}J5u$5TxF?wbLkHC7k;GpQzfxaUCV2JZVgRq;}-t=QUp`N+WFXk z)fbX2a!By;dd`530%drCSI+|hwYM;5iP!}Qm&|r*mLCgapaaW06uXfTGl8s^5o=fm zmFs=>9OKNz!!}D%sJQ0~=KAw{_O!r6FvbVsdlMZh(_AZZr*2M?+`>`Qr}%#&Z&2FC z)7ch;xN=*%CT}YVBV>>?+4)gOB-0pV-C106UJ1g|wpi|f*K%D02^l1J_5uv}QCGO{ z?ND(&+;C~YQp{2a^(=>d-h72Y6GIBO26t34)#PD=nf8Mt4DD1+OCwWaHdo>7FNW{- zx&>FmpOJ#vqpa(f2iz@(rc1*40oF_1!%fHk+*;P}h<xI~Mj$b!`n2h${`9)QRw4@m zJmOv+H#V13bE9nx8N9+FBVk(FKb#=&c<##r1L-j^)GGhC`erAIXn;&gZgX6Hypn`X zGQ<7A*z@);iWGAcWveb(hrlDr`}V*cpVuV<Cy(gi>T#=T>*F27KV0nb`Sj{>9^RGf z(H_B%opv@D2Wvj-{m$P@K=ZVeA&p<9E;5x?-Ek$%F8z;^qBIChArs|eA8C=#AHXL2 zaLTr@fG`Wo2Ws)NO-|AvQin86g=t6y3B(?k))2o<Br!}t0QQ_mPYlqtA_478Kkrp5 zWD;;T?ECXOWf}#b+AlL;Vf#A;IOA);{}&6h8S9enED5c5TudPhy9JLRchn**2W{e- z1-~6Yz)@d)KQIn8y+U!xg4`gem_tHKpS4&1rejsugV|2k@LIPU(Kaomv0rqA(dl&P z$2=T<g2Pr4A5u7Txt3c20?ZwEcs##}907S(Qcf=>qUEuEn#~l4RVL;>DW~8&nDowX zXvwbp5`Fe+ICBvZ<AKdLnvz7{v`y;XL)0UiVIHk0dEMg!xqoiz3?g4QP(k<F8+WRI z)<J31ElC1q6B<QCr=+Z_GYOc3bK+{0!J)~E%hTGH@4GXbY{be{F{iowD@cFa@QVW@ z)<90@*2lh%cPV{@43x40FC%wGRZu<dA|PcbCV6(Zkj`II$$PG)hod=SVQ(%kGHzjr z?6c;tw<JF+fy8)Y-tIbZu^OsVvRC($uIP2Iug1@E-Q`#R)rARe!{!a=A}&*DxTsV? zK@jYaYVp0>k}{`%^wF*Yh!n^Qhz6Ou>NcXQl!f-y_Z3Rk>ro5!EALAe4v^>4qkO|X z=BUws1Jux=@kI=k8ZM4zXhNtrt(1=-^1HpM??;UKyGP<H$EA!<66UnoqfS->2qN?v z5$KszjZcG{%9f8RFJrY?>fbNdwQ#)1!OVM1O`fc5^HXfDNTd5R4>?0Fa4Ml_pyFzK zOn}_1aI_~eAesi2?_Y$2{zVw%hh`%7>ret?&{Q4gZ?ha@3s`}-CBHiJ^%>B`FRt}I z?^J_tO+``>ZAx5^WUV_(JK+ERs_jv^W4{8bO4w>GbrfN&koA_-@95Z6i-VK;4NLbe zp(?BzOdamKw5o<OWp_^tT&SI5z0h(cL4_mYZwrFjsB`OfXy~$wsCun%I2UV`frIs5 zv1wX$nC%LC>vHu8E9z|TRn&F0o1Hm)rKy%az!ru;8Iz37iPnuG9I|iKQLsrnz@m7Y ziMHX*+d{Zt;dwbR=*9#wNNv?7g4199SWDC)8ovA&E00)8yuhS;t{paZ9&w>Mi$=CT zH|x!aqD%fb5rkGLna>OuSicG1qMs2XG13w{;{6lba1BhG>~~tYk)iOu39y@vUQTLp zlGY^Coy8WmG_YiTl;2tQw$^COnk3)wW6J5v$jvbY)V*0s=YyS}rR{7bPDHfXOM$>w zZek0Wg^*T}GNCU$sHF$}cubLHQr|>^re`R!$=+atUAyp;F>6TS6uS2}#|Q;(FM_EF z1a?*5MC#VZLVZ9CzE3kS*<G#I#xhxE^6ecSr@90GU93~jKK=C@^>5>)QOZ$RNUr*_ z1@7TRPiL?CA3PCI^-Zzg^R+ArxiHy1TE&}X6+81{??QYfPA=F^^nT*{1l7h(Jb{J{ z=uzyeTo3<qC)sVUFm~hZfXz-k2SlEG45Cx_*?hLReDNj0+!PlI<*5IJ9cStH%HZQN z|3z;r2SH+`;?F&@rmfr$qN7w~Q*L9#?a-PK5t=)jqlbMFo_Uu+LH;2mBPx}zbY0cC zAE#4v_2TR<IB#k4wyPCS1~_#rrA<`Ft@uVb*HmkiqoP9Ax;(2u|2Wl%xubVs$)H5` z6NWi5ZnmEMtWB;7&tR}g-}@6w?Xv)CK8EwLc=qS_y{XXOg7lDo@?3~&h@MZ3q?E1e z{R+-Ouq^F;l5nC_d8a8y7G?+%qE1k>0F~)9L#C!{ee<ri3eK@AwpI9vg`%pSsJwRL zJ~o|>21u_i)|L8-w)^c0pW8c~LV-DD_mLJFS^@k6S@$Oim3M}FybB*`rdV!&xZD27 zCRYZF6N$bRjGjx-$v&HlnsXPg`|ttoN6Z%_1<?h?r?fw^{qksJQ}_GX&mnP*6iG7( z5)OOkN3q)j0|VEloZY-<iJgweVwgUw82h)3D=I5YNUy4U2zIGv5jXmawLc2-Id54O z`Zq=?S(WW0nUp-OJ@1p$_*STfUPE}+p4?9NAayjm0C#ZXNo9)J{at&zI6s2S+0e%X z*Qd&Jb`uoSdsxF;wS23lk4(;XZ;KerBR6kbeJvIWjZqJF&!<Xs$fvr4bp~(TQw=-# zR0h(7H4~Y?d@-g|prmA?l8Fr;pzk9u<bXDsr-WS2@u$s=b@OfKjMfKK-b+)a9RBb! z(u-`^&uc8d0oNp)8waW<@20G-CnOl8W@wV+k2jz7<`ogocK;&|q%ZGX;mh|61=Orh z@-*Lg&jVI+NdH{I!pAl2+=f=CDHUfe=F|N_)=>7=$K7Pk`fsrE7*dN9J;duP+Mj@g zjh~Bnq3A<)ANx2v@mR^odJ`ZzNzoPh2AR%NqSgDF#<bq0u~V^DBV`EOr`kdKiiNL9 z7?M9TcC_?qm(Wm3Cj=hiy!T7{8bd^|C4esYz1&-_4BHCma=*QqCLZHB6hlI~Peuyw z{{cN%3;s1rnAQ<v7B<{q8tnc55O6MOSYM6Q`Q*T}<TJ;z^JmP$=ehpxezMj<!$KPs z)vd3&1A7ZYB6gN~+MPbKm-BPxme`b-+Ouyh@Y!DO6WhlqWJ`*hM&9Plit5V`D|fC- zCR5P9sb^paY$=Ml;y)<2PV5`c5|7Gj?oD-(R!@o0sgvx+I#Ob3mn}6H6PjUT%Ag=$ zsI#m0KN(KWNxO*l63yl5$aSD>XzfQc7(G657wN}(7QB8kgTERyh)3@)6iiElE|Pje zY*V|q>8EoYY(_%v#^-Wk9*J@0dVOA6_59lC60a%8E8yg9U-a-BZQMpr&c-5J=Z<&Q zo%U+pqKcA)(I*%=3u!qUe#9alA?(^hH!KL{S_nELyAuGORSIF_U!belD~Qmj2sHk= zEFM)Ez5Y!vQ0#fxEz^xd=)-D=h;x{1)mVqdF3*l(pC|?#4yFz1=|^yT=LH(6nM<0) zhXkr<*b&$}F4MC8S?b>o+V7|hT{=%wrE=9ZSADlGWRS{w9pXz>$-?(Vgq)AMk;*eW zsuV0)L;PG|e3bg`rqpcdt#IIi*9eYc9dr_^XaV_u*!s$VD!Zj^>F$;mk=nFKw}f;_ zH`3kR-5}j1-Q8Uxn^3w-x~2QO_w$_RJ?Fgd_fuWod#yEVt(m#zni-~2ShUZLv~yCG zf^;H6{}QRt@3#~I8ox7Li5fB8bRjXAc8`sU)#<v)Kv}jAvGaUJ7xOAenhS}INCfJk zU{XAh?{=YIKPaoV6&OC+naFysI{=68iyKBSgE%-fxdt}A(kFD)hI0exLI#bVOfpZ{ zH)!2^E36qL3gQ@0aa2=BC91U}C0*^#0J4fcT}^vOc0ar=AlK6BNk><8u4y<2a6ZmN z1dBlLbMU7pvJ>so=KVa*#e2U<^7+K(^2ZPpBsInQ8c$7WW1X8Hhqh_Om!2-n84V7+ zgGzmoogXhfY+~;=*kkefUPS^`wKPk9V}eNjIA?c=H^gFfW<T7%9}e?;JR3u23|U|3 zTa^6Dm(<iT!2N~Zs&Hq%0s7DN!ex`diDj~73bTUb)WPCG{Z?3M;(j5`0uC01kM>Qn zN7b0=P)G6_K*;l~Y*Pd(zFn5aM}9CHRr1o0M_jPDvC%kf)W?s%3JoHG^h2K2t~uE3 zQ;l%3$Jl*owqzK4J-bw@uUh3oRkrwzk+VQA3s9n|p0mvI7$+<B%WPPI6|O*~_+kZC zh6HMmXFMap2uIz{GqW<&>yQVIF4N+wrAo#pbrw;4nTCyuv-%BCGRbC*bE(D;qh@>4 z<5AYXZz?9V2*nvvm_GikBP|ssq)InXxU}VxQi1D3E26eun+JcyYmTy}AnukelyvsG zoK-%I^v>nQ@AXUA6|(xmaEFAr1Mwo0zE*oR3!^$EB?9Ov?^-g2CCv-!EQ%x(bjc+; zSia1BOy>KNR<YO;EpT!)d@hRcSvyyK7C!sXE{g2ZAfOK4(ZrzFSp(HyKcpf(%c7(3 zAqX{rmgL3!`|^YnIgs#kB>ZCV$w{U)?hP6CKztk#J&M^-#d4BDH3{Y_?p_Oag{VF? zwCikhdDlef6+>M@6VppNzTQ!?xi)M5oFgxI!f9lH8xe|E1z&lfF%a<!GL@ILdNtOu z(Z}lDXZ|r(k+`Aykj98qZ=jsOZ7>{Vz@#XmAwYb{k+nU%NF7tF_^hr(EuFSVnN4{B zRESL#ceW_Q@ovE>mhDCTB=7;0CVhxM_Dz*O(84Eadxp#wovF<3^VM`8DE}}vAx_0& zQbePZqdsJUNF2)+&3+E-uzzI!dilV+IDn{KDg6o%E!V^dE0-MQaNv6*uJqXklZ>4Q zplY8|dXz>O9D4;3KGv9rTlaaw-F?>%e+znr#gdBKSfw=QT$0+q<>&^HV}~OH6%K@x z3CM9ge+P6z*yA8B)AG<!6mVTH6zq8Ix4<=&p{VZ9aqx>g)Xbq9T@$x{*tCy3h6Xlo zFljx|;{>@vOrL|ztM_%1Fk9nj87M@EMf&TTBz(6T1F>%XPjkhJ#-X5KEG%#f4hVnR z;%B(j=kn1|(N2LDU-`i|3@6ilM*|zD4bB)<61*mNhp}rVrk#IyhEyc*mcDC*0}t%G zi+U7%uk2l@6LX@v?ce=moqyjt-n;k1h{C9|0#$5%4k4qI7ph7T)B8?C{j53}+!dcr zXmpP!ikc_t7LsXB?-qvZH$i@U%?1SExbwP{2=_29GxY)>r9HnosprQbZGHgJ+T7F? z0;Q+nX%kO5V~`CK$*Z-rL-x(fUeaG_(nUZZJF+OO{WjadRYF{;r9FK_#*M_zKsIjv z=h)*;j_;lk+$wFM&Llq;Sk?D^7Rv|rYob^pXgcj)*}UY5k#bDn?)>#WV!9&&8}GiO zDNYaj=no10*<p!(_%P7ZEB;e=p2wfD#Af9-p{?^Krs-{+>3x4AiQtG6>+4@dRW6D_ zN$LEmM5dbqjZ?G7wc1T*X(((%V?;BtKoNqf%Jo~W?z;A(azb70*g;HuFlMT+<Y60+ zs)7_Aj`d2u#<a;gmq)9ly>~b>EW1m!1Aa};!(eqw4*N!20;ng+dHu*3U3kvbT2q!> z{<WNKx&Be3mh*NIi7ueXIENn9_tv-Eg2hNdb`*$C%IClJ>2_tq|I%_dfR?i#JPPM2 z4JvPO$uvm5^LViG(8?LM>Jo8VEg8TI+q(XUH^-D!jET2}(Yw8bp02OEbGFKqy7Bg_ zYR`7<?tuHE6q?QQk;f5AFF4~0!PVT0rWdcnVAn@=&G*rTQ<iv=J25T<*|VvFx*|FZ z*~AP5Nx|oH?aMvgoaF$9r~|C}roDlG`Ba#BBGpT|*Zh7tO~AtD2$k9H(B~jngfJp= z{PD>&d5z~rReV1s!07l}9RA-w3BlHX=RRe#pgik<kZ*>nL|+-Z9tsQX+0$da(gxMG z4!$D@-ojWgBBHE$(cbP02dA5mu(ufX{z6tYZF$#*{3a$qmMNfdi%&au9XS{}r7}T~ z^cS(qIR3~iXXm@|;wa|r*{d_7xgriJme)mMOkp}F$dQzQsl<d$D)c)rg9Q2{_Wst9 zAzAYy@R&-yd7%6-Ogmn2c!{)yU^<f7aB!$VB)V~6v0J9YpP@Ph^)fbA!|t=HZR<3$ z!FXXL94g<3ceXyx+_=OTBdbv_JzUc7*|PG>DE~Gbo6dEigzI;m+tjcS2^E6(u;aA- zyh`X>JTLS^yT?D39%z3Fiu~ewY!)q1sa3K&u{tiBph-Q5E7Cp0FA{pLQw%3$a+R6W zj6uS6_Ukw>K^6JE8O>&d8zaQem=y|d0QX+Y_a45998_g8k|fWcQeRo`MS=i6<sGmM z><RIC5Mc>)CapN40B@y<lTQ*bfxP+-$}hg}t7`Wx(aZhLjQ8GuUTL5HG?)^J#hxs| zl=et!>T`PD$(5a<)39~%haO@g{w1xC=+JLc#bVr(6_KHQDS=7`M1H+>7C>=@bkfbQ zopDT5Pxi;i-pz*$q`sreZ}YLW4A=QTET$#;Ije;mhb9HXe;5t>9OCvdvk}{WV7MIQ zh`DOVq}H4N<16TO*6lejDKpX=;fK70O4LXzR``^+u<POr-QM+b@BMDRv!$rONeg=Z zp(R~DFT`)>iA~2UBBnqQ*cpn4@h;GTJrRFp%LG)HYk|*HAS+8%8BSUvfMvwBio&W9 z)V$oguP-MjAs%Ov%oF%<0h2!W%X)`d^{cLAj-qjwRsrWZ-XUjk#8tZ17|+)3*p;ei z6DJtsV(`5Aq$Kx6;-z__Ec^?SWV)ZBCPw<3^v?^|Lm2|_n}qT622zqVJU)AV_%St1 z*fi3@1fOTcTi6OxMc4&Afh(x&n-jsvr8kKGk*M2{VK6#Hs9TTC`CG)NW~Cb}B-S`; z9DDc;9>m(}NROs1YMF0`JP#W(EP-4C3YB_q!+P$C8ov}!BYQHWy0<-NUZD+7rjR+0 z?wbzV%(Qv0IEcV2FT<#+{WKbw%*U^~NYWotZ8O6;sq9aRY$5W*dg>>K!c4)~y(rz~ z*PJ%HJieA~!w72+xwWJ<&`721($9zxyV8zP1s~^Mr7)ymg2;6=_zyC^WBgl3?Ll<Z zr7{Qx7urXRA{VHM&3kr*e;hHmKx&Vw`i`1@b~5}G7>#sXhs#@J@D*`EW&Ifu+$`-~ zWb(K4F<wnoEctOvgM_fL6^yl`hM>2ZqDKup)@@qhuk27W+^lvs-UP`q5PaEwbo*o1 zY5v=2zK2RRePt$WBBL5HGgfx5B$iVh36;XIXApN2|3e~D!rOW#fBodA?SN^qaC=%o z()n`O*cg>o0|t5^a4XOR80S3p{?Gc4uTa#(etQ*$;ortdQday~IkA54XeOV6&)4!I zu?5c))g9)LWjR?|ZQK~@*|NF2e^qrq@_4jcE2n>jKp@a^Xm2prT%BB0BEY?K>= z1r0y2&#yIW4wW_>bA{5Zga77^OBZk;_$x6O_`h@o-6E@f!dKQOZ8bR8&RUo~LfEOb zpYyjFWSb5(E@M{{#V<Ijy;Wy9<S{8qicVR<9y40460ASHz5%`Gj%*H9cl4aTfi?9V zY#C)ZCqu#j3zFG&h?}y4PgYwi6CzUA$W6#i&<_h=C=7g3Qg^wrU{89VdR(P~dB_#Q zr5Zv0(OdHim4U5{;PV<6w33g(uT_e5<OQrc9r6?64|uUQD{=oVR0nL#UXa6hdz2<3 zA2X!q7b~qg9A%Yk{<jh5cb^1K4)CCRPSu!8QIqX2IjRoQhJlV<QZFTkUkg2Y7r$lf zo|8chEjUm4oZOVBAX!sLleAkc9>Q*#x|9aGK@&)u!i_*-(xBRCj3^g7w~jauPD~wU ztt>4WfnF_E7WG*SPYJ;PA?$s{<kFltYEh?BnnbM(uBkDL<hRY1mCrUtWLGQ@tA{15 z$8l?7&Jh#^3~1$K;{_0>R=r$qmnp$K(COmfE$a9XRAe0?tC7-C%Xwy&kI8V291Fn_ z+2L3;_l0RYB@U83^_au%wd{dXeIo25XWn<ue|8`~(GSri-f7c#USypc3+SVF#C`sC z=~>!us}txfqk<CPk7@!q)eRoc?_YCSrZpKN|E_I0@ccqpP+p7h$G(L>=6E*1r_)K! zcut2v#gJkG1$H|Q$nzy<lR*&?m^dfT3OCo5&<*~#juVFHIMxX^Dav(bz}3D;Nkgph zSyryfds3lzLDg`?{!r(yjhS$K_Pp}mWPeA9EM%!mB80UK4zq`5B8#sm!NCrG{S{fn zRF$nxV^Cb-TsAFS61M^3^3aMgpYvi33GTma8w%Kf7+`V2oP}%@l<RYl-#XksKOOJg z#<q|AgG+x9U!L6^1K)dWFp>bM>F32k#2W^-DAvBX#}<E_&~B=hLinxM&YT}wGO~H@ zVT_dmG~R^k><lxgw7vV68dzVz#Oys<uuE<S;DfUu;MeM~|4|VC01AM%=s5+Rj-4Cj zx1_kx!eYSsbJOLBqW;GSqxjd?D7YXmOy&iB0XForln^YtZx+#h%KsTAgs2XzrC#AX zF(AZjC@3s&Hq@8a;Q!CKk5Ryku(oK_fcEz5q`<(AE$YVq5e3vgX8@2qWEtE-DO$+> zk}|;jB>C>xoc`yZ|HqTg1%`nXOr-AVU;_eyPxrFMvLXN9VG20Fu##dmcpBJ8hy;8% zsee)apJ9HOHNZHq+(!Xmc4zN_rC-0_ms<KCzx|&R156zUP_=ik1@GSg8x;zXBW9us z4)p&S>Ax!h%zbxQ_XGs{hRQz|6HF%PBO~~7CMcEoo~ScU!01cn@Gjd<hOch1Ud+o! zZWP?2Jyyf}Zy3H$=fo>5&%D&Wl#GMn1Zy<bA;t01rt-?uC5n|-Z!JN5xZAHOG1Xzm zb9m%f4TtKoJ>@-K=1%AKcJzFe7G*T(q!NLajo2onUN6gS=QPF>S`^^6UZss#J?5!n zpcS9>Pz5fBW`jOaaz7W@(*=gk<Mvx>nao6NYUvF-Ie~wj6;sICq7tUDfg{5S8z92M z>--xtdVet!Xss@7B)GE4em=w*7TBEh$$M$)=gV<{CL@m`kS*cOHW{4*F6KZnQCJej zp|#zZ^ZsSx_IiIM_sx^{1HP<wtB&3YhUL+Y-kiBqcZ}1?Z&r_b5ob$vK!+57TK_2S zWV+L-9&yduyHf#MybDXl@A1)^zC)bZWK0<7NPW527m7hX8K>&p(6Z*Z$w!?kcD2Xj znsl<(E=^k2Ra$<T-QmafWpuyFV=@zHtpR(y+!7&D#Qby9Xc84ni1>WQ)})2fM;-z{ zey{ReX;>5A{Fm_LA=iSAu9OD&QD#V76QZ*kj0fM8Ew)q!rv4eORx|rJES`8j2%dKT z?UeJBv_{|uQyM+GJ3a}RYNb8Iw#vb6V|_<FL5zsnK(1u|a&Sb5###dMGkxk^|LX^! z-rK?4w{+>KQa$ju*%wr))VHOhJpNvWUxtTprNcMfh6-UZ{I}U@CTm3>N(1l7!oy}{ zKQ1A~>Nh#hU~39kzHuh!)QX(#N=+J$C8tv?q<-?)C%MYFw+~9a%@>>~i~dfL>c5?~ z6>6?hT%__8czr2120wvZy(|GipD^D80!2wLP4owvzEVJt>e(4-q>TKKtOvzfS9;!8 zA?2bqZT4KKmvlZ{uA78tr;~9JiS+WaZD^N!a$P$o)d58Mw;mN_P#@?4<O?NLj;d&% zy+)kd!H)U(&X7Ng@3FH9>=!J~lB{})Uj!STFbTkef%b0kE;3UHE`{XR$KQnOQ<Te` zi2BAT^N-yaQfsNz%Nm$}<#IFht=Os&<O|xN#Db{n5KU4igc;YU9{nx8*69zCLw)<# zk=%tG$;_l_;8CU6goOs|XX5^aoY=|dV=ct|h4xd<Jg@4`Pv<z)inUe0FrVM}s)qC1 zjr=Aw0Xo!t(__vocsrte;L%-jSpDLX#70|7DVBbjPI8bba8I?;{lHl`@dvw5GHaYp z5s}(do~%&Nx4ozWQCAh8ZzOZp(HV306Tx^Ug_WODIicmX>;_Puq%jg66qGL>S@qy_ zQ7uSodGeSzFPoWAa~}*8;g-qfvftYG)NS_?ZoMd-p#Pvgbks^~^9ZY&g2j#yMG!?U z(~8_Bo<!U*UF(IJGW_{YdGO@!qFImQV*Y@P)(nC8Qv(J;pI*$<x1lGD!qLpEuZ2>_ zWpsx?uEIj2^zMu{NkO6o`jqjH9i_K}%ewd5xc=6Erp@VOLqJR`ct6*01brkh$d~7y zLXsj^sHYZQv+puSEY2H7Q$8=E?6Z}fo74||@pZiF(-5in1~3m4i%8t}$1901m$oAl z-*2PX%;BS$xFh2gxQM>vd5?1QbDU#sb~wblVaT6j2k9c<;hUmrs=Dvbe2L~N&bd!N z8usy;r#w1MMlT{xaMV76+Y&;%3+64JdY;%#D%9^=x2}Rcy7f9`IA0z`W7ncAhabUh z%S^f)2~&c~9*uI^{Nz-;96u+xB7@yb#_Us%(<XeEVZ&7fV<%75m=ygt3oqMt+SCh* zV;_ZTw}#Xl<F%scKIFV<S<>OUs=t4htS~w_N2(F`t(7}_>Vsn~W{FLm`}is0!XTwO zy8`CIz3=8b5gt2BnkEgA-Vf`>$h-cV(dvT*mhQC*<b!C=3&Xm8!rCfd<u%e$_Aj#& z@%3Cfd;K$!yO%uuH`1D)IYE}JT^OK-^*_rU6NvmDyfB8PQ@<Yxw0fTKThoAdd!$pq z{Rr2S({XbD*u67)7oJev?Q&(c{2sS)Y^A2}{O_}Z16uh!H)#hICKN&{xEnF8IC59D zxnEFLIvq3EN=_Bec32HNsTIzvF%eLl&$6IWWG^(>myixSx9VT=Z7%XPrNbykEK!}$ zJh8h44F<Es$12MO<yt*s#=3H4qh4I`no(It&)S*u1Jdp)D5{oA3j|Bn?2#I`^`cVc z4XtzmhambKm0FeL{AmLmOj@z>bRtyINIds4X+7^NxTT@^LXqdBQKx*l#|oz8?O1ED zf~w3Gj9x?~kzgjEJNdVU@B}nr>n@ckpv2D@{rxb(&x$WCHd?F=qk2FK%KfL=6{iZi zFt`OhCP^LfvB{vWrOs#*^y5$4tClhdUC#j&AMD)abpm~JeYskASDqfw$%_CBQmL`% zxMISs-g3%9Gw*|w3I9N(1+Q60;QDX7pWWnS%iR(=GUTG|oYl@;S>5WNjHw9>b-IuG z?JsB2<*#ZdQpO$k%Dd+DovYnm!nMr62|gR-SNsRbREQv?#Eey4<KO|fO~dkMBU*`P zwW~bEtc+i2Mxl2NK4_y3hBF<d4eoa$2RtV4gtdi<CM$|oxi}Ag=|$S!uXhVtygw8& zwwUI`2=+LY0^S}JkR-*TT`%FJ$@!m`!Hd3Dzz_M^ha}pPezj<StMcRSJC&~gKeYg) z3=Y{>D|But{aZB-8jgiC1fITAh7h4B?Y!P{FQ4loV9aLwWfh>SXj3`niRL)3ZWzd0 zD$m<^{y0;HVp*#uom@DvBlBjt`Qu>vEcXMkYXH%~I61rDded^n4GaCH36Y20i$LYG z8Beb@7?e7P-e07ipWZ5$|1nxb(5GEXE!0m@0ZGQpbSI7s*53~W)OpZ!{E0pED&I>? zTEVh0S-}M3R2IS2rf>sgwq)$l7n*Je6${(4`}21*8*!TG9!Co{;iY2)`Fw)<?a8am zF)X<#&Bs=0gNPj87QW-Pt_2I<>0#oAI2e1&ML(vz*~dd;2IWq4?*`+EV`5*$xtkw* zXq?(7EmnHX+JbYbTgl!p88^<6CO<={9s8oXcgJ|vcacuC7<T1;>a3sgd(GDx6!Hix zY&;-6E~1mzB9WEsj&-oKM_gqvP#z+X7GqlH<{qPq(pc7Rj9~rBf4`#;3FoEIxq1J` zS-{Sm+WF(4U}nW8#iDd%7geH`8=d!j!mdDt3!gW7M<a_CkM(=+P9HdDqNhAjJD+D= zw{BV2PB}NP+pd7g=+gyYt9SbEXP7ckZPyfr=`*{0s-Mv}xcymN_cBPGfKbVy$0JjJ zrclXMQ|g3IMT`CZdJJ>ryR)OM2j>WEI0LjnxHbP{czHX_l=w&Lirv&|!G;^lCmrKw zIY!MkXgXtsml$MQRCXWLgQi0<>7wTw?aha(1cxnMR?C2An@t~m#v4M;;d_|ZF`j?g zVP@lQb%I<TWc93&&JbL@gdN2%KYy}|D7j1h{UMulcmVWl!*S(3mC^pT`fo#Y9b$>| z`Mv!NH~?or&J`y3Ozpn^;b!v2v&sB?pOHFon_gW-gQ#$7mUesqbyeRFH(<D1w<!WU zpXq1Bf;`Bv#(m_;f|dS{D)+sp2Bq-4h;WT*7$L2du%EAbO8JW9lNdMNlkJ>pXQ>;V z9;HR~v*OW7^>@q!3VPw106SNCjXHzCU|+k8O%mH0y9pWYKa<4UWe2i5CybVj8Kn1X z9z9%^Tl?nSZtOuWtRWFsT4pF&woN`&oV{Eh3^#bJv;2H}jQUYh)U3iNta_o*@SUq7 zczaRjAVC1FZ~7ge*{hc`9P7OGGFej~j}Bkl$?Th$KO4t^BEGS^tC;n=zBM9LrE$07 zB@ngRe^(n%0^r1Cra({ohCyTyUlU00+l0LOc~X)$j(30<DHnw~)CVC6be{qk`2x0C zM1&{TrXjF9gJw~F6RzLPz;)xOlineaO}Y!DEanm|)Q01WqI7>&4VR&Aq{g3xu@$GH zPo*3p*ul(a-NCdrQpRBD{bk`*W76yBFUvZsF@Yo>B;uH*57vhom94N>ukpiD-)N<M zw)q@Uij5+a3d4|ihHO$ZO_+Bd#T_(5TVY;v9-<!3z0dBRhYv-;qNALwH3jc9dm}1> zi0Ij>OSefOd64k(%#!1ah@48Ue$%1(hcw`oV6Cebqqfi`eaZ9i<H>T1W+X15Aht$s zO&p*3%v>@Daz%tXUuK(4^A)Mz0F_(Ja;t`WJGwfoke-xEiSYgm$v5Q-q{Ez37w<wi zY2EIcs$jTv0D;<TTmqU61q@P1xxun)hc=z-_+7oC#5;VYDrZK#tMKmTFn6wxxAC=t z|J<6Ys;HV-WrQtaB)jVfZv))HIaPzfxmBkPCCOX4WF@WpLHSg5t#kK3%b_zd#S{L` z0YnpHx7%8kf<Atscb$HXiK6@x>dbox+02Mym|glh9!C#P$+`UA{hYI|9Si^mZY!1+ zz_)U>=-Fu&VNJb?-ydV$$HJDJR;5k|biFhbW@1<AKfz)VKgpKHY<UtS+ms8Y=XLj6 zG#`oBs(I^i9`8Fw{is=fz1$YlRpoTmmiYN8Z#$q41XRcvOx`0_>_zpinyx9l4-|U6 zSoDkubmNBNM_+D8$IecmE&qf#mWp%HKAN#c8u<==<^^B4LjEzk-SP0<mAU$-Sj!%; zd_OL;0snmIkk;Z?+r7un_dIu$fKLIq#MxMYkI62lBn{Z(YnX_Ly-}eD^dV{Ntbv?h zOq4kDsNl*g3&*DCEgGnF1EN7z!+#=T_oNT>@dXiG)F8tebAaiHF;}P2|FAbkU*Y6X z+WoGb9-stF2F$H<=V%bCg_(FJ!KqxP)>b^6!J`o4UnZ>(Z_tOK@jg@9YTCOmGCQ)r zx0_`f)xk5`)84Y~6lR43*&3w+UEXD4*4a(J2paD<2kS>0HzTs&apQp991SOikPcIT z*wH@PW0;tH#V!Tm(__D|^?5Ion3?bD9wrbdH?h_jbet{q_%xQ=UdP4~EUfP7o%g|R z-p&0@DxUj;j>RwUI1bb=9)X^Uy`0j@7pwhO_DxcC!=a*04SCjw=09oQ1S}v?Cq^Bv zDy>=mIUrrUhY9$>VXT3qn7XpHjyzNE<L=!Zo?|^_Ii^js;~`{(Sq|S+Qb}}^b<HaL zUC6(EGbr1-LFkJOPbZd%K=ylWEvZE#Y#Zf(tzz~e3#>}1g4uhY+YMRZ&3!(ez+Lp8 zN3i6(Z2$b7b`%@daJHuD>WY2Brm{vLD46&wT}NVdt_!C)yl9yCt<ByCmt>c$A;*<k zO(bibUy9z$E%d8ZF!;tY=SfRDr%;GFm_n`k$A)PAnpMKZKKC`87kx)|2Hob7q|1@+ z<MU`(stw2GxO1CnL|W)fiJ@H!pUl}z>>o(OJzFC9cg7q~t4QT@U~J*oy?lj&J`6|u zoZWC&h@HzgZmc0-7WTxB-ey-`xhCI{9s_1a`IoRdi_I1_%q1+0(Z{My7k+n+oF_nO zYckDaroFqgD99jeK9Y_|p~&-jxrlu!IOo^oMs(_U$LAh%=h-4&AEOzxB~UkmFObn~ zd)Glz{a8_B4614^9vF$S$W{Qa&5`(y|K1rpMMu<KuX^!jF$%w@8gY0E#eXF0Ifao& z7VIXsd7Mo+Q2NWQq=0><rU*y_!2y^`Hno2)0a(A)(57vHzTj)b&(C=$2>6GmH&(?M zNlO|t@2nd+fTy~xTTgYkq0V&s;aM*QyULQo(6ipIPhoqrR#l749OX5h<&ee^!K!|5 zaPgNyP4vp6#fA_-qB8B-eR!i8KIbNg`TqR095-W5%9n-uA3R3hg<3^FBz|6ctc7vD zmVFdk{cQg()!y%SsikXh#7uw2Bx{aU(<V%swUMar%{ZwDn6Twt-)G!e@i6<TdLK9h zPTel_H>wjJ+oT`z#y@Igb58Z4S=SuC1qsh5VHhC<X5%Eaod5FOgg@v{hk1ffVL0r5 zinjXRFnJe)m>0y#%j(JkISM*^q;N6@+JAgSCU#OUL`{20bCK9|4rp-%*do8}=$08M zo91%}R01q#P`77EbbF&XTjt{=I1S5Il&a#qiPey(1%f450*z_2mFs&Vus7T3L@wN0 zRQejRPk!7b<-*@^hs~<j*SzDtUuj;f#J35o54mcZy#ROsNj;OAT1CBO-#?$bkg`xs zv0+>H(^q?D)yd5<N2=FU1U8~TUhei$x<v!18K+7y!AWxUvA(Y<m5T~%37b+Z7uT(> zGcgMYt`q)1ICqn@31EIqy(NC^)JJ~52lG)stk;D=PB*s54lsN@+kbefw0~A}l=L9= z6GYiYr|G|4KUk0e4Usvop*427t@Iwp1smuVWb+McyZ%Vm`H{q0Bzk(Q{Sl7+F&;*Q zkicfH0e+BMSC3s_t+;ZxMWDOwex-L;{`s==hsk0e_Cyvxv-aEq!-T!t&X|U$gUny> zg$5)ztYM-#L<jpAC^U&;!ja+nrp3q3;1KCSuf=z{IkE`t$PmfIF3Eo>x!q6`le;{0 zeZmZ86P92gnC@PV`nYIh*ewvw0NuvUWJ8_Cig8mGus&c@<XXhjf*Fs&vRTY<><@Zl z3x>TmItJHBr^f+AS|?mMPYN$MjYz>LZ=rh~`92JM6<%&BpcntzS))H>AG^L9nd)*4 zeTaoC4+2xL30rLjq<%Us@5rn4nLm!_l-Lq@dZQxb*z(=y8!QLV60VD&yYJ8MO8Ul9 zl^L27IWBa|Iz|S3I?m1NhH}ceZMAlwY;Ts`Bg?=+U&th=TbED%DPsmPMSlpN45-Gr znsG?W7m)UJH_Sgj=5YBnI`1o}XZy8?`k=>{Gb$4cPzp6MqBvQ(z3oj5VX8<+)^c?L zIvAbP27j?{@yN3s-L2l%p@&et#j(-nj%PJ0RDwkmmCcT2RFK}ydr4}Lq=M(%YJzZj z%%L=}{GH)mZ{xDgFHD=cv`M_|-Bk_VmQX-p0|x{}LgV6hv5k?*xxyubZ_ZLTLB$r% zF7H6f7RPotumXOqs-BRg8qv^YFS{#VATzUtY|~vIh+prfR(5K8r%aIjlxU9(y+Un$ zjb82kP``P=rG0ZA3a`v^@!K`j-M2}6h*izEf%P!s)Sx%iG<1tTefM|g-=K2KJK(vo z{NMm#$*lpP?U7!QOK;SRN7Pi!BipFp57Oh4weJ;a{9shI`JgNX8&K%Y>8$-cmoA=G zMXt`OBzNSE(Hf!W6th|e*)qEV5Vw79EWd2)hbMj7?MrqzN*+4#;=-_J6aA2c_Ffno z&(l26h5Soz6n7ow$ni!J=Xg<j3@-}%!_a)1{t#2CNfcif4MOCNILjgL)hC$`4e<)b zV@4^`HaOLwuR=XSQO$|wy3@7ET$BMcz|9J$wD)6Iql&rP0{2Dfylxfzjyw<WKp6G} zwf5FKp6>cszbwj>u*tr$At2C0GD%R$bq_1O7T6T9Lel6o_Y{#~!gZm_s57BKvsRWj zjr_O??cyDD7IN3=b@INq?w1Dsr<J(-jbk3W#syQ?FQ$hCH-_ca=7-mNGWi|&M@CH( zC{^2n^r`mv_9TNzac~>3HqPr2YKT&_MR4T;>%T+U^G26;$;Nmr5r*B54EPZqm#I|A zvz1+gD{m<fMb=qS{v~bUx2f(IB%vI|e7+Mv>%vRdQ0}>!n&3i4<RE3ll5G4K?tP}6 zykhH5{1B{N=*t^`_Vy*+ny4Jp?HQ6^S3f-L@Xq9{XyTc`U4Se(m<VB;W97h$RmW-p zmer&KStyp!q9Z=wkXD;>tS`gi%M7RpG}W=M<_5<u4<p$Qlt6i5@O5JZ_XIEo9Q8KZ z$5g^{(ei4Y%?Foq&?9nJ%&D`qJ8Ef2O@p?l3z0v;!@sZIu9YtGHae1F5v6c>o5{jX zlK)aOFO;(}v=49v-(r7n+O4XgMw}%AksFqzx@)@??1bxw2%RD}#pbTLnSlE?ztW}F zj^>%ShFQ)x{&d>j@~Oc~TxhDl!34`-O=*w{3{A8GH`Bc--SM45{a_f~WeuGEmJLiJ zHN|Fta7o*3q!i1~wyGGM5I3!w5|Ij4*TE&{btx7jOH$L#aR8Vvu{{2TzQEs^Mqq+q zsn*WR^OY^?XgUapPl8!R2^Tu7@?wz8$D1{Vy(UD$iTY7?JKawd*ED!n`4AxS^&AGM zl<tY+HI&(N3T$vsFL2qqbH-ZC%wX&}HlUI7uH2DKmYZvQTywi=GvCAerurZ9Y`#WU zs1-RU-To9$Xp*h)l?UQJ`cFcT;<1l00PK2p34y||uR<ow_NNu|K86-krb2N?vW|IR zOfiY`p#d%9i*)D<$O7h!wvH-#(9EP><Zu}+-dktkFdMwliPlkX&@ZYhrz|3}{bmOS zTZJAO1!~RwtdiS#*Ei(@lpm;lwf@6+NMPKQ8ra;g-U;L9D>6U*;6gC*!nlDzMcZgi zzOch9STmDeRGJnh6bh~o_lz>wH(&hT4-%PLM(vJuvAVoV-B|5k;+!NF2{Un537O@U zn!mNUqTvyxI60Q?joPZ|Ei<_da8~W0z-O`-l!s-HxeB>AdF577Nojqscb*zRWE#T3 zE^EjhMe!inrjJaEiAhPZAolUg>p{n7!zL1yW*z2bO#-=JEEeaxiIN&Qtqh=hE*$WD zI__adbKTobG$t)@p3ao>wLIQwRZ4^_Lh}5iqn*=0|AAt2<_UZZRh5Q6r&Q}<vTopN z1&nTi3)E}QjZ-Mk8s{2uLzpQMaE7~qOG_w`5UB_ujmNrMBxVW$K-Vn&95Q~h<kE~y zTO)+b3T55GU<GBeyF2C>d!5y|HRiZpsXye{Z^$3}bLJ+%`H=`H1kmMiyY4S{U=WKp z^fXXjzJ^s5Q~zp&Db1`OI1XDa3*DpKt?-NNzH=Kl+x$TL$=}G2nfWpGeHRA4s@kNx zyP-O4t=Tl)yl3sL@&)voZv!VEl70+@3;yVN(<IlPJ>Ir!7sZIXA;ql0MmqT&;jhb` z3OOz~@hXtuVkm^uy4ABtDEMI&=w$a(BpKq}cweKjlVgnbq2BoUy)TkxXH@Skqw$_i z*?Sz>;vwaKJ{PdNc95!2nr^4Jn(>dmE2kb+0?*f8=@o7y-vL1k)peIuNdwX6^;Q03 zfnt|z9-LjBq-_-8Wh`)3Wbu<j<GwKOxHg+TbIc0dC{GxVOC;wBjBZYneuxhx`%-Jd zFyf|qQwLnb197A#{|r1X)cBadM)gw|E5bQ)vVs}RATBr#4{a8UHdPOn;bp%^x9=VC zl{a1L23gD6N4(~chXA8_7GbABlhKp_qD7NPBo<MxI)Jij_7izmfe#C;5X;URFgywr z^`5n`Ywy<_RF?sc-TDU4fb;0NlmTh+?w^IJwko>RUQGv?<{h3G%!9uC{Sb%2pl$ew z6{%<(K0ggIj%<4RDg8{hb7!3g44?7i3Bd#5Jxv~K+~fHgC0n-(e0NbTo9Em*9z(t{ z546u~jHg`hN1*z~gzigT8DNMAM9|gk!!Y@{)uX%N*b+cZUQ!P_HAqSN3GC9}rhCA% zS>D+C$YE+{i^1FO1TWMIg*7I~unLxa{4!t^v_xbG|K@@P*5s|3A4Sw1gn=se@qGKo zV4EJ$HiPEOS&i$N_+B&D%{=M$TanK=E0cVVXTxdP-+oIy+!n%N<ByYdKJjOPW|aNe zb5n}cT4%DaG3)KipY8v$089hYI44*u$Rz-rhSB1#{qeDMi$|Kanp2rYbjRXLjY|@E zfFz~dba2MLU4N*1qVImiU+|U8(TbdJb!`&8wz7eB#BTp{r}1_C+QliEXNa)?^FLr$ zIuO^bTKG-`oJKEs2pI-7=S4B{#wxAPCIUd8OJE$gNK88$Y*XFY)ys`dLtpKOVjSg_ zQf5`;zL>$l3EyB1BUPDe>oO^6w-!6+s_e6!Q5IL<GO%qbA1e@BSUoG$(9tS)3M2u2 zhU<pbqj8syy6*ls@&Jq*9fB9TVZBWT27^~s87Al7^lChWK~$!_%Rph9!0iHWSbn{W zpQuk5FEyYs%q3(iuvH)kG|zc!wJ^{-Jyx4~8qbKWLDdWW2A|-Ib^o$aHVYFj7zVZz z0L<(-0`O4l#sQcz*cnJ$m`OS`{=jJSUeU%`{|8(j#`16v4xCZQ;mWcirIWO=)OPO6 z_$;AmbG>cULLWGdml+mbT_H<3V*p6kFI`^<Hh;Nh3>xDE;eb<hmL+_ij(<=C;mES7 zyWc!J3_;!Q1rLYOq%op#C9@5rS*mH|lyCa#pf(fQo%@b^Ma*NRzV#r1DAHBR)|5;` z-UB%h66;43Ci1`DAPZ29E6<2qJ1L<lv-EST*Ho%pc>iA4wbWH44{%{h3ZDSgywYMd z{h9>s%_>YQ-_2R-=d1XfVgF@po7JEzBBffqn&>d_(D9veyIb@XjG9n|u-sYCT#M(G z6KRV_c8j&TL+=dXlThysN;WarvG46jOS;dMlkP1qHc+VMtk;zgc~)NFDl{V9!p4rA zsip@2cN_3Vk%rGnW-Mj~H;8ubuhqg>MonXiJN0ZPi)EzQfy1l4_t)V?q^un6E(+G= zn=-)LQ7IRrVPh|7yVqSmy%R_vusHl=ljB>QCj7E6W`BR*xrn^5;<1zaQe?@lW#oX$ z+m-S(GgChiK-6Wlgi2znGgNg{6ig)NaX~^gMepOPFGH)YCfNb)suwm<1W>>NJFVUn zbWL9O&(s4hml0(WBbe`LrPgtaUX#x~=s8dZKt(Eb?X+e1h{Ny**;TC$GoR3~XDAG! zS3Fj9<ssArh%IFXgk6G7-betDlA8xCv>b~O?)-PLSS<F$xPe{or-&mvp#4A`ZEaU| z^Rdwu!p-O{5>9AaBBv6lL*J?<9cA<w6-}%`FOTQq6zUS{DHtzY5KkS!047T4W-q54 zT6l9}&A0G2{A<ff%h?s?`_r|~@{b|kyuKG#&%ZSkO?u<(VH~bJnP=UU$rdaTij4K> z*V^H|#oU>_%zE+r1Exd`Rur+I|LWTtm8uB9GGFTDH)u49VDK7$j&SeH+u=xF<QtA5 zp72@RU=K3GWoOC|g<}?k5ETX$)~H)<rkLa{N!LBsi?H2RG&=sw_oWly2i(S@xh2O1 zQTiiHkYtawAYm(2%-i3(kaZiPVlrmx9PT^DX0%!ir%XVd^e2xMNpr;rp7t50=w$lf zKm9r3?y6q?^^y-OHQ)Fm9QJ@*i(-j<&Omj)GH4xgb#QzdQ?uyklFj%kYjE31RWAnV zuYK<E1=QL@Y;Xra?pWW)fv}PG3*22?YfwxkPR1RIO~uyl8_TEL4XC}_7a7|XcC?wx zZ-u*YHqCntp;Yk!h6FYf>$OI7uMc^QEzkz0elBI=qkj$hQ9v3v!xH+4ZQgfCNh*Og z@-SlJ4AoRY3>x`uKH@a=Mw#qj{%>>abRi;|j~D)o7Q6$5sEJ+Df`4&ko`oP(oK(2| zK+G^cNH{#O6~7mZ@ot023Ce#SD{e)19UrN=_&pKZlKkD3DHgbolmEF39T@1P8(-O= zH)WAXX*tfqKIgG&F<{|vEgqvtSUa1&HUr@x$6;7-4P<%wbe^wdZS;BX+#2ACIvA(U zHgvU@H@t;ln22P+;e(sd{vjD8L;(<Rz!)Y-!jIKDU6Ko*Sv*_XJ|XEs+og!0vVS3n z!lOYiRIZ)DtRL0sAoeZv7n+m&>Zh(@Lc;BPTrGGtnllHby*^&zrev;y4fVD!ZZrGU z*A@zNU9zOJR|e48uEM@?w@^^qFs&5tR`NRa4%L$~jd|jR1H1agNYCMwO9a2Jd+J5} z?G(Dn1KV<%Hi8G(uR%G0I@mC!QA7Sqj{m0a0GFqS9dJYz*r6Glv7kkS0Pj%-Kj7~# z{EK4n_aAQgf$+(rPsF!C5sDHB?w~S77YO+u5F`Tl%)h<~O9KoGaMU$c5$Vx`&_}A- zsh$A7%>NpO2o(v;2p<{3XX#<5fK0#-)ciNi`aeI1=pP0E#2~}R-xk1+bWj0)u4~7) zrvK0U_z(aw@FVOwNK;~lLwH)@G<qRS|1+<DCW(j(k=75HPlCZhb`ZK0*N&q2zp&{4 zm?WYFWZ2c^kM$tGH3&p~TJ(|UFLD2`Il2Jj7Q(-q0<+702P{4Noern9pwD9%Q>K7$ z_}6#EL7%?1meLgi`=bv%=lhUk^W>cM%XnN|G5_Os|9|k6feB(FLG~5tokj>P>|>Zv z50c3|%ju&R_0!|l%ick7eImatl*e5oXX*E)Q;(>to*3r6VO!U!iBNR2UiyboxKaQb zgpd@-nz4RHJajEeUJRq=0z?I+Y>~l=ooDykI2Rjju{Ez&f1j5tHX!LF9mtMLTcuSC z2KX8#GbjU`tK(Um7jKuJwXjz0`aiqj`#ieP0Q8USAJZi_dZIqJC*NPLfBx<|E>%o< zR%wtC6Fkau`t+X%VBH+ptRwUIBp2Xn+KB^(N4*|%b}>??CQeYwy#1!qSKQLQKdtN@ z+J)tFg|EkW=*tsm$A?N|`F$Q!XPF-3KKQWd`O0O`F+BNfV?zSq)RLJ&ZBh4eO5Ld= z+gDvY_BZ3!$alGIbzev1NJC}q&0MY?%^S=YpuIt#{v>)-+$z-iIF_kbw~DR^-LH== z2b*RG5yv+#L!#v6JF<n<vtDH>N&xMoWbRw+qmPCx+Wtox-}~$kW=#qEE$Xv}XDNig zo|S)2gO@09kJjuGRRIy6ML`k{6B3O_6pam(+KmZ3%71ujlC)or(!E~UhGFyoIGKsR zzJNC!eiz4(<OA^mWq{2?Z*K)pEtMPqVQoQFuC-;#KZw7M?rAwrMhf$^h}eYyRf+z& zl4?EBKC?(MD_!Xi!7rr@D5EQNrflv{$`7u;`ZRLrQ1%BaxGV1cFNZQ(FI=*NmJ&El z(V77&+|<!VXX85R2H_zHXd(I+z8rd)OCPN{1(1M{z6}fA|C<{RIcreB!4-Rztqu9x zIY@1*cGnx*1NF+FrdPx)7vW;4u`W-;_2xj;ZB0qf_Jap6uIWP{ENuo!4Wa+?ew$hK z7te%szlAlB89>%5@O*xV*Jig9gukp*JRw>7%eZC{<#6om8f|}h#OOT`Xv1Z2At~n> z;0q&TypPF+wtb}e21Llyc4Cn8QvTrOk5d(EU1fE)`W(|gT@Bl&<A7r1%C5=iIDt6n zg;K!BPnXnt4+}UFbmA52z;i8Utz&9?_}tkJ%lc<=8)fk`2f*;brP1~B<cJku*Hb@r zXLDU0Phf5l7V1l-eEoFwz`^5ui4Z|7NNGHh6l*vTRXQA=>ZmrJlQ;0=53_f23dv54 zFf4Mj7*6P~lQGLur9ga@$CZQW0vqmF*m?<op0g`QS`4t46s4&7^tfLlv6tk`;yJgZ zTRGsTxxFtXsQ>7GK4f7p9En@lXLo+sK_eYL-O1@b+@)zOBjEQy72p4Z7sadMoj`FJ zETgwsm!0`vF9{+O<iVNq`e7CX)FDCyiam^1W;g*h<8cTQRzGMd$$6fJeMo6+CkwGE ztqOU$<yqw<v-LBWMfuUcX1dJ_%X{GxZB1diq$)7HKl(Uf->6|;e-!~*yB{d5@x}i1 z&2=%hk;hE6P*?^$11HlmLQ6=DZrw<4BQ6rO$Fz3}*>bxCj+)J`bY>URgI%UettSxu zUy-kZWyL)IHBQ1SlQWhE4!yF>@ZZ|oelu8UM1$@t0f6oK09ms+X5)Tn1wrvSpSr>* zyX)NLq$7<+&z^X);+;;XTg>0#{FsIic*_Afsc+h>me8cjtus6^?gy+k=!KFAt33FU z%j?$}jPgE=h$1Zqn(Yk|0SnQGzGn7l_BV!u(b5R6Ugpu{vtK#Vopb4a7=P6%w8ysW zTv<oL1x&*7lWXTu(pefgA%)->YS^Y4WGxW7Ud?sd5E*K@o`ddxy-)j)_i4X2DH*uq zh_XN?Tm7VtuLCSoQUNb#p`MR`$V!r5JoVgYRaqNfOztiaUoO&y(<?w1B^`YGAa*2& z)SJ8P8I3fntXHikEO@l^2`JF=%|do*f!Y4{@o(n-go3ld%(OC+c~!;tAs1U=8Vuj+ z!3IDN@HCkF=mY7~Ku()9cjD*84H!=+a!L<MU9CGFSvuvI4ksY2+u%HdgF=drOo`Z3 z{pTYe&}@#gP|bdJJd4jJT+&Ej!Ahen?FMba({^`!f$uJY7{x?irPM%LDw!UDKWjT< z$_FsHLX?gJuM<DEo}U12GW%HZK>zI1@SdapkgV(d8u<nm*p6=CV+D2L3AW*Pdn;k} zY)bc6n@Cm9X7Eog+xdZ4yQNfuv~znVi}aN)<xIHD!XiAEW9W-@Uhyv6A&2(r#$y3Q zJotGlmE0AFzdQa@84ImL`1*Fcavn^u(D6b&C?+|~S;a7S{e37^$*cH?OrlFObF(Vm z#Z14dTuv7CH!RLFClaL>aSq=AG1}&ef<0EFeXHb?s?d38ly{!Vg6cHUhpS(8nYGeh zI~@>|g#&K&)GG6AC>?U0zG}XqY}r&(=nVP|F|@>XNM*$qknUQP<I5!pW6D@SkZ4J_ zx}_ekv|akC*<Y1E%(*!Bsf6*T!UVx!Myn%-eAnhsKpI`ovht?k@B#Ie0&mH0)K+A^ zT9Gd)LWt&8=LBZU;5(FReG^SrVfD<(@@N~E<vQj@#1M?{CcIq|4D^qx8WxL2VFD9O z%{AXa*K*X}sDfQ9P2u|fcP7EFlO@fw8ag*X^)=r<hG_je;Td8n^#aCO{d$fOQ6A<Q z4Kx&E>Uj~-!dp*yT_J$-dlA7{_gREQgpal5Yf`DAAo!3#j!+87W}E55f{ro$o;+pl z^-c|~R^2c6nYsJA=rfwC{`cpbCMgpIAO>~F)eY&D+Jwy>Oe|p9Ri|%KeN*b7A!ZoJ zR3P~6$Q?yJ1(d_>-&Cb*fn;9;58>Ym=v8|IeXfrgi&@7j;QQ_X-Qvjk*Y*&>vNX~5 zPu|{6JGYZ=TB2FpWxMcZp5F;rf5HQH#~dYA-$WkMZtMPZ(R%*G0=0Sd#QBGk95oP& znD^EaEWR^But4ExQS46ajtgurTHsVWU0AFUSux>3&nWkB(EiW6N#O=DfA}2fsIZ{@ zg(&pl*zu((Y%YQF=ylhBaPI>jZk(?_sSouy*Q!R!k?BO_-s5;S&zx7?LJ!!`mw}Ae zi-=SW{L3!B6a9Fq%l-TWSWNG8-5J?d9xD<K(6q%Vu_5tC(PdCQ`f9!YtmQ8Wo_BaT z%`x(s@+6t0yQ<?^B3i1TI#_lRqh{p}31O?!KHH?JSs}oR^L1c>7VJ!Guh|(<TUIB0 zu-j@zr*d@mlFVr1secR98m5hFNee`c)lY6dJDhfzge@nEu{IHmxpmgDNOAyp?gB0H zj^fu;U)p~|=N&gHo&Eygyt20_wYN2+Y6GprOO<d*71|lAwku=C{M`R#j$%Sw8kFqT z#CSwtQ!wKKh=zrr_0^Mme~e}XfPCsDt_va56djBWvVEq>?oR80Rh~TUG4#{5%I7sB zI4+4HVfBs{hRmP!$35qX&EJVtKZhY6(|6|$(s{o7q++;X0(kK@jEf)x5Oe_1NvKQ) z=KFPv@4Y8LTfxn@p!+_pjx=6*$huQ)fs{iQBwU-FM}EuRsMBar#oFnU=J2n69#LUb zLG^sGX#D7MZto`mEKJhQrRx1r4=@bm4!;Zkhkt<RzYd9e*o_rOi6h?fLb|sD=1nO~ z2$h?+Tfie#`Q~s!$UZ0qZVe<$UeY3EU_IWSK0x)dB$4pYG*{)drRKG*>a}&`7lskS zWnkjZ@K*Z)eoR*mVsXgn@OY-fH+$%{*Px}>_wk$@MOVYJ*IOgfqtKw{iApW|abQ$< z^eF=ivfUXWSoWe$P1AnNBEEry0la)WblaVD+a;*%{b*`oZ*13O93_Ie-r}$<CEWaD z3;yNh^g^;2)-#le;xU1SKv2}FI_+Nm;;wf>2z$-Ln{}cgg%keO%?jCSdajr`s$&^t zI=k!l^Z65q5%~OVPf95{2Iub!eQ6@_IbH*8#ltossdVYW>%+`z2daL7jk6F#j`}s| z7Z_;9Yq#LF9o<}NycR!Ox)0NS0LsjU<%%uQ4|%3M+%2d6p>Y0Zrc!_+aN~!&atQ!N z1Es)>CGH-s83D{2YA}XazV+u|!vT-y*4Xlf$6t#EaxmTdkAX~sa59`h`9#S<!N>#D z6z3yzhph}gpyTzG#MJ)fPx4aDgeH0|OR^Jq*^@c{7?Q^j5fe~e15v%^a8~Drx*q_y z7K%`>+N+BF3mf>y|09C<e^w&KTp~RaNRayUiWYX|Jd4mMGI~r(e7Kqei)FJ7Spws| zceRwK+y>bKO#rLaSm%Rww*=-NLROJTzV8%iOpZ+o))!E)3=)%nRLu3J&yx*@Vk+y^ zbOl^PR!bNB%6uoY%Xz$mUt#}h;Mu0XeeF!hvyg|b%^y#9F&!_;^GvAy0poZjF9Dlk zNO?DYst;EkShu|lixVoo9pZ_PSoHBBTF!n~%)J{7V3}`@JdDfkgKPVL{IdN3*oX4u zj@SK55c}Gzml5C$7cZ-8!xht6y0qfG&iTNZU2e*ldaOufY)~6^8~FWe+GB8o(`CCi zkGLKr)#S@P_DTAvTFky_^N^Ff|NpY{+#q(|Bkqq^K|(@UfSngrf7V)HX|lJ}Xddm- z3DPRhqt3T4l619pB4tmg+jbu`PC~dj{n;i~Zx)Ijw~ikXFt2M|HdoBXaTrWzmM1Vk zq}~&)a~&+YBbFIxxp;)--6$C{L5o?L&S2T{rBq3fdwqw72G0)fvU}op@FftsQAM#D zF>Y%XffZGi`+~6z-Sj=(ye-0|Ojn6LwaF${OPb$ymx=EJSJ5W(>em#Z2@pn__5$a= zWZMO5(PP8U2X3*OUR-I|^*?d3bn{;nE)e)kXfPi??dHB<)&Zd3@XY?WHSlvqMw7mo z%MXJd*0o{R3KZFRw7Qx!YhgIdT>?7?!-5#(N^^^aNx{0<uX%iv$}RqvGWZTr248UK z`>~*X<REt1nVr||nJ}#|J>PLl8+-u#W)8rboBxbgJWScrTUfCY7cR$cDFArhRk~af zX5<=l%Jh~^w2Jv$0F{a?hzn5|#=EAys!r%Dz>v1UW4kwZ>ECLC@n3RnZ~h6+;~&Af z(TqhP>%pU(<yV+3K$>}d;S}u%?r~@&R{Xp@ODFbUs>%TZ9XNpNqm&SDQT%i9!H;lt zo#dE#BI&9H%hmV=JDoW#VT*^i52oWT8r??Z9tGN;eNU^_E^YsvtOv<>Dp<Q@h~r_i zbwc!Rw`Pw9*u(gaFr}xz)c)(qJOfY}(Bsi>DF_Ba17ILGU@wQkrp8gks?e}%6L^7} zPg@hYSS{0xgo3Fd;HM~{p3MvD2_Z@ItNQ*2*C|NSiILER8?LE5Ng)>LYem3~C;?c0 z6SpT0y#Mt}yO0d`-A{TajIeLWA#lc2CKdXWJi?7qU1BMc|2Rw~#R!yPCO}_X5jwng zJU#`OHhbq=!;n1siAHjN566mjpxjaEQS)GSS-W%YpLn0YA=<y52cWMcAXIQYgAq_h z0(eE)Uovp^<Nwos{MRco4+!4`<_$Suq`Q>?`p>h>t<e5IC;Bgp=nM{+qnEJaA|%eH z4BRYtDXD|MvC4ma>2KVp#}Pn|792&HIq{+6zXHfpyu+gL<bS>DkUs#0NCDlXKbCT- ziU`XCd>YytzVPM$4wD2B$9hSY5w?$%>k<&ev0x;w>K}yZKixV<U`pJ(ZzLe=;DM}z zN4xOz|8E_XkaZv`rvpeCg$(eYg2N^0|34q>xd50R3m4f?t=<1s2<d-M5%w))@(zpN zrbH<8A$wk<OhZZl{lABOgg}>K*{#X(K#n{B!llKUYydGR-2X?|Uxmf7b#0??u;3ot zAtAV12<`+81a}Ya?(Pr>5Zs-{-5nBW+}(n^Yr|ix=Xu|Mec#^Kwa++cTB>T!ImT^+ z%9O!_!drr@{?00`*>jf*aKpWWT#ktjCGFxwV0{>>J@#9c`}cbI2i;o_0WPe~`fpm` z2YSf^cbAOZ`)UufzT}LsJ(&~!HBkzlQDBK0?dUFGCu+}f#R;f90M{MIVQpJDAMD9; zhqP7_<NLOxQq54Mg5h<RJ6}?P{0t(s32r@`>^0wZSzzj1q2cMbLhd0IM8^?6uRA~B zc$GjuSu21Ik5?*q%SgQFa=lS0=#vM?rsX@Iyn-rTJc1|9+{0`Bj!Fiv(Mf*IlW=Z2 z3ibZH57<}H2Eph!2VOvdSCD-;S1~!Du$xvS(yoxe@kyn4iDB9vpkVvHF4h=gW<c(X z1rk1)k<vZgw7v%8RBWDk+%E%ppvFuY5Br|L$^O>l&ox_O<rK#|qmJK84_+B<|H4|y z)n9J{h@E~5aE<~?jGYWDLyl}3OD32*%DGzD>bsT$AT`PjZIz0%%lF<w{g2c)-t}LR zF;s{e3)+J9Zae(#bgR8~V0m4>3p(SoNk`IdSW&Mt1O7w!59v2Khn<kp!Aafo-vCM3 z+oU>{C6SiII6fHvUcr2(T)p#?3TSRQrvlRZUjXO9?!O+hBNY+v5GgKeA+whIU1#sn zwvV$wmS^azO&N5U<joV)p?O(yz3^;0ezdKb-D39ui)yX)4^45xX*l5#YV@?R9B#zX zCMx<Y^+!)irc<Z6SWpz94%yll;u8%BObW7U0o3on!)@ox=WIDKfkj%Wj1ibS(b3y~ zA^fi=NEoow){<wa6C`1ry{_YXeM4vs7Qi3)O32ZfjQq@MiFT(ny?Z0wlsezyK`Y#5 zG2fikpoyeLwbAtbp{oLR_&u(2&xDwe-o;*)Z$kZy<QH&i$siQ#;X%KnF&-EgJ1m>W z%Q>^D2taDIasv#nj)m04&Y!rW`Z|d8fD!yUq^?9}xuGavq4+<=fF_s`IE@<&TkRpP zshn-xfSZrMeXa0wBIw(2%eIyC)B0#R$Ij1@cbpM<*93I^GRabN{mCyK-oq`UUXDOL zzySz{>D1faah|gG@D;4BwLWsA+Rh2FTTB-S;<#;DM-&7xtEc<J^|kxgCVgr$yCe)u zRxOn9_ISEuwbtm6(nuf4dFub7RQ6*6?e_ocnn6(|8w7gF_ajE;fuz@|U5+o{cwLpe zWAL)rOvl9Sw?5hlMWH_ANxpo)WxVVwC-R-N{l%;_nZQg_7GwM{T~J%J55Q?HB<e2d zybpmE%sCDzLq(hM{nO6+aSq#01YbZP`E@MZR}@bJ2>*9ww#uvyA1&1$#8?8WG<$3? zWC%jo3&zq{N-S{~17tK0@YqZ43|}xnexzmg?N5to6{&SGMzUrx5V70m4!IPE9Tgip zX^^8Ej6~C|B+w>CGO71>;Y=m&ZP-;m*d0&0^Lk$37s&q1oHQB29r|#3GeXoN|LW!Q z>FGoJ75H(pz~pz5^2ihwZ;=~-BnFeGwm<q{4;2mI>0NW*Bb(DqFPA-Le!i}482Xb= zRq0?Gc`|+jL<P`KY@&bp^@&J4_T2bc5o*@SEYSp`lA?*^*<*v4F^lu7J%4_GoUw&Y zKHz)AQY9u-B?EWl%cf-lxoN3P>7R18It=9O?FUIK!tuVnY<>I!>i^jB78-39M*FS7 zN?-7t6@YOg*aXn*!HY41bogM**^SW}or&G9t+mA~t9RN%0RUre3NMQh?L#tP9@ciW zm~aik3jbn%JKbho*qkS_SKt*~41}J{+WQ1{#GG9CyObP#vd_d_0b~{Ae9o21s_a0? zgu`8=x6nhc);(9;JN{?()73fpHr>Gx8@P0D2Q|`(V(OP7P>qVt7|VQW0?k@1ENt)v zsi}X#rPEE-{VGB@1f1O8MJl;cnHP;+&Fl!KFZlUmG0G@3LyI~(q<sE$_>g107CTt7 zH7H*Z?l51671?ON1bR#}7Hz?NP&|?G?7SOOL9&r6e~8T?zSpZCe=Q}E5pSX|fi~#- zA?6gDiBN5-$CM@@8PWs-MP}Y|U1ub`sWO~}wXW~^i1l7xgnwWp=iL_?5mva`0NW&0 z`_w`Q6^JGv?(-;8g$Xt4Bz_48Tyo~kbsO&&tp(k%+LMn3=}BBz%O*fD-dwxkEEq}i ze$PP4aJq}F?%_Y)8o+72#dwaC=G-UEhO92H-Mqs0?tq^mS|6&{#-Z5naC~rcyG-l4 zbM=gXF@rZE^#&ibC8zOmI?YF})Eut?o!dvIYJMNU!`(GInKhcZ+Sgi$&X;@Pm;s;} zFvR)VZ`!RbmbcJF)_s}1wld)WGOa#D6*k25&Q!SuMkI-k#R2~BKi*4PyZ6yL7@J#e z+a2WcDhb730nR3?Z#dus7DGIrf$!sh-t=`lo92&W0h@-ICjiOZf5mRcCu>f94>4?_ zFw7UxQnQ2Q!iv>D+o^FNFH;Ve;p-63A?N<ZqsN=FEErNARYOWsAz2mu5ge>>U3g4m z3YxdbkV69I@lcjJ-&1--zXH7!w6H<H0Wop`{O!pnozF|NOnC}sQ3aDThp(7&@oQvc zfrdI$O4zU2dKdEuOnTgfQ+K6)s8?v>oNxomKxgCA!6yS!vWtW3Gk<;#&itP;hin{g z@xFo|n-n^$f>c<-Hv4i>uic-jBmj#LC7OoGT;a^sdH%+?B5QOyU|fa3iM4kb>pu?m z|CW6I<Uld2@k5~sMFHUsuw-F_yU>X?UyV<3Z@ar<={8^nB^b2LG~T^x!0(j)7v{A8 zp-&KCL^C7sT|qiwdn^W(9x?6UAJ5*0$($<vLcwA(TQ&FCnoey+vy3MZI5<fVWs`RI zHIleimy;C;?xVW0i6X88#yA)41oYM(yE_FCup11=-WIB#(5nA+rL#?ZwTq%CxYEBs z{oJ+8V&e?p+V&s$mk@IaK!6x1o#!c|K{22h?l2p;fV5>zJg?MT(eWlZL-wobYVcd3 z1V4@!zo5?7Tq1^E^6dH}?g(9X2%~p95X;73?ZbN)eXz#Gc+p-ERHJQ?PCJ!9rRvI{ zWM8zN3Ln{gIn;tGJpHvLS_PN)Wk44Y*SFr!Irl>ltkwCnA{DNAuL}GADA7qYGk%Rb z{ZnbGSH%TD%4Oy2UFf)`nKfqgh0<jm4ImtfX?^P)S)InT+_J7S8mk3PX~dK&&Gh9; z7?Q7T<UP1<5NjN-7o>6>7SYHKhSH4|_dL<PCF`_yd5PCxpKTLh`kA(7T=em(NUM~{ zJlmGN@bckGReVEoU>fDaDIgJL1*LO~w=#)*FU^M)Ru9fpn{}^>2nyD5{h&?dCV*E5 zOgJeUx@yrubT0l`D}*^z@a!wY{|lUOm{y4aSKTmH;=Dc7f^tpKl6#fTlQrwbp)*D> zD<C%Q)&Hf~tRhulOfX5FNDhG7H>xD=mw_EC@mj+Yk^!}W*T3|zob-Q`So5zX))70y z7SLOWlas@-vkUwnsOxwM&7wih2dCNrV*ds#5t!SA-!oq|N^*d>UUA!|bJ7Zz*Mx*% zj)p!qxY{99t$na-ab%CGY@rH1!xw!G!`PYrqz@Pq7=C2IaLJT(kGg<-m4w0532q5J zM*8iq4j|t0vw9a8e_Cho0C4nR<nX^!Pwt*XNkN;)sv61-7p*T=0W>s>xc1&n5i$FV zvdC0cGrJ}F1J+Fu2Bs0;u0P+*5D?s_+BRA%^(gu>D!LGGijv)YW7*>3(M|Zs3I=y1 zAEH{j_dZ`1Tz)_9K~Hr1BpR*=SMJ;|e5~=t`0#U>0Lcumm)#3#eWj8%a+y(*%?`!) zxjE6bqiOqoa<m-%W8tTO!)$YZGo8Pi(LliG=0Vd5<NkF`Vu^6k_@Qz4UH9U_6qCL; zPktz_#S&`DcazvFVCrEQTs2~ntEs>Gf>QW+;WlxQIp3}JdTF#(f?cDy|Mf0Be~3C$ zoOZ09@0}^U9NDzoT%cuXZO@os-`-)T&I~21*xuk!=}*RI@;z1MSr6@jTPr!i7rP&T zTV5i3<wb*UlW#dyRG>IVoFB{_(Rh6=ej)m=k}%HiwNKNJHwhEzibVisESMrGfbBPe za_Y>6H4k{aO6&d6Vu2BIQH(?5T&)(VvjRM$6;ZGL2{;C27IwxEMLf9fQB3BjrCpan z#z!TMwkJH_!z70pQ3hIENjpYl)53MXtqLU2Z*ryi;Lp>8g(PYA>f*v@BIl|sTh$6W zt;01tQ>Z0mRwhAyhhe{VEW1%0**9vfbHqf!oPF^GS-yFr;ss_D$-+_locmOyEk4&F z{Ll-%Ls9H>q>vM>)afFUZW5>2s#V@$%F$}Lcv?lu3Ot5Xa&Vh&0}@42Dsv=zca#o} z0so&Q#U1_yx}kT!HX0Ca#Sv5r-@Fdi#6%9D*}<FrB^_ThiL2kl9aO41l#9u_rid}e zl5g$|L2)H!O=N3NkR+3ge-q|i#TAH?vjUBhNZy8St)(Ye1TkZn>B;Muzkx|%*|-T0 zcqYHWH>Atee%MZknF;AgT@!zWcDx(3);0(mmf7ciHeOG}{Qt9;gb>8uzV?S98cD23 zP%jj~q&9+OtPK~_r0byq8-N@pVd(CmN~Xw6(omnTAv1#lGbO(Uv@8Vv%xk|N^0vxy zAQA7n1TaH@0=N_yUA^Sj0OG;|<($>wQqVRE!|>IKozg3XL8Y13L!bLbFC+ln_eZnb z4=3m05z27fO|NMhZVnx|;2%NVQTP2@srj2fx*1*fo@N&LFEkKLM`v@mk2XRU3sh-w zDw#fZv)OEGZ+jFkyHbeBpUlMpaa3}@dH+D|$l(b$A&n^}DPou^k}tpWsEw*=D?8wj z;&;5bnn}@rtF{a^V)?0wGVR!2yOP3DaH<-jA0E65Z@(qs(<a>9B7j+M7Y%IOm2wCL zRnK6c$LjcCnAXjUUVS8+gUClNMy+grctlBQcE|8-lf+`yvUtH6^4zMUHvtS>6$$se ziLf#V*&5BP{;wH6Vg248OH}J%csG_H8ykvem=a0tC@v#)Z{9S_K4jC{+dJ;*Unqjm zd!RMfc%N)ckc)9f1sx~PvK8es8t_~z)qlBFuK#wU$LZ&mOBLo|2yw1tsS~qO>yW}` zY!3QOz)=jM3UwbAj>VobFc}zV3A1M|i{kcO{@m&x;<MM1M`C3*8B?|zwSKAm+@#~M z<Ypox2&)}>jgD+O`VhdQh@mXIW6*5egNl|L*lrh9IU&P!fUm}oyr8rGtXSRgz)@-i z2ibM$*^1wnWw}h^AH@H!4I@^G-qo<LMKt6~m@58QdK@GHXMJB$_>^LqoQf()q|E{P z?Nm+p^fXl5|6~CS?jD!1cOPv8K6<QBY90EeWQAD7XC>|*-dg;Ul-GYFT$YV>2@bfS z2uM`RRZ_w!L#Irtg%(CZ<XB%tu;leLup9(3JrBTzH<vqRu9kCPRdd%`&ZYHRtBt`; za<V~sEJhS-ErtH|M`q8Dt!tD!6ozD}+rF(07Lh%ra*ImMi_urKwufoQfBrXm3qVU? z*#R6Sz}>~fN~G%r0T$4fr!6AVcT5VO5<rr+Bb~_40qy?=DZ}C!1S7B{7Rdw6K1}Dc zq>B}3i$qShT@-$8w`Ekf9S6)~y7-xsNYOd5xQhWpl^hMg7i3$ZH`4**(1Bys!})Nd z(()am-ETT8_d%NyyPv^frUS;JCJ67W=>)2A9jKERwh~h8dkaAe;y}Dn-3g*)UPEHU zr@kx0NP#JS47@6$_wmd^$(ua^DFj5ZV+(rC%s~92?E;knT)-R^7l|$=5ejP$)3ZZ- zNL@vlAeAXG!yqsS*yuFkn!s>;#{?M>*XE{LBeBhGOO~Z~JAGlA^ze9cVTz+^<`FH0 zS>p=vut6%OyL$J?Sq+>irB|VCXkVR`1#qJB;BII}?U=Y@Yz^U05@dS4)jjDWL}sIm z06CgC;f$cSd6D>&sg8{8PmRac5D=y?!_+Mm_BBsK(mOd8hf&9$(3zx5<(fXS(6k1{ z_V6<Y4cY%_$$%aNqwkgd+WeE4g9s(~9-}URz}7;p46VtkJ03nsRY8VDG^E3nPTwto z=IfP{QEzN?=f3=J5h3YB)C(YM17u<k05~E8LqI-5cFVEH{uZzdem~7aAT<w~Q_<*5 zKEL0D@yMG}ivvbe7ZQeCv2f}ydmA;DQ-lw0lPsZ8QRxV8k__x<qk*8}ijM!PQhjWZ zOs(QfcY7#tn5we27$WQhgCmKNw5U_67cugXDr|len&TzTQ=*>tBDH<}(ztN|f7_pO zw#oP?sIv1%m+taC1GWJLmshuDhB-E+Wo*l~6Q14j5uP^HKL&>>vwXndFvq&27}qZZ z5^Q45?SjKIe5I2bH?8)u4W8@(TL@hCA40k6_3r}8VI$YSB1m>X<PvxTDBZL+PeF^( zxYu2=|6zAxBez-#MiNv$@FgGo@cud0t%SO1J^g*+%5B2|W$t^}_a=BGLlh<@n|RO8 z<!qx{106emw<W#in9a{nNsd+7?8u{Q9qyuyWPk?1J0ke2`dBxF$7R{=;m#T-5X1iI zOg3d~K4Xcy-ji;p=5$&-L10*>MIatlh)*&XemQLkKrI&Z+?fH5(f^eE`T&Bd!7^46 zC5LcF=AVrN#*jf~tyf%s3#axQ@`b^_0577s+02zqxlNcvZEzueh_n}FE?Yk^i)yXR zNxFHm+XSuO={mrTvrx;CM4veA<W)UHG=$@Lig)O&GsQNf)s2E`w<8sX$lR%_ZQBR9 zAZ219fL5H570nq~iR8R7rxRb`Swk_B7;T|u2F!UK&WMgp=Nvb^WSoS+wLtr4!HSUE z1rW5809t^_@VzM+{Hcq7RCt>1TW(a$NIxYGvW$Mx2Uj1Fr;o-$eLUUV=0kW+wY&1f zc^f2|O=meaF=|MbGWo+pSV)jYWK-ggD97(;bW^{mmX-typ5y^XBjCl&$>cuznze#- zToHpIPeGm=^>~F$zeEd5{63_Mcc4XByYkWXluicJ?_mYl4T{>tGn<}jA{myAD9GJ4 zwSmCyObk9Ua5v8Fr#jbAUF52FhkOpHAJ&^&f7Rlfnp&|mPzDk0CH#>0IAOU3GHNP} za&*3RU&rY~^?SvfyZ@Pvt265T!k9$SBv@l?Xq`Le#?)F2%0I`V7E#fe9~w(^CO~#H zr9{*I&ANm!R_TXD%gU@ihp~F*J=Hd5mpD!}@H4GA>owDI;G%C6<zzN?#&_@I7l3QK z);>Ugc|L3Ca4&EB7vwBd^{Na}H&Y$>0Q>aM7}KgzzqA7OoPfY*AXy_jBoWQf{Z-hH zLrg-nunZS@AXnc!VLH+C&UQS69-ztQfrPEYg4u1oDM}F_4+qi4X`UxfNXk#fQ2xVW zG8`@sy~;}rFsiUWt0#oiM!Om5U3DoFAau}Q*fJmY?Z6P}8XZ$J&v`RgRUK-gXo3?) zJj<h9k^T}4c;@9Y`&hOte>h`Q7yS5yc^LrB3bPf&V}#puI6^X`5E;mW_O;aD7ltO_ zTd+PhHH&dEpa|)iH7=A2_7Qov@wh|#EsAd1*3?978n2Y>HH{=|B-0o%Utg>KD^V&} zl?=^4a;`eB9XD(LphLMxCYxBjm2%OFEn3g%K|=APT8C54!qVs^J8WJi0*1!)aGfey zQPm+)&(4MeBOlB91CjOa;PPjT7i9Y`K1M6?J@7V0MRcv?RXv+Zu3VT|m4!?aUA<(Z z9d$BABty{m$4gWSqyXxG{o48HG$By=WU({NFDbrABHXbLULTX;j=4^F$AuZ#R|<5R zw>BSPK<C;$Bk{<6Z9w>ph9%Bmbpvvzj=UKrXFpp<2qW1&yY9$Q(T;uFZ9@md<DTK) z%NZb6G^$35qa+dTxL>iMo=8lhJtE&H`RatFcU*O2r9$5#2<RKOO}7s^=IhoJD;%r8 zt<8TJR#`bC4i(4Yx;@9HMjP9<sT5w@xcS9muLO@dx*+bm)*%ky`pPFR(hPI5X4iN4 z3^BQbVI_yD0R-ABc6tbE)th}e>eX9kALR8msNsicT*e$V_tyf$LPOm24eIwRP%oFK zjAI{#WT9W0hE&E?vRFyYHlr65Y}d9@S^%WaAo`;cBApUlb&SrBcI(JP-$qZO5#mMm zgns<2L3(W}TiQlB1V?qE6Qp@<0pAcnA6LXMT=S`~-O)7zLagO>lh%+7=y**9<!+}h zoi)No_0uIjShMu+DJIT9h1J@XiAyIki5BztJp`TiE<FqsT&Vu5xe<QV5M-w*5&Xb7 zvxF9APQK^#-cOf=1(fiTc+v%alwAZ8IvO7QmEE!jOPQm@kT`0un?%Wz!m+X}2fcqW zNdISv4p2BR;NYviS6WQl4>7kFc>ulezYHM6;+q|e<YbV9&cmZhXFmJC$e^$&;FOJw zs%0|5g&vCq+ODzK_T>dpXgm5R^B2<N(G)-sN2>c_20FY8=)vpCtYe-kA5<FS`Y3dE zjuuxYD|x9i{v`z$djC4n;eEOjK@Emm%q#5`bGAMEuh09R;&%O2EWo$*zW!BOiwgYh z)%#WRUvKokhenYUaJh;Q)KYN)e(W9aLJ<nUHUIz6lvAwR0-tu-ND%i7=uBP@8oRN| zDEWU~_y7D4#=F<5BEmdD7|8vj0AAK5o9OiI|M@caS4hEqS66}wHWMEB)J%Bi=A!=> zy3!5E3}vshlQscv4CU)rMSxBi1~|X}`JLqdT`+)W;#Gafp7a?gZO&f#)n_A$)Bimi zq5k*2cKwsQScFZrf(89AYUKZZKl0>Pl3ccy)(8{&`BhP7H#$Mqsav?jZL7E#r9o>^ z_kF`+`;`wPn)5CC%bZ`pbG{i`hU(iwGe59}xb&OHRH149kjGP(DT80G(dJ2d;S#q> z?bn!(>>JDHq5Il*f(Ddd05SHfI@{-Y!#9D(ob8nvT)FfFC@Vcc+6QDBDs39mdhw<L z0gMv>r7&r$#ykZ?=_vvzxYuF_#||0n^8<u%<>^qNeE=x1E|x#{uNmyKndJf8yg=i# z_RFza%k`WylC9v`lQs~<#$Wk|)*#+kfM9S$uyTrd&^I?74oUe6pVfP^u#-E$_r*Z& zy(a}KkWbyU)=G5pEbi1E{5ar{gJJh7)aJJPn*dyQfmdLWCyptTFW^=44uIi%zT=@q zDgRdtXYW*yo({Iv);X+=F>&oj?IY)-v0;d(Yl-L#w{!jKPdvpCt53(j_<oLMjpEC^ zf~MT`-x^Qw02X8@or>AyPllc~%}>oisz)LOqhSH!B6olBn_tLEzrM2X0Xh_pLQBU8 zl8;Q$5U0phmaM0JPeK4rZC5UXQn6iJHQRgc@Qwt6f`rUsNM@~7ZRR6}=FAUjZwe@0 z$j{C;U<fBmW1dBo#=<Y_gq#n4qk9~GP=gkq8L(b<xk?3M$kdyYPnKK6<yz5!A>Z}4 zk=J$lPx>E@-sQY2nL6Mi_bC8K3tez(;6Eyo);wPBnL`^OM&?yB`|CRMCj7susv=-V zP@Z7JY42F+sN|Y_VudLN#1yYAXJw_UOZWSkth`dadQGQXrI`cXGAyUToBwE|6f+w5 zi-G7hVQv9ZMX@=eMyg7Yu3xVU@PnK=kmuuH_4;(;QsC_#L;FQqJe>WGiKb0jlyxEf zw34G)70&p~L8=BOq3(hwdol+L;EyU>_B^+B4H;N9CJiC|6XVJH;i4KRz6i$r+Qy(! z#Zx|6GR5Jqz1BxXh1ZikhGbEJBt-TiOc10L(okLm8i-d+l(4389BoUW7Bw|7u@9DT zvJwNbMM0fg`lwfRu(ijBNmC!E1{IUE?_UKF3FigN3Gdy9gfgIuAU7_$>0DUr@QoFk zBEx2K`6pYZhs^_aH`b8!omi4$$yu`t5f4@^G|1`#x^^U&VqwLKi3UGoI!nThmB!%_ zmiN&sqy~tv-T`v&P~NKKBlY(8|Di5Zw(U;by^;whr3H<8@yY>!cmE_~>D`5}vq?r$ z2B3x9tgYBW_r24Z*Y*DOm+9NI+3|y;Xq-4pFL97-`~BsU^cr*;urK@b`%ZdNV0?ew zzILM);NG~7=qty_?EnG~l2W*|`AvnbdqGb#XxIL$mvdX%c-)0rdW~A+HA>9*I;}5X zzeL{DRW9N!sufU1d_%&73}GcoB_#w&NlPKTj9&<5{XBI(;d4B7+!a{T5!k)<$o%ZO z15R;V-32eFjCpmgVTZe`!yEVh60>mR;`lfi5$4o!wA83@u{-wpwlKzZjlu3C;qx-q z(vA1y&5+tayvg;F%rOkFDw>alK|?Ff4*uR7xY*MvVF4f46h6NDPoHL+xJ;9#*$ONm z@cCa>1`HN%k5`ClHoHU8vB0KLN6e>f4*~U7fx%jIh1rmJ3zX@Ht9fksq|hQTBIg(m zcu1{+;#$)c)!Gs7c@g3}SvcUFqv8`l5JZB|A!Wy^+i9A}kSw6<LSz7PuQYJzPJB6z z<s%aC(35a7r)XT660<3rgDtz@@NRlSO_{I!iCK{*v)xKKp#4iN%Tp;ci&r((iReAv zuTF{`h~b~T_P2dC3T(U5AG;Y=&y&EA;g*kFz<k{B6+sXQVm_gNJ|L`SZ@3>ub@;x~ z3wrEx)${$J#bsT`^iGo4QM*{rt`=nuHGgs^1V2f~XFsd+h0ETJ>(31%HLk)r)@*rA z@%@zy&kZ-x+%Ot;B{b{smrSozF|0Ss5ZC9XmDMP8o2Bs<`g#(MQ5%7hs}u)Cf9RWN zP>XS<^F~3<edO3*JnY8pzmsOazkX2Qu%UVBx4yrO)wux7f3vB%b`zV00*A(fw`+gI zcM2QVyF-J}T8I0%x?y~Cdc>nlz|92Ok_?iL>6HEGjnz{kpHqXgI@T-<^PVUNvov{o zeL9byG&(qJR73Hf-NtG356~lj_J~I;JPUoQaC?`}U&ZwBaOq6VG=WQN&Y&72Sm30- z^z{b#9pC4e1(%u_ZKB?((|OmPvwvnf8_yzeOxUJwdbYa1xJ2Pwe+aJjZ)wJ52x|r= zd4;6~;j;)Y9+9fKllok-It=Zbg@&X4-o4@86tadqAhYXs;zl_)F}I*BJFJ-WA@ug{ zmYAg*mHTm};Im7nLJSh~Xqm_G>fmwFi;^D<K&`tx4;PZ`q+GrRqwd&i=+-paeS-2) zHBnq#_3f()IAH3InxR%{()~L>uj(67^Y7X%OqJ@N>PWmVSLXB<t6dW84}08#iHgz; z4PrYM?*0%iSPqasz@7&&U|){M+)z@;qb(2H0oM)972RO0v)%LduBBGuP>n>Zs>lEH z2hvPdrWmfd^j=wQHrvHkvsf?7iuTj?=6TIyA%;NYdjs*vKNEvl#)ioxJn$RwC9GgJ zY_Oe;_v6g2LwvPUxHu^A$2nw2j?HX{X6d7xCrb71Hs)YLolWf`b$Xdbhd_vrnS2AJ zdgf`Ad^A6k8^1`e{6);=xo$;srhABKmM3XXL7UqZQq$W&H21mQbdsxYk{KU+E&1?{ zHRL6{DDjV-7b87kY*(khUh<mIWm@2Q%v#v@PP~bMELSi0&Ps3J=KAmFElVu{1ULS_ zk6q;y8)X>h6j&~6iB;%Jh)UNOol_g8R*rXjQk4Jp6U0iV`5SE?IlO2T^t52zQeZsG zq~3W?3^@E?t%PW878}XEWM*y%NQjH~VTh@RsZVvOq8@n<6z{H~=+rJBp0G;f_i@tf zB@cQJGgCJZj=qTOkV5@wf4kZ7_i#*rGyO*Muz@O;k-##M{qNM8+j3~P+!L}R>XKQ- z*P@h1toBgR&h^alb&AmWHMO>?D;eQ|l4C(fYa072``H7xYpuxIzws$L$6eRAq2+5s zTvN99Bd2@kJES;X*Uw*1@5g;0<tn&g*kSl4KDU9X@LA_E+>IzqKZRU?>D5UY92&TY zKZHzS<0V*nMeU1P8t0RV?k{$KDyX&_Efv~+v7cgb1Y;1Xfvq_Vu_*?Ei1P8yZ81Ed zxu0xGSI>l-w_`k4R4yhWxUJjJ`KruYE(Yifk=`vQOGX;p^!&u$-quidZXDnJQKW~4 zt!3xfvA3{4J1Ned;p^_+ElLr@na_FJ#`N>SH7U(v4zF#YPY!Jo!w3TX6<%btY4upN z+WSc25}k<}*c6d9&V}6e=h`Z?9c6&3RgQbaR1l~$7;G8SKk^UoTL;sE`cOZt8dC{` zQ!jhXx@Tv|k}M8)6iASfm%J`$HzRcaDJ`GF=RCNc+`>OFtwRSUGREuJ$Wh;~)<(E^ zoXUOEwB1zT%x4{ojR&-cr^W<*9#wdkHU6{!>Bf>jTYM5*3v)S+1=Q7+LaF)q%$J+V zttBXUR_{W|YD9u@k#ch6r*0Z5oic!x&F)=y!Fc)RGT5djfbq^4_o&TvYq7@r(6q$e zmp<k84(c`ysd0_MmDBfk1hKn!$6oK!-7nR7c%NHhW(&j6ji*UiiIJ#n_#kyT;eo=F zEl3n~a(6r;uH2kTiOEV`)#=3?z@<rEUE==S5auQHJbK!UlCH7a*)rIkwq5GY6m4QE z@w?sWX=bxL!Z4pMsnm9XFdE!@a)1mf4N~!ljGECbWK*kpz`x*I(BmU3{(N_aM&`4- zC>*ZnT$nD)64JI&Yc-@PB)AaA0%PD4&#|`qP%u|~1V;Zx92geiqWwNC^jvh}o~yA~ zaPf8;T&D;|9zJM%#^WVsaIoOCVKj>Ryv#+lY93IaHQYhP)@D3@{7^FY4icllceV9^ zIKi0YvruPH-UW^7or}2p_CV!%_GR;I|C(y@RQ$)9khZH?GAr2)(RV|$4Q`y1=o4Fg zhAo9dl6}Y7rksl%8OMCcc(?&}K(YQ?Rsv@A6N`zjbl=K<=HRzi2+8(1v#WMnoi7&| zpCn9ls2cZv=H|xM0wu*=u2wO3KHonw9}-4c2Bn>Sp3c@8W1(RFoh;c(owBBN`y_Oa zx_JGg%%)|9KjHgwmA^upzFvNi_0>J}*Ro-Lm7Y;$<KY-q^fFHMN@sj7+IR5smHuJ% zAx-u+J#OXS4oO{Az{6pAO-|jl4KEv+<%qjbn>7Yjd1Ncoxnp?&o^-(7q}5%HI+&$9 z^1(bmVQQ|3hv-CehiJIe!34}X_0G2YiV(4)ou~P$d%A6~f<%oxN0)CE$(lDqv>4yE zF%X}v^WSLh+@!Hx_(%}ix<UE5`P0nd{A_=`{ETS)5jeaTq<@;ky+JgVAm4{&=Rn|= zKwyYm?=;i33vbs*w_T<s=cQO3uYVEXSuZ5w&aMjcBf`X5Ed?~grQYjJ&GInV^N-Mv z)bGya&oCwz`>uHgVPPc#aHQ9ET$fDxhCi*q!mb*ED4)4+_s4q!AFK4)d0);7ZzPSP zIVx`b8)q;y$Zs-~PaNnmisu7#$idncHeF}rA@KCm1rm8l0lbv^Fc?YaKZ5l;-O@bF zgVT&FtKIMyu7y4G!W@{meBHuyZic)bxd=cQQFydr`XP-wv8a2LZt8GqVODTRxCytq zzAE}~enFv9mHHvq!i$0MrptVJf)qg_H2x-HjjZfPy&OZ4DlEfMpN1}|QFo3yoF&0% z)?4#<Yls^D<V~MI4iYKwAJv)P?)UP)JTX$W*4N>(VG!9VM>c{ZO$Fl&j+!CSZgO)g zs>7Nu2C|~C%GEtfxVNnWRr(}iSwF#Y0w5fH+Mfu!kplA`Ktu+f(Po1uY1Qt-7wlF9 zbXaxny+3&6&N}1O;cO5kj&7^xI{OCOX*f*uHHhYPy>qE)Kuj+k@+@j4KX)%Kzv1ZQ zxK-oCd>;Q4*}l;dj3usm0Z8w%0;jA#hAsdO2W+^nW_es~kip>PQPVEaQ-6F$6<dv4 zqD#*;4Zr0ANxd~b_B>UmfSJQa`1M_oA=(fJ%#yd9&2_s_jmFKU{#i)2%bCbKvguo# zn10hp_acV7{j;!kkT@yG{=BD)U$4f&``h`O;}Dtb48^WC;%;@nF#k1D^^I?(Dq{!| zrjRSd8{r2uDmOKd51z}VoExoQsH%GD$^@vrel!pxraCb6)W#^2gI&b1u=_Uz8u^!S z`DuM8KaT))f#1wZ{CzEha+^M_>-F``-Gm!{UV5@G428h@Ls-x+={u5n!_mNgN7K9K zw*zd?Rt8_6_nWus$(6f7x@AlkSIONCz7P1WhV}Wvad+*KPhXw{48b6+b~fd1l%CI^ zxHF=_wH1Nb5j2f2xw7>Y{)%%Z(@vn8O$;$g*pbfd(Z{36587!Pn^9|4PcBq&1eaBG zMB_Cb+YyH%DVKJg)n8Y&c?M^ZnS;*Ka-<nIuL^Q1Vy{Y!ddkTzOb3%=fxs#JrjQ;s zOs^|(smky%^&I@A!1p+*Rk7{3DV{=4s)2%|3J9${whHXnjb1k!o;m6+TX+>H?L1M= z>A4_^0ns<NCi=dA>bVzFwANU#``2CZY0Cl#hQb<2JoNH!wW{>J%_g8dN&TQe9C3EX zdn@@d+f%pa?z41ww<Jv!!rumE>cL9+=UmR;6*Npv5U+WYG(IiMLoW#X1!%Y^8WX%1 zg09Ar`qB;d%nMGNw7FHS<sMJASI~iee)9fRUD&cDu(y|Huq;M}i{3{tm%4c@ez{%p z6~>wmqYqAalE_~(dTAd1eQ-pYK-;!t1a-3cv($OrH!Fa`4=c&7PS>B|%CVI^yn&5F zm9}*`$JlMaT;gcJ*sS0<{M5%ML>z_)SH|8Z+!wg&w~q`;@xkau;XXvTIOeC8eVXuD z&VN8O1=i<7E3WljjE|6S6VoRTqvVuef*Z+*zNfC)vzIF9AQR2jZtSQxp7wiCh`A*5 z{8zt?99rCY`U7zcx3awnSw2aTRd{&E)ep5Uw}rP?ik^-Kl9JAzksk^CF!$YpYTdmn z5@q_-WbfOwpK&vQXT+;H39q1zC2};KK(i~u(GB+g9?U7!s)=(P_#^XCc(AZx<&i*| zr8m$K*w7Z>RY>sM@Iw>-5bWx_a~KnK_sFp8zmYnM>$Dz0YG5QG%P?dh8XKfbyUM}V z+XRTyEBY8r4pm%JxDa{|QKJGsRuHXfx9pf@Wj%^1ZNF6k#>CZPnTe-h5y}b0Y3F_F zRn|=#PL@+^CiAK#P6K?iK+C4TWpf@YNO3i<_Qed2nZft@4?Hj&yH*y?%6UKVdP0R; zrN4q}BlXR2mSuEybf|Z}6eRb2sf``86hHKECYt~r#FlrCa6yD{XxPT$FNA1mBFBTS z?;M=p=`TE(m9!1%yN6QawFLu>0HN!?tuC6kMX*REPAXduAQ{SudY`<JjG=n2RoDZ6 z$9clba+EG6P@fdUQdsL)+Ul}K>nKWrc=lM;qn?e+LV=}!9rn%`Y`M~&bV4McX}wv* zIi^ogxWyY^kWub;i9WkkMR_|~kwRjmTd7k~FzGwj1*4j+iNhaC;nkuV*TP>5d<z!# z4}ElUiytc#gRCt`iaTkA2lkgFygV$X-&khMw%t~Zx8sP2t*>GQXsW{1x-CJuP8T3& z#syxG8CQ}W;Kuw4@g{AeI`V-gCa3A~z-C@5P5t$i|FqKN^w=Z$5t$nVfALTtwS}5g z?HFCgC#HM5xTE%VE>6e=6R2->-fN%%^(_NzygZAeIGlAhl5A5>xz+PBhv%dn(fh?m zyHa=kikLLgzI^`=_{)r^oQ7}XSI?p~&KjC?c+_1HmkQC54T$*lD8sqw8uV3})Zdtk zJf+;!+g7QnuELYIZ4aW>r?Hd~f>4&zR8}g`E$qx%#}O=qVcDGpFCDY|)ZfxG2zE0_ zd7hb0JG+auUx%REI-RSM(0+ttug_JKp@j7#HDTyp&%K%VtDry@_qMYqQxCk#8C`yB z4-mi$d(_M8BI(MOT;A*D>c{d%aSlT1Gf#9EQJF8Q-?&*wIxDuZhollNKg8?5o#;iq ziRL{#av>C@*_#(5@R9DP_wGd-<_v>!8<!9PRTP?{0NT7plqQqo=Sv04;J3&sx56A) zo=xLUH};Vt%@|E&Etk}uVZh|<YzU!rvi;H?lbH&OGBrjz2F%x~(*e2WNAH=QpzkX? zkm=B4sQ?Yt%2+y5H(xWKrRLrs*y42II=`xj%L$x`cIN?C98Z=8OKx*|GhH|7K1Xh) zP9d65{gW2_gX(V#_&4CP*$eebG9NxCGDy@7QSlsMJ@kzC8KMkR*zX4-e5a>olYKT2 zf!3-|bTn;*!%991MKXoF@|vkDL1R-I>{^_@&eTQ>k!TWAo@ZSQU2;xr9Qj@8_qUkG zP+I}93UXBzLE>1;5dE(q`EyiF_ZpVc*lJ2}$Sdes!UJtPB(?e)Z=qmRZKUj~r(i9u zmTQ}EuUbY<l~3U`uYANtJ2w{<laBB&Ctz!H*zq(Xsrj>Sj6m13kSk?4_FFi(24Sbr zyAzzR*UwU?P=QfXX0j<&)R#%&Tl)<lB-8T-7-u_dpH`MM`pp0rYemjp8O}DhfX|<N zmaa_58MF)taAlBqR}E=@x=WGkLcarVei(#E`N3{d{BQ~&Al08a_(JqFsr+;M!<Ozl z#u@8{{<h%H7+Y}N*Ls{gds!WwJrO1+K6J_Z92Y+5r@sZ=6}gGlB*<`=+1iFv5KRfu zhgZ<iFA!W#aKL9(aJP!wB{Cc1DV6HnA6%E<ZJDcc(M@ke+NuESO<Rc(X!ytNTcT~M z3du1EPH(tc^5gR~XB2chbk%k`Sr)K#H;U@p<HKcrLz)b0vo+DS%DJdvOj23N+wVrU zLiR=_=)^WzW%{@>Y1dFw$7H7tm58;Vv^0UUhilj0jMIqAM~v`m*HE(kcq2c7J^6h% z_?DP^m;0a`9q&K;ak%3YbNAxzBoen)^(4;l8n=HYCSdDje`|7wXsUZdSajflEpsA| zEi;<-Eqfdi&}97fcuuM(s!{%dpFSX^rQTd4PN04zuEC5t`96bLxZ?GZ9MsRht}+~` zrvl`GA{Yl~mW*;?-Y)&9iN`#SjC@ZI;^kUY5YqkL&(ul&tyXo1@G)e=w%HAgO8V}7 z1rA$oZ%KTIx*^}7F0N|uhD3Ww9l3+WBoT};hNc~)i~>aF-@x;u=OLP|X20B_x()Xs z_dVO8lkyoE(D)Q5ZMB5;B$66neETjcetxm{#fZN+SMvvJ)&fKVKg_qYcsp&N;A(iQ zLU}`q(bQQhzna9nJqFnoEQt+n+pW{I^kD3+6G-tc)qi`dj>BKA@E!8o+yJ-h!LtI; z7TXgZFIFFIyU%j+lm%$C9SfzM$HOHhH2u0B_}lNo>=|toYOQjxYO8IMX2@Fj&Z({c z8x4D$P&zD4azA2BE6^JBiIabTW%q`Ox}!U>av={OI?FYckm{oo#c=NfcY_k8!hBr@ z?(r5h*sK|7FR3-)*kQ%2HLed>CI(KfP6fS-?fbD^TU*no0#e=@sWi&m&OdN9n&zoG zC}%uSbfcVb+SS_s9DOQbihuAs!po8@uX<tt8)C*@`HvDQz+Inv@_g$CFG+wyW{y{v zti3SnB}570l8w^zlp1;4R!-M)Gc(Guw;8$457uqouwtRNEphppB;f@w(*`1#KKO+a zserxqsljg|fF;Ic3cX5CCJ03iT#MXvW)0wsj*<IS-wJ|OxvhrZ{9&*}5sZ|!K-Ozk z((mC+(#JLcp)~=6GsGhaC-ifo%L>pB=1HYN)1hdc0L;L3Y@uT-?&UrHwHy3d`D2X# zA13|8V)wu+`kOh+`Ce+YZ*>chA9&I!gIn<>Nuo6HOvbB0=Xzi;@1f?R$K|>wA8>;X zjo!zRPkIQRcU7*opxLtuaxmS7mBY$NynjOL?$=^WczXCyyGB*-rbI!rcuh6CCNwR_ z$g*;$pdOn)M*%@a;eAF+z<MTAO)D==9GmVz2}tQc3FyBb*=a0S`1#ptxhAI;S3`yY zr$sxGkGbe9?`5|mv<f~sTD|!(Jfv!G;k&j2=92BP=WzKo*^si#Ho1|m9+oiZvL)B! zuy~CkK5Bn~np(~DC{JV}&^u(<{g+-|e~@9aiTHbY>TZVLJkc)4ek!b>flDBW{X zK7JO41v<A)fAC1;ta<F!AAN%mG}ke2gv?~V-_cO`gzp59g$L6VFtx>@7bJkvv?0e> z7${9yF{BW?Wqp6oM9^urUJs1YLziraBAL>NlI^V|bE>0HxiswxMNNgmfnpQLSGx@R zi5%&|=CO{Uq|8FDuuI8rgs#4-aiKmRIDZmZxrP#jV@J(K`RNTNFHyz#{*oZPatun; zc-`OTZO-mkoII36d8FNVBzsWF`vmJ*YUoD<or|1SxT_hl99PC3OZ?|qBi}*SSb|CG zfA_lfC7+Kn5$2<Fj#p{=yB)6k#LligWFN>Yt)G+)$BCthebT#nQgW@7%7tBoK~~~B zGk<|6NgL{WED1=7kY7DMaV@W$ixv;07^F-tX{}#_X#%RFWKA-#NLZ+strxR;UtQDj z7cKq~qI|vTJS29)r9oWWZ9P?3_IG8tNh!@(k|iSA|2!=DtoY{MirrvqqJTD0vCuul zZdEh5m}zE<<4sLobNA#Z=$}e^oD{&5^4peN`*}TL8<nBceanok>e?UC{EC}p?{2Ea zQ;@3tX#AI;g~3}^<Xl#z-WYEar(0F7SsKF_%KcLfPyhj}Ws`f)G0v=Yl|wHmUv!ZR ziGB|E<Wk|3;!I0$S@umpaz1b_>dv(@MHSC+4_xEeugJ4h6sNLiahX2etPbzFoED8v z9^kZ5MITEYLI&DmXt^20^4GH38d#9o<m+{G<1nqRGv5ON1<r9|M~geue?zDt(dQAB zRc>OkTI%A4WKD%FnRqR}`D58zj7p!h75?u|26$Xnc<l0!E^oHvd1o5PZ2hu0r(&pz zc})X)VZvJ~nz%o?mGPJ8nGyA^;($4*L+F>Y^--~6O}y2+9P;yR_h+2NF5xh>a{1qQ zUz{wlRVX~dbl*oGlJk)7gM8F|t`_<o)T=jLBJ7(!Q*Ov{*_8fRZlg=y$RV!f?ZMhi zkF5N0L2dUWOnrl~Jjp_jrKAeSp0FVhwL$S+Y!`Rda1hlriuTa{@?i81al>Y@YPEq$ z7w{1|5|1&LltH`c^O<+q@`0R6JtJ4(4l(Dz84Q$CWIk4={X{QkqO;MkloX~5Vj@Qw zw1KU|Ypt#_OTtFwECl79*WF*pVyr0yUMTsi-Vfd9LVpAHk@4#Nz`+V6%4FKHOl;E- zQf-5cumyAqk3P!ejcm=B&jc_PDn_0pVa$PibOGi<u3@uH4f}oMbh^#|>{rTVa;M1y zT%GxYR|0l}gjc}0hq#MQclQ26k&wenP_<~3%$3Fs)adDa3lk}FE&1G_6s2(fwa=#i z><lDGJ%%TRGm7&aLk?$2@m5R;(1TQ%tgca^%ASlW;T=C<{^eE_C!fYs9V$w<Z@@`d z&x|<_`A-M3vap+#Nw#{JrpD4+aX?);XFaE>v@k%KY;bclGyH6(mz1p?tyPLUfbpK3 zjeKK~eIUIIp#g1J;i-bM1i26goT+8V_%<-g-8&~>jScKpnk0Deu$w`ku~Y$_TP~L1 zTlR~wdbf?+Mg5<1b1hlTfL6B~MfUtN0{w7r1a9Oc(%?0l{4X&1mea_Rj7fY7Epf&8 ziLuN{KHf>Yu}y<0Bg2X(i{F&}2Jj4p&>bKnkDLs!k)3-!@(awVf6IE8*sN9I+Gc=- zDch%XGg)R-^KJA>2>iOCW$WX@yf>wQ!urhxO-qsNA2PZgn5MEkPwMk`LiJ4`Mi6&D zNr-$MJF$6{$5CL(ZX^LW57;BTamXbD`-+U)F1a0}n&ON_X#gB^d;E(C<d@IA_fVzp z8gr%q!)7$lTSNU;i<5nZd-Laio?_wY6i%D4@NTxTmLEMU(EfUYy3OEK40l^Qrycfh zjb7V?Hx$Gbda1y$3Rf1lzd6(t0;%xq+Wh>cc1dvT+X${ds8F-6smU2v{c^|j!wb4+ zbt2!Y4`!&Guyv;$Sg(1REbZZdDOu+p9=$CTtzo@R`1cnSarwaXWH_YGMn43^1Pull z`?^HNSCGMhn4~SL?bW{pyjsxNCLTTmP{u(f!mbRkr`1vXo)`1!d?&^-6)JMI(m%IK z<Mw+%l#i762Y^RS^P)Wpg$bPWDO;fsa+Ag*nc69wOjNayknu2WmLAVPXVjncJxitH z4OidLUmOJDRHWb~cD~}dB%E)YfHkQ7GbF_lvEmO;sDaKynti_*F~j?M(mFlLr$|W_ zWC!o4_gG?@ryoi0RCGR;ys-qo4_ynmwp^{B)zY=>m4XD=hY)<i^p*9EU8js8U!`fR z4PVLcKFHG%40w8*%gEcByTsJwi^Ey5WrM<Qh0gZ8e6eg29bI@*&@!UHB1!`X%OmMv zWU=AN(T&nJ+WyxscqfZ`uaoGfVlQ4cO8t$a9+N@4CZ4cV_a}F|9FZir6Dpfh9X%(k zIC3M>0HO!HIruB11(`~=PTRC2Z<LsQO^VG{&6j3Mido4o7Tpn+Xxir-;)F|avyaj4 z?OX%&$>w43wqzD0D`SeF8XjN|!a3w|F5kC@^taEDnZ{#<#d@4>BAGTXpt;85%L@48 z_1N7HdSk}ZC+`P|H=Vc;vd*ktmh`LdJ2F*aOY9Hhx7LnzN^Qxlq@Pp1*=4{+{QDLF zikpQ~2FanY6rrW5lJ~WQZPQV$5d=0mu13q4H4oqG#FkWGD(5S1^g1O*_p`fR6psfo zkuJ0JQkzP&9TYAV2)Di%dKwH!wB66>k2Fn4RM|A4iim{+3_r{!@fKkT_qK(3!*5wW z7T?9Ds)fe7c%W3~5eQ=HOa^EUTQ0q1R_QZ|Z)9!uRNtOw38|~D$~8fdzQDO}TSHJe zirCO+n-Un-4sh%%XlQy~4{;mn>Z5!wSX93uHboy?*KFRJ#|n%ayp`vpA-7}1x)YHq zwO+Qc>)oYRZ^&8G9|;VCA33;fsG4>DXNPX&J7s%-w(e3d_R)(RH?`{t{JDl|@PjK) zsISyN68}N@5%C!0a0_wJnc<}<3-}Hx5702Ju%>&{67Z$?t0_h07@o755_0YL-KG}f zhvwP=Hm((y*5Oqi{S&TJHV_#m&*R9ioggj)$~bI!dLzbs?cp-;ype1>@8h5GdL(qC z>qV!ZW&^0T(CtfRYXo2K&MEjo<3TW6#OC82MF6GW9Yev*+%ULU>2~Hl3Z`t^^hK`L zdV46{I(dJ4lY1|5M)G|n-(Wi*F;_?n+Y?pzXbC+<U#-j}-8D%9jKw>aZg!|ziJ(EV z!52cb@`<`o-c;HM)x+#hluJi8SiLefZXi1X-i^h2Vs}E3a|S}21n(UtlMI3$6cP~> zDeeQcy7}kQYIf>3$cJHLwAT@5luXoX4@bfGjj8*>d}Z#FS^Q2q;Jf<Vo|2HCkg|O% z;=VZ%dOUAmzV(B9_B^!BN7|QYc_A!LK|Dr^*-x9E<*!<|)pF#pDAd!2YAhb^DlF9H zH2YXRtD=3Ze-JB7qgiJ-u6|<78iVY+?0`=dTZtB{REZXIwamL$_`(BE6?(3z&m?9+ z(<V93!M~yAKB?<l^qfk;vr~PK+x!5#D*{5C^{hhhFuvIwA1(&^bPJMQB-6MG|AYAb zknt`~XMiTU;Z-bAZ7&{~_=8LdH+;&q2!7hDQessIe74-Ej!}++eA?;Q>cz1QdVozj z`S?qq7pbU1KWXV$ND8TG+oH=i%IJwJ>U3qb<xFI+U;WtGm(g?W?`q|yIP@sy@~;+a zMS&Buy7NNZ3ZL`uD{M`&&<oP9!0e>422lN`U`i2oGt`DOE!o3FrV+z_jlG#g;jV!j z;W!SCeCO<Xs}sWcC#%(m({?5{O0)2E^rfON8aaLd`q9<7!FPScs5<~YRQs8d<c)B+ zE94<E<6Vy5pHn$E{HLxqY5x52oLNBVCH6z1oZN<c)&XNFfZCfz2?_)KBjKHHpq?Zh zb`Pocd|N=O-t)5IiR%$~wp#@0IHrIkB{4DaINYN{%u-oCX5{-|KiAa)^Jkwb^fLIg zLrGPvKFp{I!=09Z%QDGcLIPNJMzV_7-;tqzzt3@wkXd$a8Xy>wk6tc+aZ`UB;cMz; zT<}hbBN~joen?h!uJ*Rk#nO+5xlKLb*RAt`5=0=Y`Z0RsvClXM$y~4{??Y9mPNdah z0<U}t7D>R7Zi;n&YQFBSX}N1*DqQ7jJ|~&|3!^#vONxEbf`r^K7d_(LXKl56pSJ{; zsf~c%W5<VL%eJ;<$OF!N4dydF#l=Qrt4U{E&H`V?h<ih^s*w-#p+k!jaBtPqAUBt2 zhXIGKok1QuDxd8Xf-IY!V~n|L!2?7w<*RQNyseTXp?!d+`2On#FBZ9fI^~b}01f%> zFO3fNfnh0w`iQ|zqIf#q?e2<#zgw^v-F=e0Yb$EQ;Y_x0zyT*?Py2G@9|ui!<D;pQ zw@upJj;`Kk+gIvCA##iM!LL}DJ_Oa{0n`Vs>w@^cxDjJJK+R5C_AnStsPeW(pne-A zVg@&`Dy-nA`6K=A23wey%pO#eBF!4}6V3c^jBYjiSpj3T!Z>(oz%?XCJ+KzmEYl0R zba?7o@bdd+9*QT^wl$)K)(Noq{|Cwm^!OsQ(U;`pw}1UtM5tewA_`aFWtuJNTFMM{ z3-xi?=}e=9$cqL{gL=1zEPyj0AO$`=?hS}GD~oG(wC-pH&K^wYe1i8gxWRP|+|eu2 z7K-U2{|RDciJGYKoIW3;e~Mtw^0y>d>Ho*qTR>&?JaOYnNJ<Kb)I$g&4FU>CBO#%H zbR%5?(j|?O(ujZ{sC0Kp$3rU*4N8iXfOP%$3Vg-i`+whaIG*c$=FaZjot>SX`OMt$ zW^uV_za>h8>?{YzKG#}nKVezZ*-(6vlDErU;cAiSH^T2mbi7r}ps=QT>QGZ@WSf9e z6nqJ_c$ucd4tTl@sD&GtiDk@^(KuT1m}U-GrqIXla+DrPXY`<V;}GLgRuEco+I|ZY zs*EPL7oF$2(_)Xk8nusNXJ9#hle^+Gb)OKkQ@j-HF}fx1&@n(9F&Nxz7{#_-ka=U_ z#GKc#UUvENZ~X}^1j<7AEE0*$Na1q1?aGajqc$p@J~!q)dnSdT;6j4nwyQXKbt!MI z`$$qM4Qv})5Au)Ns?^p=$=}(z)|R4pTcgZxD)#ouGZB;j`%83*A%PYD&tJqxAX;;b zo|JD<2ExEwmn_-w#G`=szW5gz4t|+k;zGouL?(dCr>pl7N&ip)>H$CS%bi72o+uOo z=n`cSUB&l*iKJkhU`KyF@(c?&ySE8}<NHWfodzS~(!YPgJkU?w!s_IJOc}Tx)26>a zAOJ?dKY!lhFfcdxuHFtX@88fgdF;Pox%t0D={LbIX4PRt4BqK<;F7UAG=~0viGI_7 zUuyRgb8mXr(1Xi-w-;>;|3d_<>1gm?viKaxpjSK)DkYR%{o21h2QMPr1MKfuE-m%j z7!faoywi!g%53%?0H_|DB$ZdB7rY`CMGD4(v3$6Qws$m?WM`1c8Of%5T=MkugQ)+; zLHIVr|5a@#&uignD0a5owpJIDyfYM;xO6AtMyN(W|B+E!c#(6DQqih!%*KIDeLFy7 z(JtuN#b{|}#1aiRawf7(c~*+3uxZjH3TiUB?W`8=_0~^o+?%Nhu2J`>KiS_@D6;E- zZ)`0rJzc<^f=g%gMh`6Lz~Z#Mg4fo^MYostBmopp{&>%={2Gg=pK|WWi)!`ll?kR^ z+3Un*Jx6vMizaaHa%FqfJ=Qe<N1Svw3$q+MjCt#%?y~7rYE!4Ca(pyot%T50KSCIc ztT*x8+iq5x{cX<{o%r)>Tw@?n_oL?HqXVOE{)a{*Oq)h<)s%;Y1d=j~_C0a023I4& z@tRxgxJxDH;G6c|v%Mvru|ZzdD5cJ~6lL(XHP)!owOT^Z55>9FgN)tRlGWKyH%U?w zIHw}y$OL2Roz`L%P2x%o=X0>wuEI%H;3;0qkweBpjJq|d`xCJbOAh<XP%{+-R<<bl zO;bc8G`2Sc_70<PQ5v~>h~9F=spc%RjC!6p&uJEIAI%kgZ`!$eB7#O*w;&Cmik*+! zlD;KQ+05-sd9vi!vukdUzOIpvDucj*o+HbAA+P%<G&^cn+&W@ntb^mj$^KbTkb;^0 z$%cqUiA)k!#5A5ccplL!q{HHCTtgQ~sY3S^?PRBIZJ>Na+Xu}x-B97KdYR3HWn6FN zE~eS!5<a8zuZC^@<6IDNgs?-NY|l;g4=N8%j!RM;C{3AozkafaqdoGK;>mO3AK8M- zCuTH1Zu1Mf`uOK@&rt@jm1$O3u}tJ&lb8q?m%q=$RY)L=-6>BJdI)e;e!2Gb!tYFn z*Y%yA1Q->>^W>Dz`G=N|yQC=B=Su{*7v{y^c^91H;+ENs(=hXKtvnzWl%AuNj1%C< z`NXT^E}u;5CH$Mo)fuB&&gK5@uOQ)RitVM5PXVS>+>1O?tgK#Q!|TQh?#i*O&ODY$ z2Xo&$di`IjGZEL*DPXTz6zgGM6PS&7<fW21re<CJ@nxZ6=K}F-{XMt?6&yIzL_Mam zrf$(zXeUtXGBf!g7);%ce&VGocn$C8hWr>9FC(gvA@|6^H{cyHu*p$Z1sET~R*U{+ zhNl4F;K<*D`}X~)7t29tbkT#Z9_HH3pIDrRPfs3|0n82FeO)o8`&fo<Wif1mM~|-b zIQOfzZ#`X5y`Xly=^&kh!T0hBo%b~wC<xO0QlH|+L;tb0i0U^&*APTsC>LqhZdkAh zRH$aD_i7LvmDvZ%{+yDi)7p6rAWvd>b$gE0y(#r;k=qW|1-O&lZbCE=>P5RAsD09* z>?Ds*Q~=7$s73dFFq-nYRP_S{XYOcicWx<Jj>)r{wkwV?YzWU=PNhF?5oICzm-SOL zc|E8BX5;EhS5Kc2M7~_|yhE<3y~iMb<8~WK5I-7~;;klE?e*`kc#W#?;jl+<pN%g1 z7#d5kYaWkcZOjH8+igq4JzSx;n)Opsh=af;_8&rGc|*3sRX-oBF0p0UY*wR+s?V?@ zr^}AmAsW#={OWvO4A_irD!vmI$+i31%joOYeqd8aY%tFdFUPa2&vAD-LG@;RIRB)6 z%d5b!%j+MsS6*%NH&++M=@_+8s-myl^ZZ~$GTNd30`(hHuRL$s0$sHV>Zmh_tNtWR zuazGZ3l=$e9$h%DVZ&UpYC5Wum+$y-&fk-A^w^WX$5k%=Ccwsy#^NsV294pBYiv<j zv0&-1>`s#9Yv7a2q@9%P*-52}iEY`EJPYf&&ixs)r2U{**K5QOc%>Q)o*C53``8mA zEM+h5g&hjOD&E+k))YqS5C@>7;)Rc4F%TS?4?FQG>UICx@9zf@8>o$sb~w0l*x!$R zYrJ>z62tD0F#|<KOv=0i8GO9au?iy%&(9C-06T7zwZ3)YnnpmWQd_l6nYGbT=#QBX z-DE;h-nnGxIinh6sh<;JCBk|y*uS{UDw7VBy(aRmAvOje>8X-T|5BRZ+0}WLv7^b- zV=F}3f}o*)fU^xA+cZ&^7Kh0<Lbi7NHgXcaYg!LY!blC8i|a$kw15PT(^60sH0l>8 zi_JA=-}c)uyc{d}EU-3Lhmg@_!%Tndu+II|;pk@Z8pKZG+Q%8+H!HA{AyQu+T53t? zd&H|NKAs!B&7ENlXVo8x>q@!VD|bU*+p~5?jB9ncPOfT7wHr|{J>*?9?ZwtHU8(S} z#z$RX3u8bhx7_o}%9nb_la!K7+axQSHR<mv?ritD)pcB5i--nptJK3O4G_cB8vyLL zA-p4E*U$Yu_Ju3ukmc9Ts!j8x@m{C6Is^YwyCLOhHw~w#?@5X3{k+5t*w!C%vvn<6 z<b#s`-_u2T)k2N;GY3{I=}xDZXwC_*&4;c80Y#1L797n=qNDX*DTOcAarckn3aj?v z&DpoL(u1zu+D%EM-}W`aY!0@5pAsDff9wB*bYSwnTY3`(uLe^-2&wv4rZx~QzYe7Q zRq7)q64dg0*-tX(rj`|9c&7Rs+#QumO{pO(tNU}Ions}O^DA{RkwQ2nUHRlhdsTO4 zsX`6V5VZ|ft5}1iZDFk>vEnRQTKM#8uw+t2V*pQ%+QcQcd)DG7bq|7Lp_1;8DzG>0 zzTj3|z*kBlI3%GSzG|wj%Ub_k#ccOcN`Y*?0GAx`_1A-I*Wci;FX`#4d<n`c`0X1M zN7TTaypLj*kI)qI-DYI2L*(k{w<!V7N}s_p12c9ChvAJGQsQK(TC$-s*8Xt8YBepm zprz8~-=cfzJ?FIme3<gN@&=A=_XX9r!VZgO&=_?quQ(PGClo}6NU)1}mIiEM)(p3b zU$7w~+<^gkQ#Ylsl`f(SB~pg{OS}$4if8O|vdyR0cRoJ28eLA(9j8N>@)|efA?0`- zUcIdE@s#ob<-OUdyGHu_e7bs!sBkWHIXo0T?4;|)ne$IegEv2fy&g(V)JG0h<u1&_ z>pLD}6N+1Goxp2b-TiTGDPVrJe6ww3pj70^QrPf9;YgYMYn0-Q&6L+8Q30C$S622c zyKj>fUsbVDeUf$iW3o%0=JT&}Al!XptsC<p{fD8*1(OlIy&yvsM6*S<^Ffkb7sg4q z8YWXdgiT^C7*ft-WfQn3&69i5j95Qp>LfFMVn}>B#mzGP-lS}0wUJ3>kL={T=}#%~ zsIfs7j#aiNenGyTFR&Kx&EvBA>Fo4+=iV50{M>@4;*L%qerZeTibHvxpO|qSLBCNS zmH-5<!qP~2x8krwUZODJF<}(+UM$p&6XmW`7w+2lcxzA7J&YxJ-ptSQ3$K>v?c=^T zl8dwZe#7|w#pZ3roL#~Nvl?~}{So`aFi!&(Y1k8Vx){9=9EEu`g|EiPQk~SOK678S z?SzHFS_vqdad|n4z9E91gOKS%0CjaY^lV789OKnhw?%cqr&`(GmwVsJCMKvRtG_LM z)jZrmDrn^E*9z)=b^pw*H)OvABK3H@_MO$AGRK#6DH=OUt5KWvxvRb{NJu+`XXyoW zV6aPHrssMwhdUZH!l5sRhQ+St;J}IWy}ykLuNy&TYolBq%kzmJ<rdxV?(iuq47qFH zG>{);@fnBs1w9{CH_E+^JTxE$oVERB#{Q2U<2}UXT<>RRjbN?Yeo$GjRL_oo%{l<A zEmQgz&fFzb#IVlEsY=~pl*{_0?sZQC^@!62*Gq<S9s2|wb{yh#leq`c(+G3NSjKI* zS>Endgsuftc<+=fQWi1$IV?zV@qkJ-1H1lATuG0K_k1voe-4nc#KJm_XfBxjQaIr3 z5&1qv?ID2_5t8)Hczt78wcO8hSb4hEph!GX<Qr0we5}xH5`yVFp{xC)f^TU()d%-+ zE2>B45)9GE;4y67QxBMqh2K^-H0xK*RwZ)II;gtWY_RadKVUw%8ODduV4}wDG({K& zB{>W4c=!3KbAwVH^D!U5Kp5Njfsb;Q5Aqb_mf^hg;M=4V&&IcpyFuMJwW17fGXWmw z(4~MP>ntT*^E;MisCX4dg8a4>rSm>dblM~tXoHj7FMA)ohN+bO*!)B^@=IG=u28^k zbqU{uz=0)h{M!LoG*snQ9C?Y{E4HK9H<&&qsS^hl5ON5N7G@HOFO{y|umd)G^MvRj z3!NHVpZH1A)_PWANT1<`A!nePq9eMo$WKBAw+Tm-6(uKKZI#EEgm<4)9Ql<NeYN6= z)k-{Ac5ykLo68s<9KvcYnlqLaLm#~Jea=bD;7#7<0XBez>=7^4?C=8=)PDz`OBi&l zP-@Q@Zw@I&I*oTaLD!vIeBx|ZC-alih>Z(gG=>CP@9Qs>4`kc2Fd0Y$TlrT!)y+E@ zZYA+uB4G|;&3#s^sccG`*4ZStRx*vPRR=ubGSyd&K961@B>dc%3IDX{c;6?2_QRK+ z6winT1(xWhZ~;DL6}kpXxGI^^P(J^hEbUzviNwPDV;VyuD7xV{O+CR-U;P;-qu_4z z<kPC!3sXd<wGmT@se5Lxpu55YM~}XFGV}2p-$5KHssE0KJ5R-M<KXs((XEX6qg>Hy zNrEnzR-^f1yP-@#WuQTu92sJ?g9zrsvpO&pTRPB%z}Qj$<y)Gl(dNB(hpYNW)gTkH zTx7(P24+(apE6#QUsL|?Jbq44#rj?k#IvqWc*&y`r`28U0q1or!l-C9_u)41b*#;C zUQVX1x<6p@2>A2Vd|rn&ZCx~PUP4YkgKaDe38YLJSSAV=k$w#lCZA+(q27^S!cGlF z+BHSyRoxV?4R7qVud->Vbb+vdgyTcwk!D-i2NC5ZUWa6I?G5qA+3e^jB)Xay>Y^B; zSmJ@2OOtGp<Q|qEVy%WBPkg)K@RJD~ARm6pk-`E|Z{;qLJ)MTGyaT5+6bHg5OCP27 zVkGH23s^05tJ6w%o0?I<ap#)E$nU+jHO=xqfa+J5XLbnCGH|Xcb~G2<AzWY7P^lel z4f{fRN8bhgvFy_F{E)K?o>M?Krqj^#%}O|Eq^MZX6X?=cb6}KKPv)w^jno2>xq2Fl zmZLdMsuQ`GPktXcwPsn?UK5(Wb#J|dbK_&-VKZ}{s!CONSI#Jtm}fna2x_sylrFWQ zYuW2gC+=wT9iAPg$fbV{Z9(ABM)#9hB?d+Dg|H%qalwA07?;-Ejglq&09{n(dUcrB z#QOW^bsuGhLqgGqPqqRakQzElw?$L=G^q`&OG(-3m+$TaCrRT#rso(XPeE<oP%pl= z#_+CLfvK<C2TUz6dko_wkjtD`4CQb_jh~x4FKIGCAjSE&zWRE-Kf>oKjg~P0=O$p% zy=L}#zO9&qk00v@08pJO7R_%n9E*_3!-`w4mab2I%ffKFZ!=o%ln(;@F9wsH5_jAJ z^`p{5Fg9@Gw7WE1{1dhaE+0<(o^hIz={>L*xo&Ch{Rye7DJlhymyJ#AotsS+Z{+}7 z<10@uS-W^b<NW8eRqKE=`msu=M=Z#w;Mpc288=nWdtJxFrwnzsv}T1G2*>afaXI#3 z)?%|^HyNo)4Z19;WTV%2Ba(eEC~n^%v7o<rnwo{2EtLo1r3~IrX(N=ud+A!|Fr$X; z8U-0%d6+BTL=`uqz+%PQplH?_<f)=%m)i@#(5l9v6f{FBnT$@{YuD{u{2pyaC)~8( zvtaYPZ`vfMupN92<Sw*ge|fjf@Yb;kf3WpNToY}{?X<B+ao>Ac3XR^c)~}A^ch0cF zKHX<d(AXj~qQ7y=*uad!hF6H!%#S%i=;2R+J{9Xvk9kXiuQay3T4&3>`{=k=Xk=K= zh^>~-ch}ebH7N2^9WlhF@8g@gvR2L-{><$8Om%IoskdzsxgPRdb#dHJX8-t+jXX4d z|7m5b$=!sP5<^{>tlyA1P{&V(w-T-3AOmd#g9ui!n-TNimR8j_b+Yi)8h9QDYfOga za-r^3-<~b&(!zF_vJYb<{}ruRwx0N>^qkJ)8uzHn5_;afEEE0KybamQz<D}m_b3IN zTH9aQ>Rd)AJ%oB^DT05Dnyb*a?RvnMEa*Q6YrWi4zHLlA!qroKOVony&OKz5)qSfI zto?jM;9Vh@3h~L%eNv9g99LL*eHiGSetmf7LrS0dx%tZE`cIa)Dd`Jd!AM`ogYkm_ z5s9Mc4=o1g&uhv%#0LedCR^_dbAQ1+{cX2yYaBkCvD?W@x`}^zg35KaFoK(uvMaF6 z;wf^2;REq5(}Z4u^d<>hpKHb=escr{HkF76hQu{iEeYTdYBgRvs=>sG&Q%cUlQD|O zufNu;A@<6_u}+Gk>)XJ+bSp6?W^JwGD>K`UVavPviB~`2E>U&~ICrxQE*IW;!Jjj7 zI9;-bVeGk5<S(%sxT`0I%|%xsM(HJEW)WKMIzmv56Kh)#@x>2=1dktQ8WWKMvN@5y z<9Sa%_8O{^l4vzm@z7WWCmd2+^AeaNc|`>V;oX|jXURlxko~G}(jlH=@If{xaRp5w zqUjY;d9`qb_Y%u=Cw17!JrZ5B4JtY^A$3E4k&yQ)N0ORYjLYu`A1ZU?6+h`_&YGMB zr9jM7`}V)>7MAfU>Y^<wMfaDFOr)yIA2B00;9HkT{g_D(HTT=EU(QhN;7!RVq+nZR z&b*zbUoU050<4vIJ!J2bBW844@_*M8<j1Vs4=%mw{geT$CsMuM<lRrDtgo)|Vi;gE zxBV)}!=UN%pqG!-DEi@w&R5KEHICbc?dn{hlwHFUqS>B#iu7Ogw#8qGzVfGD$_dnK zDLO1A$^DTwC;CFHi!KRWct~;|V6&-DAh%Z(ngbV?vnT#21s8!zb-usD{jV|Ra}M@N zSEw6A%fD|%IZegddW;657KVRibrJ%(U=M!9^OVF9-Ebfik95tg>RdEp`}!nRK~1D# zAG^_?9z7(!Bvo{fud1`kq*1O|Wrjx)M8^7D>-}5oQ_2up&z`h&1zR_wpkSqVj`w&j z=1ERX45cbaF@LeDKiLkZP@>A$BEEq<?AX(y+CIU;7eJ5nnJ?+nyUGNAqV<&;v&k95 z+0uJlIT`P^j}qCsU#pGzZ}@yg+cUJXDX9x9Rq)~>j*K6t*09rl^%@sxQJKrTF(7;r zcdUD?_jYRoYUP5Ty4`Hy&8Cf+haic7wSi)_qF>D~Hsbh8a_=Kz)r)RyIx1krbPSo~ zaA44Jd8ZS0eKlZ+WcTa&+{9_TaA2`>qMn`Ri^~{oMs;-9v_s$P$q4c!sfR=+%)0sW z;pxXo?=?ai46foHIQQ+el6|o&+SqHesqZYa;T@5_9L8QOynhqLRkd8Ser3|3umoO> zTh}yT)3$B(fNtEHr5FUH#@RH7`6`gp6!3miM8TY+FjJxAL!`a9b!(hM3IlrQqs<{* zW9;Zc74HP1B5<548d`RmExe9Qs*P0W^{7?dOqF@7pT`I>*|iVU$Dm=`#@@nOb;#Bn ziI4sIFrh+W-j~UvZ+sS4RNL>5R^M;maiFSUdS6{0sksVVU%UeBnidg932_(CD(lc$ zrAAOcR0<Y@L6@W#fOa5B=_>5GYh8@(KHU=U7m?%mEp0@h{m^VCz1OgKt|Cgl+uRnT z*o&srA{w6;9LU5UbZOssoFjAn+2z}jw^n4*>u_GMt>!2NZbv`KufHvp>^btP#FnI) z;-a=92msp3ZxK_)4`skdcmTur2hmkm-AzxfCY_aG<6*drjX_x_a3<3>8P~pvF(9y> z5;rHA_SJ9naSL6tyqzI#rb@HrAiuK)rHH>n?8pFT*<9<J6S>&eU)qfJE}B{Q9s)C~ zXTxX_-29FbsmLxy$djjSsLh5iCHRqWnysOTg*Pnbi<K>0A>9#j;M%qbG|AFE>dZA5 zU}KP}CdkaUuTtVrZ{5yK487#`1V%FVHje~;F&NQ)1BjOTOj|8@MIU@A4N%;PD<Sd1 zIG^fiSONwZ?fFT=32E3KH1KtThM^Zn1u|}6E6^|1tR}wFFa<ylS2`59`+tTsn86{x zMf9Z$c;sSm?4o%DuI)m=Fq4bHeLySjfbXVv<CBYHb9((!0y0Fm&@9A|GUdT#)(-oa z<OPqyp9mGBF!%-YIZdK4lC(IuByb?kmib?zMH%o*)waNx9FmI|xSV{p@c4gU@C6DI z1c$mrTpS6K4WZ~2GwsvM|7Wl6$pHrPgvQWhLI)7mP!zSv&EBaZt4;d7qjKH360Y7D zQGeWrFEk1<=^}Jcpy)`M$7T$$HAH2|9b{d(lXL$DpS)3^u#*UXz#xtHHR#ftZ<~Mf zg<y#`?Z}ar*Qf@_%9QSmW?9f|(ym{P=D0uFKF=l?Ws&_AdyY|MT8h!52UYRs^5$%W zXkY-s-W$x6Ba+a`uiEF;JC2p0YQjMJ*t=n;-yM4$Sx&Q2PvEmZA{nGJG`2WrcbkG{ zf{hj8O4nF;Mfce36c@RGtZvP|4QB0)h-3OYXRstlKo%xV#-w38(A;~><A>iZ)MWsn z7BV}djImh&b$@6Uq!UQI-F@<9M>7M2+Uv#&ie@`LN`rK$bhVeXH1pE)SAECEO-4${ z1OyWj8>$#?c`xI2LCLEwo<G&pfmJissnmN{vMS^aC1TpcZKv~n(wmOBCBdSlU`icP z$D^VztmPDU<a?@q4HO(6ivFGCMOiQ<d^>_;@<^4CZ+4X{+_^b#%O(mGyobF{_To}n zVkhd77d&Ea@Z}zu@>+>?$X0FNXkqIhu@H(Tl;_uW4ND<ickZZ31)e&T>Ba)}e0qB> zwH=h3;B1ABNQ**qEL|PEi(78JFM5bleA1oBXipy=jg?m;|FiU`fwr)ai&KK^kFowL zpY7z|%o9a~&qE3{{*#@vPA%X|gOQ{IWn&+Pbn9A2h{?(38!8A$c#fB~RM5Uz3nVJU zO#@hpt4ops?^egkhpKoh$-Zs;{!*4}*UZsIx|B`3LN72bYT1aqlF#oM%{Ju449JV~ zLQ!!Ahv!E&1|RGlkIC6pyfi_H%Y?0leL9GhBby5Z@j#`J&kn!W87iL?wfQ~bsjw1b ze9GLZe5uLTvB4J?1D>s`%+BQ2@noQAN=n;G=wvN?Dg!v;9bG>EMuqk<u`w@>6*0Gy z!z12%$FqvDb0Vmier45f!|#2k+gn}Ax19KX1Jn489_nVxD7Rw!ufT(ahCp%}kfzA| zRj9q(@PNmVWwZFLT2{!lTh1?N5o=TRY+^b&L%+v*Qs&?aLf~XA*!qi4bA#%6gmq++ zkae7{*D)h9)9;sKVSPBp-+P4|J8(65G%rI%GUkWRie$7-^vq%L4&@^}EKy$3*o~?$ zyx}3jBm!S((5yQgN52sW2cb7S4P3-nxjStOIJ+|=V#10EB&nEM!tdUW78*e6+(m@g z%nGgKY+9Z;w`G=5Abj1r<M#)?I@FWSEa=)=sTB&nvE2@}9lYNfQcS48MA(W;3Yp}v zUQod!0u+?M9V3}>Uz@ydCzDXT;u=RWx(kA&U3<k6GRcZEpvFeYEIf|Q((4GR0mIO9 zs?I}kBVB3b?iZ`qf)j=kOCL>VnppUY3s>hQ5_H|GTD25j$z&FHYCKT4?Rd*kR9^$4 zPKu?+$clA#a*N74vDl!rTK*U!M=zV>*GrPdp8SuJxRFNr0&Ih;R;)2==08+o;QvV- zT|^mB#u-*a#oy^782;2lKxU!jqsAlRwhmwlA&Q|7k_F!g!?M?H^u5(wBEoeH4?Ki& z{A5YT#m@524TFD7RFqjxRjL0hTc+V^!$BC#LFHY_TIXzf3~8Tk72QnTDzN<d`$sf( zqv|c+;$MhH7tXcy{?v-6!+u9p%UoLW+#*M>lN0ZV#&)F`#Z7p5P$sgpkrp<vNf35L zcmd4u#_E7-y6w<qyN=SBSx|B%cJE6YJaVClrKhhIf!yjR&K+l8cep3JoZ+<m9ls4@ z{q3V3x*5JI5Nb^|PrdIzwagrK<#}N&XSbX#tfBGB_!Ysz&q0?@UOM8T>B~8Ne?pY5 z-kX|ghX*%1Y{K;)|59+-L-V7VTjl=zm{lD~^4nW%y>D)eN|FQ!9NCHMo1d9gbHFPz zSIetYs^UIu)2@V-+lOjvp|!SMv{+oK@%E^h1Gv5D_cj+uzUDS1#pDEyuJ?DOy4%$3 zz7^o|jF0|PS>bqU*u}Qh5R@^v=JM-1zTKpeYT$+&N!`h|1?ga}(I@#~oNSSb*`6d$ znN`n%C@m*VtLm}km*E;absC|XiB|7d_<m0{Da{Yh0Z&lpt{{adtA`qfXS*&`TVTAl z@>yQj4Sw-6PfGBF>PbU4iJ`?bQ}*(0Bd4+Zayi1~$G%S7*%WQL2jMt`8Q%lYDe;2G z{pzgy@;>6HHThfdhEp;r3bk+wKGMlC)ZW|=T^N-c9kp*KYr^)}U&zx|T&bJa_sJT= zVP<=C<o8m@IhBG9?-u{Kt>KjhY^22(v0HPUWZCVt6t00p-nRV<-uy*312ohcE~;l~ zNIercQD(~{NlQUV{Oke3_G9;P)j0I#BI8hT*Z-uLi^a;WqZ{LqWbv4qH@}~mb0Bmw zZIWCYGyE3My=rswW{Ce)6!svAXQ_(%=wn9%Z_48|@G%^!Qy*wEc{bd}vR1pR0+Qut zUl0w&6*Z~NhAd6=<ki>1FKLdC%&}fAZPWBL_8~O@-+=gZ6oFv}eA{ryuvec6(BSPR zlhek$5zLN$EqiJ|rO@;?Q<5Y(UMc_~uRoIfdI0d^aa?Q_%^IHtIBN2Vn5ch7=WA)> z@DO4N$7`0mX5^;9Z-T`B5fRB7-_i5Zfm5rP=!@QC)bE!OmUvruC={ZN;jN;0IT0`^ zkzxLi<8!3UnU+^J75uNe2RaZNG!t<bQ|V}Yr?<%DE?V+cVfOvuP`KwiI9#MUKZZfn zcg%-dQYvwkgOF2L*<RbYZQGKJ0Jp7fH|*N2M}6}Yh|5S{Bst9|$?#|TL_U#?e@G?z z)Hjcnm!rqB(We^ENYnuWMXo{57IfS*D6$u{Jwr?F$iD9DFm=%sxGw^P#hM%`o<T#t zSva-g>d&RZ>}K$p8ljLGnbOt5naiTL#VxvEFKt7tr1AxK(nH5k^$%kC*P@{C9fcbe z8E?}W%w*Kv^CWGHaHEY=FoF$H7*?Y{vxm5$g7U=zv#c7Vn5e0xB;dUgctZo~qDI+< z?W(^P`EBuJ<p$GeHY{syJ?gJjIuHYFujeZ+Wr-*o0TAXPZ;ojEV019c)ZYh$hd8Va zGMGESrvggTL@#@hI%V>lR6Wd8^WAzjVX1wj!9*W=$3UtioudQxKqPvg#Ci1#!WErZ zc57_AZxC@GOH_r~wlWRFm@M?xjw#5V9`E0rDx$g6Tul`^*EiRihMwmj^{jr`y*F3o z5$U}h2QsD(5J^pL&-b}39=SS`j9Izaplu7XCsbgsRM|DR_GuFMO5(IUn{w?^3d<g= z*4yypk)VrA?kb2F#Ao6mTjb=uPeBjHKrO^YIDc{i0oN|u&G(s(WsR3{Ck<KZzvdFf zS`|ENdWs2R`?jXcFVJ%4DP3nd?oFUS!FxISwL-p!2T7NV_P2zLc~Srmulpy3TVz`D z2SRGS+rcS%jxH^ob}1eCAqDTNE{Tr&wtvL1PHf`4Q`Q)(p$UN3Ti16=9P$T?Qf_cR zn$)?YYi-jfYh7FYTKh<<4K3%#h*PH)(-p&;&N<HAK_jiTyKgkJ7FS6H-)|{Jtm7}K zh2KpMbq*Fj#FNx+z+QmQh`DvH)gRN~UbkojnF;W|EA>RczeS;Nm0ytghmy@t=bdgx z><}+A32~0$YTWvZFeM16)maNK1283gDZYj-2N9*W3SmFk;JR)iquAt3CUMnWd12ry zBA2dY-<JECwWxukKL)wxiG>MuJb7hH`i_2KWK#<KYOKOi(XsX!)5tcT%Pl4JakUUl z^2IbBF8@&IDYoynW366%=s>1NK9xN+)TptiHzIeivsP}<Yi%gaU1M**6gIlHUWsT? zM&-&E_CFe=`CxwZK`~?X$olo4sz2{G0Jva*lFV#ELK8Ur#6My+b_Q77Kw&JW0txfh zl`}e-lfN1YiUujlnhK;_{qwYjc`BwFN(ol$NN$K+MZ(PK!i&oSrP7K#yKgVa?|}&R z2FQeVq$so6H7$jZCmJ6rr=O^OpNE%7?zdOBiWY53(SAzaOL_Ooee+{zF~R}VM>9Y) z_`v1(=-cXkIbO*YCn^lMznN8<1JVOFtKLoz-@aR!SL(`|42lksxnULdjbDFmom30I z9ly7&^m#qiNRn)+zjYHzjP>}5X{=$~*9(P-`urr4Eshpqb5{%dbFNJsYW;<`c`Gx3 zv&W!Y{0gmi^B=g1R{uAgxesy4$;==&e{Z%d&<d#M8|Bh)m4>eBI^EzZm1t05Av50Y zXvL{LshetmMl406Oyh;V7zp%#>hmJOi$2~rcWS>nxQ1ip<IrVcZ`_X7I$&9hfxadO zJpJLtt+--2tmwDSgEYD9s)I1rgrA~6V;8kox%Feoz0A$3`6wH(Z8NQN_ENTpFco>> zsa(1ka!9FdPx@!eT;)Qro)KES9;)lPYFC8s4fE-7a}P1Xd~N<boTuoZS1817I*P}U z)Q_j@(W{qaG#OB|wa)o0Yf69Q8}Z!<baPK1^|;4ha$AyC-w|7R8gOn1aDB&jneiWS z<a}AoT62drPPT-Ke7wY|m(r!+=M#N|#k2SR!Au57<~~23&&fUE3UaT~7J9<Ix%3e> zLyIV+v=T5h2&wB{#Vlt3;(Fx>@`Ck{F(a?BbL<0f15*wF{`1stVJ`i=5!_-n+;efO zWP}c?uN;{ZC6F*=U@ItGC>~z!QVLV=PIUBTC0N$>)j%?u|7Ml;&%2MDvTfN|al2Qq z=j$Onma$n>UQu|8x6g5w3^0Fk_E%F1GIHozXBfM*3|5&*aSKg6i>qT*(%sL!{@BHl zKEWye2~~H@J=lF|jA74NqVyKG_u8^^=QJLFxx%aeK`y8I#~ZScPY!yTju^)8tICTw ze$_ZMuj?P7G8McD)##9O_&Ii4j+@mb8`Q69Wmcm<()?OnEMYK><WTJYD?s{YnbomO zK#$h7ek6Q<YAX8F7wrM@&#VWekqz*>z7dQS{T$S9jM0qM@W`|%(1_pLdS8XV6t9X8 zXFFD-OU2<bv_70gRzH~JpSN+jMVY8EU+G{Ux1|Oq>r^rEBo7kZWnk(0>M$A_^-<tk z>{#VwZbH?Txu+*Mzn?kSg!v$ZB~EP@Po(fPo|JlnENUjceeCE6C()ioZ+C=?Qrm0= z2!!{+{Dkprqcq+|3oanpnrMXpHDp1l^*b#?$ZP6mL5NQ^<M;;V8#To@U+owG(!_fb z-;zpX&Vt+ey~MBrQJhwTtTxWX^=Iuld7*iF0i*)ddL<NhSS`!G6!_lNg~g}dPiT4F z#M<w7y_9nUYdTE*Hnm@6UY+@#|8;|cOJdl3L1~)xJU;rHRonbI0eO`fRpVbiBr45> z1jirhgFH9I1~ux*IW_9EdOeSTyAFlu#<$d2<^qJu>6&DtZ~yxF6!n>IK#<B_UI5Rx z8U%K%I*WDnfaezQ{{Bu)O5i}q>DufnUn#dTo$0CM7#<f;(=nl+yKdzt{d?${y`TZ& zas9R26axw>k=Cj(ldjH8>uwD;qv$A~5pc}B!-0|=FMUluLNLlCkUm710*?%bWBRsA zwbH67LL|FI;cu8m$^t~e2dWN6oC(?;kQkZbp-^|vcu)c}^)zgS$NNy92!(n&kC-9@ zxNaRi=pGhHEj@V=G#~lWhg;Xz@&0U9qIG#&qYz4hWPwZmpu9YWv3V}iQV}RxWr|Yl zYo*PO^&^OtW!a9|CqeL=AsX)`f$8Ke3)}M0v3X2mKEpH%5oA@a5-JsqJiYDp&hn?g z-d9%@L+1*-Zi|XBJssUJVdY*{=Gz+3ELb4wh2yes#OT@ln8zmFtKTnPVbVjhb<51g zgm&57PcBsdBeDIhFAfQZEkWekA?=rAgOqr|p=)_LH7I5<bCsnGQ!Tv0xF9Gv7B9|9 zFMGd?JjfWtd}NOAtZ4I{kn%S%Zij+O6FZOAP>^icH1z!9E+c&8{54BG{o=89<cSJm zYd2~H?z=|h7J<Y<87D<-r!=(taj=Te#X%LEPOpg1u5uHy6E8VoKO|n65&xMVye_>2 zBDk7UTk(?M!B`K)_PB%)RI&-#kl<}hA}aA*3g)b*?d7(eMOeIuGHKfNkR=ct7`FXx z;&FZM(f9>84kP5nl5t`qlSIOhgc+f7HCu`<)6)grZA*48sxcA25p+#z71^9lpLxKQ zs1ejV#6vUtgh3FF_RXHPt`TLA%}9F?PYl~Y;vkh!4C(66Xe9tuK{%^}1h}Z93aenE z+=8o&V5RImH<m<yxS;!Z5>w6j+sQ(Hy(PbPgXLyx?^C^{E5qHG`X({eT`0|7x@DxS zdLRc3?=5T@mPcruRP!whQ^i?*tCfa{9M&$VwSUJ9=BBm~&ApOoY-}#@yMU(Z-ZIx4 zJl~m~8@%F^8h3jscWU7Ig&>~~<P;(bZR3vZIvjZYV?JKqehA8RRz&fD!%}lAnwYWb zQ>?L(&AFxCp$;ip4WB^z5)2g!tmJsSV!eQ)z{CU!2g{gyYcxy5Rn}WF{H=W9Z;cre zK68AN8F{bo04K^}1JHqGkmZ;3Rb=TK2wlUPhwlx1wW(z!vF(&+b4xA7WPbGS-Nchz zOnil_bCg=2B68FYBOTzsi)ROBzH=3OzH)GH=0n)ZItdsq6W>zn!&DodC+d3E^V?FH zFtITQXXVwc6JVp_t+?`L67o@n@h>Xd<1iPrZ3mN)%(4G7kC4Bs(}{yj3JF63W`@Eg zP}A5Asz=sdIdYoVgZb%wXyDa-Lr6gTOW1w2MYXf5(6<Rh^o5n_`Y2z~#V;befKySY z_rZ4n)yBH-C+7Tu%qa8$3l~ZPE`7X*i3F|?rH0oIp^7o^n-}>IA;f-(Qq*y7`SKf` znDuH76_~y)6;KOf;cbk);uvVExRw0D(O62AUF>*gN`u*+AsiDU2Hi*ZXp$I_1t;o? zAQ74UPP6xYgICnZS2d=3Ep91<Td&T2FQS_se6XB@HEJoeno9A`L#}9E;9gu04)yoY zC*5iUl>l31gWg&BR9o(8TC@|jMz$wDzXI1>t?BI%&l;m9Z#4K(Qrw9-?ceg|nR|2B z=;7Rzdr%?E@htc(IKxLV1nk6AYdXd*WqmD@SPV%9se7*EY7g}1TT0j1%6+g9OO`R8 zq7q^Af`aLirVpNk1YTKj>-1S9WJbPEqTHM{SW%j!PE<G;5kZw^Y&G*HSa86PRPXgd zuD`w50!5i>mP@#DmUi#&)OvpU?cN+nmllRO<Q6*#%)G4{>_XyCw|dWx#iiD{W#w3Y zc)TRPXh#3;?#6JWhWJ+n2g0pl_NFjU1V!|N&?v)K6E!rfvVE)Sz)CZN`-JCv*(Hp0 zF5qrn96zw+$Cwp_O4&|JAd`Ha@|>g&ye7!p^@eG3V;WRRIjHXP-Tp2~NnwLOQba|$ zf3mxxy3~5TgPRqB+;D&Wn;7Ouy!y{cAaEtj<b)01v~4BI_dFVJqlqdS?@{-`Ds4v& zJA)l2*6NRFP_Zb<?Jz<e>YX<{PF99CT0T;B_TN;91jpgKIq0SDme?&Lqd@|crKYhi zj9BbWBMM%^*80iOdi4u^kk;m|jSW;_A-+di6#G*Z9u$FDM#be{U0LMLZp<^ZKhy%% zu$|^a=QZ#d^|6D$C#sLpu27lX-`5+PmqTpreu`w6-E*K(ykD98_jI@s2~LMKAm$S# zypJ#C(!hLFb;m~IZGv7bZ>h}k`_)6+>gQIE230n~KCEPXy|;iRPT%}QOn^hp_Ws7Q zMXTH|;jYRW(@<4*crf3u_`l9F5KOoPcCWiQP<SE=v@xEZ)Yz7$N36SGuz@D)^Gz<u z5CKXaLNMN%M7u41)MBlcZr_V|BylLU1e4qv0K?iH0k0woGgsn@g_L?L<_`up!%8@6 z2BJjWb6c6;2kDmqOinA&??({G#7E8MnN#-?%1{rKjWE;gN7VNAnWBzr;6IrqQijmx zTCq9uoS+>UTy9+A!<GK)13!BcJSeiFR}&ASJTMF3{xb@t+WBkcpCi`VQ2_(H%bd6( zNOX8mB-qaf>z|?c6Iyu*8KY_2*h>HiBn|6DEnYhrXRB2`rj4VD)9n%B^D<X7&i)%E zI1l<4iR*0#)Y^^f@1=WGBB|dF<cE%dpkmdA7W<fu`QEoBFL-9lrgTdb&Z~@`|G!kD z2PlqfL<xn~;mUWZ>&HFl0TSk?D^Co{Mu>1Sma*wxMfgs8Rmx7seG`%hpF5oJ;)c)2 ze~+4{)}Js9!S3@mLG|MaK{HRC%aswy(l3W4atJZf$-VvXx&#kxQ)G9ahHL4uy2p_Y z$zBYOpO4ZdZ?El_U3>1~*BIj!DC^Q-sgPMIoQEfc-hj=L^c1BSH{_Yh_h#R8*_ZB> z1J0Cwf5r4{SYU-1%p*0bb^8wWcNjH=S?>j0A$~cb2^FKp@%95B7uc12y8FiKDSYi< zF4A!G;XNI6C6nmFoBul*coCo3yjJ3f#<n|RwGT2jGSQ^B6hmAn@TiNyi$C=iZbUaW zZL;xNi!)o=9(6Wu!R=wl!0lydBkp1Fzy5^-OS?<^e*<0s5W37;+<3HKe*Lv&isd1y zQu9k?HjfBpL4N~vyWcPiX{1asSSkuv%RU0(u`!RRMU2W0UG&tPzoZBH=cA&54z!NU zmg_|f9<E6&H@%ywBO-hfrbo8hJV(rrkMFF$IR|A&u%Za6AG~DNF8az-E|<u&IAB?J z?*5^x?{Ag4M!@)lxt?4`fi?^sBr*!nMfCZoQ`&t`J$nOKFFLbU#iIvd<YwXZ&g;_x z*=V@3|Ez=y@jhrIBMQJuxM?|5U~q9<%|wBCzj=Nq@pKhM2tY&%x`iJu5@kYQIP+~L zO^H7ZL^MN0whM(nFA_0A^$TU5(~^PZ1T`lEh%yvyidZfZbw~js&g~mZr(KO0fQX*& z6xm)Rl7^z01kVYwrA|5NKt!ru#ibXCUP=O@=`9S4Q+qCA9wOSiq_=r-hBw{<MIf+- zq;kG28$sX>YoWpBA}6|tG4Sjhk1b{RndovtI*d?w{NM4s2YGODkd4>R9>#<6+uk+D zr~D)EMRDNn@?6Kjxm>^jb0bn}tbdd7n4tvEo9^T1;N+qd=rNN{f{Wdkf}w-WPQ;Xh z?QH*>MFRh?yp$05FSA>q2?_5exwDQ%kU;N)>OgreP^VovrWI~j79!R<gY0_yLAg29 zRo*cdnO#t@&~5K#C<&0`0AyI4D+F*`iVD)gAM-~P{~UX~>7enF=6f|rcacaP5Gj^f zjIf^)J%!@B>TY3M7d0`!2?L@Gh7p`oqG*VdQB~wsP>tiiZb>NuqR5gNHLFviuMj7P zgIE^Yi$s6H$A3D02h!-Fe2H>@qs*Y=Qh*adk)|5sDW~W3?|jsO<uzM6r1Q|^8n7uY zf>LHNz-YVpB%z(L@NS33*|_N+iKdGngGT#`5#@RM=7k9QUv9!U&><Z8kviR36HooY zgy;0l)cq@p4xocYJrOt0&+cVVVq@-I_3Qui3<UxLvU9Hf*`WY~i!h46{31v}{}b@1 z9nXfc-3TzZBfX3JFMfa$+!YYsjX4wJjW?iUHLI5?FUmD+0t$Zb;D6pe-D2p-0c(t* zn>lzF`X&4Z5^7_Lp7~Sf1DzlsdO0PTLw%9R9P)vV9>Ec(Rs$V1Ad((7%i+98Bm;3$ zoaksh9fx#C?lb#>EkQ2n>COi2L7Wh$uG+S$(>|rs0V2kcT+53|rosdeC-J0~tkXWF zqXDAD{$9(Q7dcHsqg>(9^yair>5PELY;@g{=OR%fG=uEg4X@6+@cA+rxVr>r>--Ew zVgn%qG>--N&YpY<Eiz<+y#Ivxk8o;160B;wsdv6&?|}vrxC~BL!MRkoOGEP&$qGMP zG2V9pN+cNhhSSBx0rR$mmMEEi(XI1ohE_t6FAe<PHg`eKf>S7V&eo_mIkeOy3yL0{ zJ$6bR1YthnA9K*2^?w!z^sKTeLGs@=rJ$8S^B-h7LK0d7_|B9dS@&Gkf{SDDnjvBU zjq&Fm0c4e-Ea9mHULd-Fs5vD{0z~3#)QEFk3_S}Qgrska()i$%QzVpooxa4WaxwQB z)&gO33UQ3=&xjz6I=d!TcA-O3uu6#2>apR<sh(Ag07Tc8m3dxX>{BMhDQnfh=#*%{ z8xTGJVN~||BGGGz)9j@F#A$N_kV#&$?8x)xB9S-5>0OWBlT#w8OJJOA$^XhQB*v4F z@Rc9BJU^d5X#2ZDr**oIo-grA$PUe$o^Lqg1hWUT%xTh0{&&4YyKWRT5p>$Ia(}i? zVI*51w8Ut4RxbcWzyl-DHq6ub^zpg;QGmyc&?I92ZF3zgFk&J8)pHB`9%L+p=^3T| z(<UBtBI9^(aGrABP63H+AXoUmZN7$tS3F?pY=eV^ga5;6e+e*OSYwwk4gnyHG!W_R z9^G}o8UsO{1FF6LUH7BVl6KVYxpJn9_yEz%NwFNNi$qzFa`<__Yx_*iPyiz7A)_4j zi$t#=P6ia6^QYYDgaA>;RG{VMi$r!1r}3k<(K9sz14N9&nU(|>i4-AD$`dV}XKDr+ z5H0q>Ezj2x)POJz#Hqcusp?G4+yq2sBP*77E^<PGI0+{;WSze;7^j)L|E}d4NPpPu zZT>i)KS)Bcr7C<b$R0SsK{3P&=QCl*Ml$LNx_2f*rwaqd00bE>UB!HkiJOL07LMt- z|3A!%UIS*9ejRD&^2ZO&AI>Z0^KI|+{xlRp@pK$NH$Cj2;s)dmoudERHO4?h3?8(E zocAB(ImOT6m{9)PCJh)0;+NP%rvo1$ObHE*<Tn?!e^k~ku(-HzP|xm3u>;lwIbv!Q z7l^z|fbGM)QuX%gsVbFv4iVJ`stH{r`WyZEqk*OP08wNfTuuKX(KU!u3tx2isX~<U zgox^KN02TO{RQfuauNYViUmp|bQg)zAqyq5DmdoUypTczL<6aDBj@Yr&rTr@?ewqR z0v%3m9jS*9QPJ3l=|xT`5GQXw-zVov8Z7!&+jGrvI>s&#(sFWRq5pjSK@v(D@eK8X z>|Mf`y$$YGzDh(pGp)S4AmMn6Z+|X~f2f7!A(a*NT>X51Y=&%q^1cdCr{ce53QlCS zfY}Iol*FmHM+iGYJ1E69^(+67HACLJ!{rKwv%9nCmx;O_?M}A@+s|HSj{ZjqFIL3( z)6)OiXs3IpHy&u2`kL0+s_|}yM4DsYyY%8RlG267vV3jC?$20Wz#yJ7Ih{H(i0DK1 z$mure{S?|Y)m>k0{bTAr0Yr+WCL^q;oc<QkxX1|~B1)$n!9FGW3UPASk7PNYH0ZXJ zJRpiJo>H?oCAtPpzdwJR+&`u+CPb8$s3v_%^c<T0B-cc%e@tC*Ktxces>X0iR046z ztxgHOh>ewkygdZ(2-QwkkN0mncBfUXguAu|TA{i%&bC?ywx^H(_a6T%5W-=2!?kYZ zT3$z6Mfyzv@o=^$S<&Dt$D#lM8xlhL@2Y19v*P{Q7VS@ufPY93DdzT*$iZw?cux|) z;q7Zaq{rYfzu<&_6%;Gz<aA?e(z)hwLoczJ(oxA#qh?ag>naf@g8v-KXS;giPWv4p zu8RT)uA8A_qWgFS$(e9H#qtgp+?S$s{gE1<)1}P*-a)<WoeA~78yckQz$b^#4HvsW zVP+NM+p+ys%uf4$|5!p&HlS|<)h8par=yGl!wZf{z%(vlefR>$km%xjC?*50(Q5`g z5q~5aMv5+8%&Aluf-vgni^}<DpSTUJ6U^Fnp3|41yYFoZ+K&hAFIFd9{YQogK>LyD z@73f^+yBb&&BW+~0ckE=vHoO4^lYG^Cx9;k2!QRr5BHhkvPbefW7>^|mZ)%MFF*qk zkqv3%+t!pJXKOEw@I}lB|5^=gwn{Dm5!_UpIEl3UZ-5cn+8m*+Z9-hBpr(H3)nSQS z{qjhhPpDxOGy<>&bob-E1x~wJboF}A+G{Ktvho7!D5OH3qCSI>gHV6GztTObc-Q{e ze<Q<qX888$%Ls#Rpf<+nBupJ!r{o-Z=35>bdAV}Gy#O2rD@GG3khFi<=luq}l!2V? z<B<?3-Eu?XTFd3#f~9@oG;|t`>$Jn@g^F0nj#mzmnNgWmR!|-mmKKyN;gO9gW117W z>ro4`?-I-62YH)DCQ+=36YyBRPVe6PFfuY?*5hQuak!dL?dE4<#^BS4fhWZVV}B@w zdgXFm_n_^X2e05!-g_6ly8>TddGOPW<~F!#QwzVibQuW+9S>IU1m*ZS?T3XW)8xQJ zL-z^_lCaWp8JC|hK;bQij^4%N9rGO#o{_k`y_BJLD_=kP-q%<CMOE_<!6}OfX+6i( zFp_##4sOl14&>Yl&MFqAJ@cCo2O1bV*HsnNdHAc*(2k6F&77G-DSn()giqb88NajV zg<(mbHwoU<zFA$o^N@D~lV146i_cx;KL{&MQeU-AW#3iR=nY>birwnJ3-SEZsMPIW zKP*be%^X*h+mel?7hV)$;I%WpYhqrgz*U}j9;fY5be-17@XI>ww%PSdcHaLrl_9+8 zI`3OlAfQslRQY^7JAskGI}HpOBN(!K%f1o{v5uSkHdSzab;39I!1$auxXZ@AUim7a z-eH`tngRpr106nI{7upyTb=HApR_c`-DZt_QmjqfgP7UiKOGn07s52&aYj_d^9q-& z!yF7w<b&Za19URMqr~7*%ERM<wYsI&FDbPZ?xGrqwduF#y?pU`EI~)$pg(x&BR_`# z9Xl5-=t+c#uwJZwH1DJLpDmH4tpA(Bv#!6Q7lu{7_{_YAqQ<DJS4Q(a3gUKurGx*{ zrAwsJ;v&j_-sZE$W$zIU+^IhdzfpnTPEN9FJ(14~1}rC`)+kGmwdzJhTBAg-K9a@8 zBf^i7H$z!mt$XfNp8jIM=N2sJR&J1=-yJD*8Yv7J_f706^4`^nTBjXEtH=C~;m-2g zdxR$m_v1E~59$aPQe_&iEn&FZ?Iyc>9edwlC1*nv!!BJ|_Lm6?2=D%0KpwGM_5SV7 zQTJ4uxXtrb+o`@2k%8fW!thDzY=IIDT@Uq`<u7OfJPXKR=T_?WhF1CukcN=pPvb}h z?oDj=_Me$#fA-)CVDoj|Crkdbko6$bRLukdw|&gMNpL;*B4P6lpsftLmF?d_?dt(+ z&Nww_0_|EClKrs))qOfIErERgl#jpcBbkk|K9i%nou6U=xhL?xB~U5gR6l_F*bikn z-0{9zVopmFPcc%hS(c~C_|XfnINA$*+LNqlN9?YFDX>#C%d}zejvBqZ9`6GJd<GVt z(H)^anW@72m74*VDq@AKu5a(&rVxw7!#b)gS!Lrc@EPsW@&~FTw#|1sw124QXH6Y% zk=SjtU*5CswsV<V;oJ}j?uq=d-mEj&(0DqE(7$-gkghtwws-&YKgfmBCu?97R_-%0 z&lcO6CfzhqbX&8!Fgo+-b@NNYAuCcZ0S*i2Ao7WNNvGWIcNmcz4;`};KZYIBh>dag zrPvcYKi~_0qo2fuR?U+equ%7U6y7qKd&^;ToBd<j+<i`;5#NKI42z_lxz^Hbl_`)S zc8e|Axoy7-c?hLZgCQ?pkM-Qh5{3cYOx#}FRtL?6a5FTm*UR5kcak+MFIY`=e!Z+s ze*K!~B6sEURiI^8Lt=2TayqYSYNjB6*VF1n*!}pJRFzE~MRGg4QcI=WSC0L_;Y5M6 z>LHhf`(5WMQ(Losr)V)sAwPON1RsX>n|uEai#L?8@~#(l@Mm|K1l^dwU;M775q#+f zA(dniTXwxhrK;wu<%q;|i^99jx)Zxf=6H5xs-{w#-L%q3m_AMR;Z(WxZIaKrv&{`2 zb_eqg9gn}6b-d0mNS8nHdvXQMt=uNoLX~x?hGgm%3rWx0Xy0n^bpi)D9ir?dP|;}7 zzJ7I#rT(x&`?%6g(k`ni*(JlTSs~(43KttLm0$$cM}^LQUC&q$qctDu*Xk5yRw^Tp zMIEOi>0NZKQ{T(HxnHYUTbC{qCxTh`D=XcDCQ4j|-EH?*D*0&fLr?zoY+@3nbouQ4 z2qjFxlOj5qx6RLoT!o5uB=6ZT2OS=dnMo(`Bt7kXBNpQ!K`^RCpTyEdz##ph&30}8 zwIew{;^zTVB(wigg>f)FPE@x=tollV@8c8?7U{^3>{Pisn!I^f&Q1PYPe!pf+}F#u z$kS4!?ro67ezcn}?K}b4F1zVIL658R>AxPf{1#}!#AS4-Vn~11Z?HcsI0f*O0u6J# ztA@52vbhhcuyw4bs#1TBvq|n8rEl6Xk|=k%O{aI{aL#{D+_nv6*qrI$<5<x>nu#P= zwOEbUSYjO1TDYw&UNq184zcil?G<C2(|?NP1X}VZ3qkiGGuZnRWLsvvykhE+46q9y z>HSIu-*-ja3;zA~_e)<`03o5+Xoojd`68#B8%}9+?vy-v%<H5Z@T7X1CHrlXAnTi* zDy2oOJLd0EIiE&nkBO`FrdTI6i7c9Lm9+of8)wAXD|l{kgHsC}3Gz3oW4lQQbsen_ z2U5ueVRgfS6mAy%mM1+doS8O(kBlPv%|EL#ri&N1103}Y6P%Sy@pqEbk+sy%EC;<p z(&4|Od+M@xyU4hN9lh%Nb!4gDTW$){&b|CtwAkwizCbnb#-EEKPh^kvy%5gL7ZcSx z`OmX-WO^k=893q9RPMJ@LdeqdbBF5UYWC$_0$uSEt<(+za9r@jb(O3?lBqI(-HT_S zwVtT)l)F-zLyH%>oGF_du1qC_P{)hl3w6GezrDw*UiiUMt5PaXgv>C`D?7ZGoS)fO zbLdN358HkEt|;cp4fT=9vbDuS=OX#Omn;@S<<1&OifGT)SMJya&`HORCwCkEHxok8 zO!P7YO+Y6EZ&T=?bK~6=gBUOq7D9=%ObfCIG57oj$eEAoT|Rx+sdsHD!g&})9Yu`B znj*yHt1(#G%(*-tFYx4sk@WMfsydMcPsQK4JHJh;U-)1X22I8c2UIQFR^Ykie72Wx zxDPiDi^`8<>1(?cx066pIv9>`C!>;ebMQ?G;s3|pSARv>b$=Tm-O`PO(t>nJs~|{s zH$&IZ-6`E5sB||BodQEkH%NCkyw~vD-rwi_AD*@N0a&wU<~p&@-un{=Q6%#-;hS7f zfxP||c>S@}k1?OBbN1qK$Jo&U#RdnlqZ>7=8h)Oi2VYAfZYAwDqZ=wa>vO&zie_U{ z33FF#zlMZV`E`{7F7-Yjb_#(VygX2S;1e`^;(sRC;#NaG&%m>L__#i0;LE)LamF2= zr6iAClls#7f2IzOvW_cdp5I9r&pZ)4@vS1iHjRw-K_h)pQ?%DQmCVye*oI2AmSn|L zZA5q+tr-udOA=^U87ahqrcGj86X>XM5b5Eqxyq!KiuVLp{b%5HnYey>nScnywLL^j z-YgFnF{$1IDzx|nV?0W-e~Jw62UtjZu<s;;9g*)9VR3d`?0J=-EI=h<zHynYb&W%V zoF*nNxKDn^>JWjd3bk2QsQqHKoIbl#VK8jFq-;PB7WK&S$cftXO)iInxNStD%Hc`S z$`FYaIf>GmgTJU+@7-JZHRHkVKT|cYF1`&M^!qR_ibQWC+h~82fNG|&SYXn=p}}}W zRWzmOgu<?6#?1mO<kg^Y98Hz}akHZ0RSHI@{Id@f{!8q})3jaGkriS)G(uL|rJ$bf zX9RR&-_E3jqG$V<*!B&g-aIYJUpK*rR;<1G7Esc&(uJeCK$unLZFgnCZqq1gZ4$K$ zQk!=q;?c6hWy~(F)$Iy+w;Rh5L-m>0qtWut-0~<81GBjyDz#9HVL64TXx>ZNM1ob6 z5wZ|bun9UEbvhL5eb|+<yW7rWE3n52gYdPX+K-E=(HbjdQ!y?tjY3@S-}SQ<fBMcT zU+{@ktEI^r)0%Qc;)jRzIs#fywTx~Ef*xA_NnY}d(aWCmdS4^b?xBAvU6=&0W4N2Y zJn``ujtBk{rx|!s(y!2F@dKMblJrt7sw%(^&HZ|9Qh?|TSP_h1G1X_usw%<6<3A@L z31`@I2NYz}@PMsInM1egblGh*itZlg{Lf3bc!!(w`8QHZ`AkYg)$ywc7k$Bz5ykgy zz5~?14T?q^B2r&rrhaY57NL5Tl<ahHcN2vdj@iEl_;eV`XP&ReNozJKL1Jc#(<#0P zJNHLFnBm42eWU)IF-jlx9D-<2JIn{(PC_-+mP!ohUu&IaDB9UyxYC$Wrq2@cJKLlo zu135V&#kN;4#eR|0go!1a0jl<=O~5Q=7I0Tyre6Jg0P_H7FQBEXxlkqji<(DW>ZGq zZ<ueK-05O(97i+fk&GjNJ3;Za0n(b^^P9%srXy+^_mmIK*uU2!Tnv*Dz;g+A-5<ne zB(*+dKr#BJZwwYmo=w)$>!6A|RFu|t^krjC_<c|!u8pl_5K19WZ^QpL2DJe3A6l9l zl>VO>1S>Sv%U)>z1Y(er4;-JaKAp6zeC369MQDUVc9MWila@65N|F}q4xdLkZY4^B z0;bc~rFKLW32N1*<aE9W-Sc8>7eOU`S?6A|%KmgrU*26`!Z{8G4h!4@bGWXD^%F^D zJ7B5TaWH4__8DoiDTH7_OSJE`0>PKB6>nMM{9Y?InISMNBm<wp>R#|ij+ly~jiyp- zjq9|$Qull;A%e~03)lml#im!xcQzMGl#)W^gIEQ-nUF)M9mqp_W7VjCCZ1k-Qul5e z(HMk36dzwn$Ma_kV|#67T4A*IqvPF3y6%MAcCM@6b9=Ofy`7#6YoWk8Zv7G7kPkNt zP>h)>_Hq<slPL0$lAd$VdfvcsLq8c0t#o~|ziF8_InYE~4~lJPhPB7uzk>yj0q^HO zU1i}b$(yz4$T3hxI($J!CkwAc(6E%>-8fl}Tmd{;{&H<JdL$<Tu?on|GW)Z7VwF}; z_Jj|Cbd|iSN#csEl^3{s?}y)R^&Q|Vp;rH%!NuW|HSX8=l4(chzw|?k)i0HLjwU^J zesloQxGWpROm&(_2#zsWW-7$)F6K%Z7uc;Yoc!i0jbdvOcypZ8t9q#YSl>C^6Z!1? z1csAaKjn`yM1m@SCTV)!+)<B<<+(9aq61-OoOo<tKalKh%b(2z<)J<8Q&XYPo^7Bx zIW+;aG?l)De&k4YgYCojS{h7+ykOsCkLJ&!!KKD4gYim$4UrZH5+qoEpmiR1C9`%< z($I6RI97;7D~{~!VV1S6dA$roHzz-4=q!TN8vI#h6xHY8-!#(`NHdc&WicU-xaJ59 zK6&>Zt&?5=7rKilLSPb`A^N9Y|3TnV6m#=V&PZzM?Mje<al-r>dV+u!sHHSoAawOC zB60v17ta_|{`y@>&W@84+U>?ZDQQ2dKGI4GB{M;c7UJ9Trc<OdY~T{1eKIrr5Q=YA zC5M?e8CcM-QAtLa0V{ML@}i>l7hlERI91gV)hb)VSI%--e+V*v7{F)j#h!Jt;IDHF zA}p!-=J%x9@{YcL4@+NXvb^$L<w>sWFB{y_{$Ljg)=GOxBds2&{Qi%CcLyU{slZj# z=>d={&h+w9^2!`2Pu+wsK3rwz$JmhCJvTlfQ?4S%(RyREy?qot`zbmdb<j@R-3@Of zX^Ctc_s6_>qv-#Oj!T(9JiOXyEK_-mj_I)INH_naU>3;7pN;WX`>md%fr>`rUn__o z<GRR|ygAzG7bn*!Z=;BL&nR30*%u}#EM$zQuxf3>whI4nK2JW6^m?JrP(;j3XN^f} z*oR@YvTH$}&slra%HKF>@ar2C8v7c}1`Ma2LJ^imdYV3^v7-SBPccPSKCkV%Q)_L7 z$tYL#bJAi18;|_%X8YAdUUC)*)}6`I=k+%0KL&$l&7vn4;!?e|6rP?Ft12r8^S$s~ zKsM%6bhB4%^`Z_h9rrH3o@%5RoS52YI+LtK@+;J0qf8?V6SgnLf|Bt&Q>?mHRp?JF zHx?c*AR3>a#nPw<=8rd|{*X__-n}(0X3XRe20z8qh;T#2vYx$g@*GxG-7%av#GI}+ zOWz-3_}SM78Q+nbtFv7@aTzNbATb9Npg_7hZglxCy0!EG8`cMs@}S3I9fu7o$H<f7 zAz)ZLItByEoGLV4@n8`1k=Bs0%CH&r=gSIrde>E+q?V?IZ4Y58F0CX{IjZ0B(8IAS zI&4q+u~qE_HOQ#2*bsKDKcy|!;nhj-y8Yn&)&-F|YDy0a7fQi)o~JU14UujdJdj|W ztvI)**+jDwwDhRa=yzdAny#PW`pg>7hfvI=pUn`7DX7DsCCpi(`GbYamr!s{9^%q| z4c#?0-|k^fpnZ|f<*VkrgW$2n>s0gp&q<Ak^#;NjaL`(5JxG=D1Y!v|23fEqS8l{G zn_oz~kqFJ2J!U(PmH#CSy!YgOv-^mJiNS!t5mfFk1sA}z<4H?_wT3gM(+yTid6ey= zxyFShSxyd(UI!UPDO^ejn_MEuj3vkXca4fydDTB;GDiyfnRk=wjJfMw?;X@CZE~1R zFs`i^J+Q3&&%JMhpVr90XG`8P#!{5MUvjCuMM&r;85Cx{wi{u*x-Y~{_^FvHjN?F3 zGbV}A`<3&|5!b1_=XVI$4YhKzUok<wm~0QvW$glm4diUyWKY+Zb;oPzws}N=2Yk~j z0S&I+isV(&qPkY(yKy|J2QXzi3aI7bpb@bR3*fsh+0Epvd?%GI7Zz?wmVy_N>A|?4 z^|1TZI37c<^+IdmUcdG$y-IC<T>e?KV8lE}`I{%)Kcr)oGhFKZcc*K5k*{^KJnelb zjIlK_Wy$%dB`@}(;X<&@7{t=pg&+Qtvhu3ogSi%Sm>Ui5+y$*ecjs%Sg@gTmY8q90 zhI4~Tj&gu`p(}FqJ%4DhQ&VgH9OoyrS2lrly!cv2t8c>U<u=5}{x7-g{j+-j?MZ$Z zM05L3w*=-P;Sh~h=Gue{g&tz2-5*~qR`u%wHO<yM@v<kLc7w}Z%B{yv4mv%#_H-UE z#?!UYtIEFkxAo|k<l0ST<|4<2{LAA;gNsTctBbYPZ=@RD@hYeGf-lhZj{CY!VnmP> zKgQ-ds4%LZ$dd7V(?sIh<YY7H&+@&0!2j(j*3+GUIA!&VnkiJzs^!`tlKG)7bQ~TF zjrb)+afZ-mqzv+%)rx9jlxLTc)Rb1!d#RJvzlel{p9RrVjWySv_yu-o(Aj8p5q(PL z6pbsP2@*c}mk9E{3OGjbik;PXU<d0Z3{v&vVS2lm9}IxfiwZRaV{jBQlwX@ZmZ3_e z-y`ufnk23nBZt*TXBJQTT*wVJpS+o)w*SMzXcF~c4u30|yuZ5fX=3immSC-Sa!>Ng zTlHV|iZ{&i<K&Ge=VO~p5J+>NbW?Ywa>&N_Ty*!pzh?r?2%cBhs>H9SZUU1ufnqoI zG&!@P0_%fUWT+Z=eM*mEh~&K{=(j=Hf<x;c@iR#2Y}EEqnc%;P)FLb;(73mFw2OBK zK(%C+yc@mLm(W`q`P(8X#6MoUtz@K7vi&N;3@Cka#rjt|`#&{|{kvKc&~lxN2WtGG zj2Ph5{Q8cNO@Ka=J&IZ;2CDr|J<?;|vkEmd{@<ScB>>B^XX=`KhlQ*q1)weG9=hm7 zl&iZgWZ#zYPkHm7EshV2=sgGdE*flnJK=!8-U{<PnE?j<No*(czwZbe{Qte_|Hkxx zOX`0+>VE;{fAJJ{oM60Y+btD(_`0J~`T^EI1zLjF7@91uzHc@8-PB`fWhsHY@_6MF z-hUGpf?;rfb{ON%<3Aq&I0@>tK`Y^ghw3}6k~x?#v>_7>>r^Z!YKO^rMqBPKcLY^9 zZ@-?dGA}Qo9~v8y3er=c(qJmZ_yXLzWEp=4X@-@jG4e|0L42V%7yG>zI>lc}i6?%= zOW{+0z79X$=3DzY1;M2T+Ft<;f(`va=3fc_DK0UWfNUQA60YLrF}H^?KB=BC5wK+G zuFtC7?RCl#PF2)UOC^7z|7}eP>*<b!p5S{gWjYKZXxECZ7S$@uv#`A_9a9rC6 zZK5$U=Lju#x+~7|dZ=e$F#a58E%+b}X41%bHflMSW&^3WJEvBIIF~Rd(p$RLU-!^> zXx7LWywQNz=IwlkxaYk{{!7t&3IkWXWho$!;0hM3@%wDp_Xr!=xH)}1ebJq!Qt~v~ z&N1~;lhyouK}x-jxZG4LT$soHv<%fsK{hnOU5ChFo7iCQT0wL}HuEc?9|WaSv~D$n z&qbOa3o1j#L(wtA@w?;~({*?paFf9{i0<4fH=vRXXZ>S>=2cOa1lXnU-&W{*Oc+;7 zEVlPi9^xGY44Vrg{ZyC>ebRihqbT#8lZ0*;0PC6a09~%oddE-lxn3d_>w_#uD|BUF z<{NCaVC`r#c4OtY_44Xh+}`5jcQ+TO9GACC_Z~d<x54}w+#x@P(!?iS_W#H+Ju~dr zz!Y~##tAxSz*++LCeZ}dv{pb;r?uEn03arfyHTg)jZ|t;-zPe)ZaPQO67AvhErTG% z>He6?*0C6%+a07I>EZTiFq4<ECBJ!%`Lu|vR#Xq4-0Id*Nrq&OE(Dt{e)kIbldpeB zk-SX=Ut#xM$9k#JYG{BuqaUAHjA1X0R!(k_-9Gv-*Xdf!N}5s#RY^(x@eO>v?VhX) zo1|6HmCNO!Pnxm_Zjy;JTcKvtErcR)X4nahI=X@t#AO&`b)KK*vR3YVRCGdEX{ci5 zc?4%~yILwlqtZqd!}ggsh}qn@2VH9P<(o5-8QFW=qqMo(ofJH$)@LFkTRIqhb0c>@ zy;EwB9aBHu!Xb5pFB@NnHqWV`Mlv*Gkk2LN>UhpC#A$owDhZ7;B*+>M?JZ@PUN1x8 zeaGiptP34ubyhk<K;uC!u7^uL)yB5ZKqR26?rMe<JqaUR0v2ue9U^P=^(fNU8k5DN z*=9!<^iqlH2d>mf5t~!05niS5ws^$UHsh;OZkJ~rkB`bQdvBm9=o7vpS_|7xF#d9N zf&sJ<CA;^}7uY{qMS=ap4~M(k#g7X#E9a@lfa^J6&HN_5=CK6EK(x*<&gVT+1rCG` z6YCuPh9jMNqh(TFE?WcO&u{M*3WaHWbl;v8YF(e46?&Qbd>ck7{nXwWVsrr}E;=5< zXL$p3W}}fP_o~h!czn2Zim#)};a<N=8aG||ILzsf5j$M#<VW`jnNH6H=<ZcGT&vc9 zhr^Dw1F+UEVPd9`SlThQ^X9G)V#%(AnSvAI3j(UmREWdLHAnLAT)#Aa_tvampr8Fx zdz=E)R&#ZqOzY_Z5Yve}3WKFdP2H9llYTMdXg+1F;&$~2-6KfKq`&!lS8(`ijKZJu zi`H>%_ChMpbR-Iq>-HQ=HIzdeuLIwq5Hc68e~xFtAE+2i_0vOXj(eFuwoL%C!7Z8; z6ie*5P5zO5rUH%O*MC9j`2IwlK2TXaf2=0xP0k{|3<>?FTj@uXs$i}nkB%?lY_CVE z7NI}UC#Qud7KpEX9m_4&ojv+I(TehDFZS}TQg$ewWxF3;Fzgf=vG?>&_T!oAvFb$Q zJCgY~qmO?w&jBD_^=wkOs+#!4;RN-vDUoZZGwdseV?+-N?Xtl#=H2}&5Z++iESJ@X z0E|=((0E*JOUF+(a2D04Dh~-1ThhW7Qb71W1s+&rDPVdQ#G+XU*UYUPk0}QE?J9fU z<fhWi{Ghxc3B(QCzYxm%udujbh|{+WD>}RpU~%Kg8_t(U%B;_E_@@$IYlv&0Sl;XB z!+>eOVH~VB)H9JwZ;ITEd-(DMirx3S-fM)<zKOn6<uM7xn=9bjqp(&g^6BqHH0BPp zIg+^Ao^eG^;KZz7T9~lybTDc<{e`^W8Z~Q}jlE34_6Y7urwV({RLF%nmi3tl*re6q zu>r37;z9td^|L64&8(kPb)x<g_~4ew08;x~WVn)^dGcg?5x}JjT6Y=N!YiCbNCEgO z*6rc`RnYk@1%l@ky>763FqfHPlTERG;@_tHO(ZD8nLd?-uS}AYQ77+a%Ano7YbQD& zCD)gb5uQbdJ6Ejvyg|((<B3mMYo&O#Q36)J4VzoDh&rm@X078L{rLHi>-T_l4+%5M zVPXJ5m&=U{n2s?sfTJ?ar+2JUMFI(ddl&Fk|KPgc<9-AnzL(7E(w+}{RRBh$6Q-@o z!V?caCr*=L1wfA|rk~x=Voh}16wvRh6uK`gn9jL%y0x(Ol+AumAlG?aZjhh4Q;`w$ zb!1zL^+?RPmbg};&ro4LJHDnlAr<3b-JAW<Ko-cAD%WZ^fCHTJ42b|4fi0`(7iw*E zdg^AqPf`NEl6lDfagNp_?Eu!|E`v?A1$*BSY^vX10#h9@^jbOYP?=TgM$8x~x$!+h z5CVm&{Tb-m2M-j^AteT1*=Dbz1{mU;-@i6!s2;XwH9va6B)!y8dgB!{0#FSQ9VA!* zubdr<RduAota1VM-qIT<&<?o)?u#AzU|T0CD%CY<@;On857mV4gpv7G6nA~q4)Qa3 zRWY_A>z&`e?S(M{Di(-x#F`~bJj!^9e^*3WfrrzE;(wj|T0!9>VI~j~>p2i1DLstY za+`cxS><fh*P)M`pirh?I9rl9c&^CG?pPP5XDmjKm@?lpq{N)_@s#FauIW`7wkWhX z@>PzRijp<Z*;}zhWgkn-wr-*$FGw<R>Zf0wK@`T;&;BU!$CvNu>y-}T^?c6CPK=bA zqoceCeT)a1QD4fKozbm6U?8v(OE~KC8<pv0hq=Ys?bgi9{y4ZVnbvSJr`qmwh-ztK zes&9rsv9{5#z?ju(C;oj18FU-w<@&l=3#%7bhO%LT|B2>UT?VDVNq-`F$b@1GFw}x z{zC(z#*~I-Ib#-fjF0UAm~1G+7WrBpXti1^UA`~i+SzP{FA1=DOVV=8x95t7)duDH zb#+##Ql&x27zJGTqhD>JNqE8_Tv&`<^EG@*E@wiL9$itPkP-cS?^YYK*`)H_XC|Pc zKD6nlCM76tLfK>xB7EQIQHkEFQ!XrJ>0%7WzR}F{Zt3*tgnXGP+q0OvDlCnZGcro? zxNm`fB6N+r_n~lSx`|yAxS=2vh~OC6UKPQ(uUZ6N`dWnD_;6Z$P0^jEAR*U>C3KYj z<|J&WIvN3k&1y9rVlAc(UCGP}W+j?Yc-MHa?BvIam-$rzl#FNtDFKK006$`e1DiPp z(@Rguv?R64H8pah2Ixeyxf$wMqN!Ee^7U%`31FRP+PgdXGo{c5RsGm-Ets7SpdFxo z@mKdi79`u<&Gt8642F&5N>TaHW4_1(%NKiCOiRv{S^Yj7I;}kvfuFR);gHwrS5ix# z)<wQr2UbPhWSipv^($qbjm2gc(Vi4Lnk0^G&M(@pk0SF$T6B29%C1}m*<HB7jNM&> zSzpJ3Hgvo+h`{6)YDqj7ZTWQ_k~oQ=%l1-6DK3R4R+(c6j`*t!z<u>_m7;D=z1&7` zFLJPR1iw6sx8EMiD{zY9p!%YSVvI?!TwyM2Q`mV+++PC>7yw{?7ndEwuWPR<Run-| zE7AYv!E5xx9b#UNTZ`KB3kIYa4kk!Jgq5wP)GP7_2zA?ntCKF0H>zof7;&&NQISfj zfHXHIv(_UlidVSRz5b8e@e1+JvmAI@A*ura473s28>>rcjVZ6k)=kHm11(4LFRdE6 ziUw~oAIu2~{iXSr0rXcqM)qFY$0K0_b|eha=SfNA>P~#K7xLLj6Cg|!+4%M_n5`%V zm;?Q+h4f}=)uEu9%1I$?1#Q~yCT1EL=F&Dul_gc=-dx-(9hKh0C^ONyVR%trbvb3a zx_N)-8&1zNe<tqcMQVrgjc;#T0DE**-L&(KLa7T^f@e2TAENP~$JNQG_?BEtBkt*< zCV7TbPNAyiAQb@lXr5t}<Pwx3T}L^+0`8_|Oqy4)pDdUIx#FwTY4PT|7`*7?#I+x& zt4JX*Sli)<4W~yZS!vJGku~hi+3{IZ#=<>7T~TW~Z~EOpfAD4PN}&fHF9cI}`8W*j zEtYcBIjJTo^YM}&a5VTsl-I&|RA0^%a;<FrVX4`<7X!Pv&GP@%gz3V_-$}-?29LDf zAt2p430D|S3`GaHEO2nht1rwbs7e&VDoQt_Mktf&D;n9}cF}uIi2!EQqV)iQpsMgV zCsHz?PEn5roUNrgnM6vb6(VH}?yfPP9K0QB{!w@&!D?S^@O>)`OTCQJj}TuIfpV0v zmR|u60&d41+>J3irc7!>!`cr%H(Uv9`-y9BE=%i_g2fO)6UzYHBHT3dt8Jc769_50 z=%G%Zb<*J3)7U3Gz9(8#Iq`}MVv8vR=XN7wyI>Mvl`;a!`uJdJYS%6&Qr6MwNQb(6 zmpktPTPSF6>-egYtpQ|Dnpv{ZdG*O=03@_GUY;zea+}7$#y5DLS}w-MpFGa%aes=m zwOAza8Uz7ioTE3w-<>H2+nFPVcCQ~}91OT1^F=<_2L^>uDi7e?Ao-*1<^>yeQj$mp z=?1!D#R-D;Z@J?%CLdyam_Q)P4rY16qaSjBky-oBPY1O|3pI#*P2N0XSc{Gtgg`=! zb+*MmOnQ4(I1G254m$Si<u+%66*t-0^kfXR3o?niG*RA@^9;B<E)Q^xYl=X2)O!*} zt>O$t^QljhO;)}9YHvZ=)$DX^<@Hbd_GhY?<8&uVj)`4^djDpHyodT?^ucE5THFtf z4#G(69A_NH&jWyh_G7<Ylzv!*>W`Ix{0M@tyQ{StNrYkVEM715#mnnhxV`nTrAcCX zaP}<%pd@=Z*;wf`j-nnia~WyhS=N<3Jx!b3qWK(%V1VuwVoFpq9ndq*(gBD=2|#rv zMT5J6&KH<s*x3redDUTb#~1nY?3CVm+5_O$IPk^33~Pr3JUIT$;MO0hm?J>dB{(@~ zUNyBK0$w_0uJ(K6{050Vb3=dX`PE-?X&a2Y$vQyW^N65j07=0*VeU5;TvA`cm5*oU zU*xReywi~=B{Z+@8h7tKzjv|o!m%TggYTP*SCf_rltmHaFu=Ts(zL=q2uHG_nZsI- zWY_u_Orl7!;3Rq+J>t7$`rd6({fW6cTo%Q(@kny7HLiEUVAQNisxhCfU3Ht!uutQ7 zMdeDL)aGhhSApKajl%x`yW5jea(pn$B$5;lQ6#uG;4K9qSdyee|4n9vdB^^Dd_~f_ zh3R%ETwRY$UVx%>J#wGc(Gm4WCICDg1#ZvmtL#&fvy{yTzZ4fJoAZOu{HTE7P_FNe z>s1<q7e*nSY={rF5%A}wR_(!cJsSU_Y;pbPL8l!X`UPL}HCotQK&C?KleA%FOWje& zU5mYT)9u+pP5RtZuRjT(#o(t9EvG^S34FI)bFKVaIgovb-Hv8)5gmeQx6K25ArR=d zo%Mij$|zrH_Td^z%f}BLeb>AYRglu3_11LV8h}-!R(WVJK-d@!z@p#4ePSd$P?m|A zg}ueCqgN&BH(9YoTIvSbjr%Wk{?=~4V6|HX(wfa<?FO^k;P-w>9O_J;MEQ%y|7)4= zeX8!s(hI$fRlTMvBP>DDvb;%3F~LlBafBQ$-{k455iL~qPHfN!_+`QH);SaIX^Z*C za)djs1jVunp3@U%Cgm(vI5O3*O80pY$lV!Ba_|pc0l&)5D|ys6S?PwA<0UK^FF3&- z>}&H)9cK>l*xayWItq7<;svf(OAIplLvx2wnfZ<3obD0eQ`%*d-#Peg>gh|QRsTbT z{)#{OqnrFE$E#qS{tCj?s(8|4ifzTBMG=iQ9u(uUXBxcrX_0M#72TuIV8y9-dOd+G zYdr%yw(2OpWf>wQtPpry|3bA6k)r#o-5U;K^DrSq6M$H?QAX&ihQF+qlC1SXOD!pj zdA9_)kT*CpNr)QCp!snRPwy8O@rNbnQ~9;-{GwiIwgmT*PG;3_Oqm>Q1>~Tc`W0)b zYTCW3)?i*;tQR|wRCASyuW!PrKL>{A0BPFi%rq^w&-@KmpHBM|YQHHV%n6&IZsm)! z#fE^eN{5tEf%^LNU-APc%xWIKU)t_b2D${35oc5)Z^r<KEmpwcMRDr!fe@I+EL0Ia z`;*wt$sgV-@JT5sMqaJ9OD^Gy9!(vlcFTD#wN$kHN*DO$czeU(m_1jiPV8mDg8xJ9 z_3<gMY+u#^R~*H2P~9Q+oPu67oGnu$oacul9O~Zy_RLO&+erZ}oVQfKcW@Q{c13JX z#%~nnip46V-FPX(;qri$;0dE-Q_Cjv$_rn1m=mr{v=j{W#@`eaI=)!wL*V>0BZ=u1 zj&L-c#)a#hx_8i$KB7;#PTgc@?Kn=(o&dz~ndQH-d^nh9t&KH|<PrUI2Bba7;6~pX z2L}{mVH9ROG^iSDn7HUgSXRma8~;2(8w>A%3@nF$l^b7SNd+}duij%)0dq)jdCHI! zCeMnpJ{^%#F2@>16p2*~Zxu&Lt`vPu>e>%r{}NS1+W%_hG5^#2i%5KYJ8u|WX9<Tu zX+BpMPl?*Emdql?=Bk~i_q*>Voy@gPTImvLxmN^gN#@kIC*@Pn{et^((euE61U0@e zlZd5sjmSrZ%^WNQMoKgG18>z&o7VB)-O$4%0H}+SE2n@x1sodi{KD+<(Zq=GmA{Tw z$NxFo0L0<_vo@H(?{JVw@)7dg2llSLByGZ)0&AEN&fxNY<8;n2Dt%9$>Z1_J7k1gh zx8%NGX#T!xe=d*A{{<rfI_VOO%<sIB@aj>11_lPL{uNID@tdTs3ZxC2{}y-g!zf?| zij0qz9m25d1%{=K{=5)rW3;<<=6IZfe<tD(hSxUQ_eaBMPu_e$RGEtoF}ebT0n`5W zb^rZDWH5H+TY9`lj`24btIRZq&F*=YE*_6(<NqhKb_Rf1l!f}WUyqRrCiUI_)mh<= zY+P>9&ZGHXPxLej2H_ImqC65dhhb~todRLK4?u}05z_uIxdAn1BW!0K%-I<}<_m!0 zoE>baujcs;**I}1GoH_DF#$Sl_i?U=ENr+aUC2YkO1p*D1z89`Yv=n-b`S>nFKfEU zPoclfXj}^39RnxPmE)r9wmp%{O1{O&ynEcF*U|2q5FENlLuWQn+$3opZucqC?aq+U zpX+gk7{GGp*g58ho<78OuON;0IP8ztK1~R|kZ_Wb3)?w%@UJF<3oFe;L3_N9Nt;Kn zeelN?f4<&wpk@FPR``0yRo5T)=n@quZ|!&<oPbscC-w_qO!QipAAK(nVSXCE4*E@` z%D~eAb@kU*ZeDRCQk<UvK6oQ(&8_`KWL5rULvzW^Wqk|>*t2vecpt^d^CTWhHX;ct ztOB&1l_JUK7+&#mCWuARB>-SL1~-<$cw_$YTDID#bvJgXvH8)ZKAPv*mc^C({!6oI z>u^9~P|ccnSeH?}HZ05cQ$5y6zCym3wg1>~mj*N1?dZ1rxzSF85}QR^;x&TLuhEk( z@S6&c$j}}9aB`|euWAtyT9!EIw7s}^yr&w%gVW_&RcC<=uyK{f2HIfr=?$#Ga1>#j zLkU(5Hp_65g&V&rLRK01A0P4ceC6b#axu(c5`kpEzt_HNA%?A*NX;YgY~q*9XMpX% zJ_=f)izbRvGJ8ojSMMPuB3QsEyZ-ByCBo-njt7^mY*aH19X8`$N}#7dF`+IuWmu&G zUl?Omy_l#8A7+ZhIzxOa{2Y^Kk|mu5SIB~}gaS7mWMevuL6e28TrRWQ3$$$7iTY}l z7cYES8ud#9dqTY-Zc@^%A)#)suL*I_oLyNsHD&Hxzmy3w{F9%@lICA*-ZG097>u~v zT3|u-I~F5NwdsolV}P|Oj!nS=({lPef%!i?p=080*p1NTthbh&u06T=pn)hZE|4Mk zBb~=p^+jzy9iVVwLE}9`3pi}^L%`;;n)`CqJg8c0m{F9(Ax8wXI)VPFnd1gP9>a1H zuBJa)eo8jXRia6qE+L($84!9|MmI9{t~>DU1ZYu8+atG8GD56}$N~TsfTlh@VYz9# zD0w3m)JE@HDOXUN<WbP!K9k&Bny90kErxgiZ16Ai*}eZM1`Ma@K-srJp_`Y?$(}c> zZos3p3Qhqtx_Ygj`HxKvKg@8(XA0&xV3E9K)4dg<8>j<jp_eMA8S62xeILwiHY!B9 z#U}U$&`Q{tKD9NIA?IV<Vi%x+wYn+x=l+8_(9XGj*}u(SwL3dLh~YFa?v3Lx$>ZkA z;Qv9HGhk*kzzXop-yGid%Qssogk(}=tu^@8y)OIHmy+vUGvUi^vwn%?!f%UBs)4XE zl+ItUDqsXqDWh<VWJqRMR30w1G+co$O^05%6G6*^lloxBrBa5xvi?H%9?@JwMw%Q+ zPw|vR1gY}N-1sMmiPAV7P}a}!oK4st<ooLtdf@J4iq{dWaxywIp3&94Gg2VA2gp5< z<TJT*nz2%Y&xT%WN9n6jCd_nI($V}h-vID}V!`R2Fjd~y;MEyifraqU&yCJUd*Z*Z zr*r2%5r535um1Vzdky+V`}LEZhQ;5SOFrD+Ll@VfI3W0=dSLaItiQ_0_)hF~mtQ<B zAE<}saxu8rAZyBJa^F5f-@kxylkz?_k>xgLUA!0z_W`9boR9U|aeb5WYI*id9TTP_ za9LlQge4T_cZL-zTQ(#UM5u)VL7yzhQG>Gfh&CNZ0h*5op8dIMqw|bDw9asy;W%5{ zVDfjGNpG^G$7F^>nY!40wGi`Gb;1V^I_YPzi$m#HMZ|z>?E17u*ch>wRy>Y56QHC~ ztk__9jf73hHokN5Wp3PK4SJ*lIAZ;dd8MN#fER>fLEe6|Kq#Tga51i0MoTVO$0A#J z#rCGr?ThmTo&rX*3LVA+75Jf<W_tYkQ_h5+eTCOU*`RDeoo!u9id$q;<5e@lOw&~O z^7#PqXqNiN-jmalQN5{`u}sFAby3TF&l%p$8C0(K(QUB_{Or#{-EVrxJ7An(zrUYS zQ1nXNb>t(T-C$(9kp2x^sh(+fo0v&s)NpFvcr|}7&<0+qYUt6Fv+c)(;OuZ!_^K&J ztS)-pk9d(E&Uf6l3H)xJczcleTj2Qx)=6zYYZr&tV#C()15OU=!mI1r4sC>}j}+Ok zO&&*5`bes7g`LDU8_b{F_Cs`?jt`#_PBiP~Ii;SGWNVU=s|ls^+r8Idp!ibv5!0#Q zo%yS-#nPfm^aAG(t4f;G8-}GOv)AKn*3c&#RDUV>%FT4XgxBHCvkEV84I2R9fVV$b zKDK45gk1h`X%ryC^J=U91y`i>)&k0<5zTV7%ArX7m(|-s-OE1YKBKv4y9H#xCf1*s z$#s0NjiCVxG7QZ>^$)dj?DKcc>l78gDX`}Gb1>cTvG&Y!Ag}`51OBwcXRH*4Q1B1Z zQ^c==;6bD`{_I1s^1R`4XO=QnnR%|}b%G4}D@e<t=~{;!a0zMhgdd}fJH$1_o!14X z`!Xe8p%)X6-JicvQIB`~e5lKBc#lszyLZ#_FgJciwGb_U>vD!>)P<{$q#TwT83RLE zHz;Lh0AZI}7v*jBu8=x5-4)9OX=hIWPT<z1kg~Ajp=)xVe5DaS4T(SnZ&hiC;isbM z`!g?(?rW<FH~;s}zK0Vv@<?t<*Ow8;<3A-%WU`JwOQ?cai_W2Eg9o$|QTELtxJHc7 z0uD<SKzj4NbKs%5R2H9!8ECavcD`2SZXYdQ4K|B`3|BUM3jVA*i}`Lh-y*&H*mEGN zy?UaPmN@p>0r9NKz1YDee?BM3v1vJy8-xiR!H8{ba;=Q)^DRy59X-hKJb<dVUARm` zkFM2kJlbL(7d&DfN;k%YZNqLl6p)pLl5@e}bu**X1D&-f_5R~1@_HSJ23N+M=-rXF z#fz|;U*T=@Khv&{vH^9HbWPF?f#y4}x!dZFi;L%HcWq6E<`09e-|_9`01eg;`3~jv z?_y`*-h!zvpXAD4bL`dtH<b2b=ZF8dCS?@W+3^%7On35KvxD;i-V7SgMz*c^+Wz1M zxh*$&F@q%WF789F^Hn`cbo)R-Ws70?g<@?TFusV*gE7bH?OA&B6!yU3b1|S`Ri7hg zbqnzOW!!DH&~!JeSF*~7BnA&;E^@-rj*n?KOf!n7dV*`&bSB2yQQC3o2p#}dv2eXp z>OO#(x>^$YA_(1z@c;PiahIv5ioT}!wm>)f#l<`Rud9@|(~}D?RpZlm?S?G{Fx_5M z<^6m((G>2g^@;C#d&XMs+3PNF|5mD{-0A!Tcy~WUvfbsrr+S5o@x1D~y=qHlTA2_8 z#<b4o@5Em7e=SNA_!y?+oYYogMexeWO%uSMJU`@>kl8F0sESU8X!Qr?UGfuf6@}|I z1tPcv<6x$^hG*iJcpH(CrIRCeU_*zw{j;!*3V#i>l=K3;d2_7YpIlN>9n4wqALgeY zT9N>t=~B(6oETbB+UXyibQT5P{69Wz>j}nJ{JsXns62<OoG$C}*rS>u;vd!%y>=T_ zo750K#xU079k5!f#RYtbLV5@xeI0cvo`f%}SrHDR=M-q|c<28n9Id3yNl*e#_zsgM zv`IrQj74LzG{ykIbK<Bo<$}-%i4CS2v-hnieb5;eZ`#DypN}t`g;O%y!3fU#kknG) zm|{xH$LX2>eC3rjK!Pn;==p=cW_*59^I4tRMFGhOKEq2~O<j<%k4Q%4{2{SyLRWRK zzLLa0eqaq{BkuLMcCGi)z3obO^s^zn@$51nZ8!$Rn!RB(pUj>T_|i=<!Xes-16^=J za09qtJbpvlO-c=-8K*=)LC84RKKtQ`Jy(sJ6jMUh-p064(bhqWdOp*9S@{!Ag7<8` z$@ky1hv`&J)0LDHL3o};m3$G3R!!U5et!lo2Ah>Be&H&f_;cvgnZ9M>cC*7kK3LG7 zX}xB-*B19!yp}^c6tcciE|u*}U{`irzX`I91K98x@{|1ej1HP=NdaJtvq`tN-GPR6 z+@-koL!yMxUeHg5JhvSeHC_Y8ic@gOQdsGy5w68}p?FhJr^}p1E5VzB7O4h#E*9fK zD%YXLivuCH8c7BZ>vjPmp(pRdwE?!rJz0&MEdvSwPt~G>cJ2p2+pDz<lkz3IL-bM$ zIz}B8@Cn`oYHf_!tKLRxQP*n@nMb4pYKMfDE)BL_?O#?|Q>R*STUHPA&G~JAV|=l! zcC!Mo=d+6;G%JSL)E3eSaefM)*D#6*GDQV4tR6yy+2lO4XwNQ$WFQ*nSb~qNppgJ7 zvqebRHJ$QHO8}F>bG}yN_@(FIvYzcfg!n0PDu5X>d{DN68gS;#;k-}3HxbNBL6Rl# zLoSTVR2Z?RW&h!$1lWw$Np3sV)(<A4$<kM~RIgSQOl6Y~GsWHJ>RGeG5qqQRI({&$ zq5Vc_f*wCX_eD9q;qkNnpO{a3Z#eGW3z<f0-4`nb6+S;XG>J-dQc+{80(5W8RvT7M z-#eZs<bwy=lPdt0xaGW2?KQyEFK$)eccghnu#ASAayAMYeO8?>Y&9PUNrXX>L4Eh^ ziEqk}qd|E7iF<w6Z!_#lna-`?J@Q>JgN8%2WFt^0uTel0!R6iZ$Sa?su5qhgo3o(k zIeWXZ1{hIv2MW#x5JQaM7=O>GxJU_=oo8I<8;NgNqE?noe52c|U&Yf#t-RgVBWb(+ zd@HD0CM}1@kI6t`OaW^$orAQp>hRF_%14$0o518!b|3zF0rXtNywQ-^p<GmIfj(#N zoG_;Bh}*pV6^fy3;-@|_GKVm5r5U0^NLn{(HanJL@;l6t1E8)WNJ&{YgN=({dh7^X zl_}^jd8XdG+GE&(t$l{a3Wdcj#X~r(<|fLGb53`Px)xrf-yqFMvMsg%WOvi7$y1sl zF!<uj$^3B-u!h+UK44hZ^8pA9I$i+)qcV^;$9Bixpq#xLz|VvZ3%yvn0ZlN7jg6n= zhIm+tO@e#J`IC6UHU-9t^g1VQ%M8dtXzgp*LV|7ckl8w(2D6h34CdX-@^0!|Q*TG_ zm;YI>#Qe>ZHhf$Y>zf$&FIM<M$Rl>W4s^8;8AOZD{6okl6U@qT03a2g8HFqPY_E4Q z(uTp{DfXc+P;ay7mcM>%d%%1ch1sH`NhK5kQLiz@u(W11jUtrsm2>!|L;9N4^1j}X z3#0S<7W<o;`*G41)miu-n=hfe7z~?Z07@BGxbB9IvH2@VTxKKXC;W=vbnhtt2VvE) zTz!dT2Zj2@)7vs%Q8oL)t9isP0NcqSxbS6y?zwURMpbG`6Z0vW2MqV@PwDbpAW0P@ zj{e2twX$72@tkom`6}ahP3G=ft%91y0NJz($`_bEu2EbYu~OhwCpTm|#cA*0q#HHP zvhNe>zVD2_Y?o0Ofc&3RT~7s5aHu;ZaP=wKbGI1nZEP=FeszlQFHFS>(k%zpn2Bsy zZo~whetLZW8%8Vu0R>O9uYF$JT@9vhgN~g`>!dAq;6Tl5whM}rp^m{A<5}h(5meOJ zWhG3-)GBjyKR;ujc%`4>+V09F4B0zNJ~kfc=&BCJ{*4L_PB2KYtbl`8XimqrU(VoM z>nGjT)Rb~K{^Mpc@u~2sLG?{_{@A#i>IDVZm$2wp4*Dh)0Ajfr{xmNn5JC*!pV8c% za9F=Fsr^>1wQ@W>FYw!)3&<<NeUe%bz6;*WU}HxEXqt(S(NKc5;mkLyEKP1S1ITP9 zUmIh^BF*U#Pcm#7;k?~E<U-4KWHyR7lJ!VcoLnidKS-&eb#LwpePh;_Y9EaYrQ$^& z!#MGLSP-~t5-Z|St#7xiYQ~29?8_a=aeXik;YE3ic_V!Tz4qydv%j#~|9I|J0;zz~ z`Q*WhVtTWRM@J?^R9LVx$jEL=1m076msSOXX7e_;uV7olNGo7zHTcZ`&%PeUhUe`* zh)I=1{%9~wK^n=}iGldEjlYp^BlXDy(CVB}8N744=Lh&lgKFrR5oN=kn=b**)5%1A z8GuhH*-Vz_Vv!Q>TymdBJm1|ID0kVk+fRhJ*A=zt_ahqHu%z%*B|34jJGR50uf-hn zhFQl=5HlsYHxk#rhbZ~oHQo_eeX?2g>(BfcOm`6)ssqpUIW`>enM!TrP@`_e8Q<JS zYr862t3G@JgC0#Mtq4Cm*ggsraTYfJjHLj;&#Tu<1i%J}vaI$_A)*49S%*(R`u)j4 z3XoQKQ2!hH{g{xSaJ3_V0Q4u*cU{t#%51hRfh@P5;16V1zntbW5gT29+n29#GFUsG znOZ^TS{?i@GBnPOi-L6#>{F?-9r0uakXGuph<4Rr7<;5vKSLo4ULEWmbNdQNcoSsH zyv=9q^PJqOCo#g(N~#wp@f3_T_nS@5chaJ?=0)%Bw8P)l;hcu+uDg@7Uky$u-GHs4 zKYH+SclO%RC5fLy{y@nx=LeD@?i+47woDF6wY@t6M$t7j{y$40Ev4Q1EET@VS5`?i z0;C9i2KFhAeetJ><h2?X@Cs4DjRo%PVD59}%j0!xgl!WlpF9{UTJWKQ?D4E4fla?j zNp^KEF#YW)ouDF{hREcf1{b=lC1mGct>^G;QRS#XR@Eqa6^8VAM?_4wmk`|-%Xvyy z`kT$IqrE)ybwHO~HhrS4qNg5JkXX2frzETe+1MC0xca2hZjbNw1Y@Qe?+UY2*TbY1 znf<6^S0pySW1R0<9=XzfDL@G-xVNn2)Ow%xwv$+tjVhRda&tICi%q5rMMU#09Ht~c z&*{^n{&0;nGkiRkUilxt%bm+`$l=+0zCA3#=@IPfVip~u8-G|9xFw42uhZHNyMNFx zdda|L!4w>2Tn7pD36csYKs`d^i+4R~Vh44_>#h$7`~t2F=2i@LGX+E58FU%HY$8$| zg`De)qEmxc{Oh~NA%$*vEsJ4q(b=tGXe{UM`(~_e{V?4Z45F&U`m4`(tt@0O`F!~6 zxK|EzR8xN>IW94@p&s#+zhc4OKXb|OlsSmWUoFqSDPm}Y-sX9OmP(KcMPy@Zz4-Wx z6W2aa@B`!C{;Kl?)<MN|hxJnz(}3`Vg$3O$j~~t}C$aTJZ09-YZ<z1vy`@{MCo`bb z#rPA>E2tveb<}TX2*>W0_V#y(;f(zP&Db>6<6h*`NXn8i;K9rwrtWKyGVbYpLEnt6 zKbQ39z&Ck|ZUJi&86zMVVZwgJ-RIJ9)b~jAbU$wJ#9}b$!paHRAF^OtjRm(tM2OWn z{(NeZp8FioeH9Ua%KYD#nHdbcRCe26C!6#)(ld@KeyMzyQ6dc9t1l$RGhTYG!r;b+ zM^xf3F3SkrbzCN(8ffS`ao~Rg$83$qHdh~fHod#FRbUClSnmzC3glyUauWBORq3}Y z<?WO|SqCC(C@@OUppzK~M+_iKJ!$GH8Uhzg;O@Gc$<pq}c^HHMjunF!U%Z;}{hrS; zEF1HD*$WP?3#0AgY=a^SIrW1&+OOt|KDymX#|caG`y+en?g8(pnl;1KEW8K>x2mzS zmvd11zmlP43~CCI#G7s*-6Y0A5`#G~27g)Qc$|eP*esmiimM$oVL_!^7ia~7xZy6f z?vLQqS{$13z!b+xhWe5`8X<LsJ^b8((rT}hD+TNh+MP~;UhinTlo(v4eeUnU)B#Yv zgV|dQD!DgFdMdSO{rJfGM&n;5q-iOWPO?KfsHOJZpMhJ-uzjigqVKYZ*6x&8J37eq zClt+xM=Pkryapme&#%OqLk#@qX%v3;dow0(0j)>2EHJ=6-g}P(BE{jKuWL4MlU6He zbsY$%__gy^uNhYZ<>~zmS7E+Z092HQj1?;k#fUP6|45N@UXXEGFqL0~Rb>i^lM2r` zb&_%HWS9Se86l5NK+CUgA^!bFi8jtEXv&v`CZNLn-5Y@9=cKF2s+VV|*l1q0TVxQF zwGbPGVLC#hoMlt#m>AB0H`m`#IFHsw-n$zaW`Hx>$DG7dsVOKmZ~Ys+Xvz%7D(b_6 zJ5Y-Q;JV9W3f3l*kPeWW?$|)+<Us%N=ko@>gV*LW4o+cXxOFbJK<TjI_BK#o^E86h zJa7ZUmvEl}{BW<aLMLrn{CP}h0}QrbXI{LuF`S#a*t?XWWtcRsgX@UkUw&voskPEn z+20db%j5$q68MG8q?4w0wH{*$WlTLZd)AvKy!V%=D6LE)2}&MHVS?n5rW9^BQ|vER zJ=Hq1>H|<q5iU5bI-3>Amvxv?yT8RwB`~%Hcda&bw6-2}_<>TpC9sDY6BpKTf)}57 zq?r4_M1#5mDph9_#BETx^=`j?`>sXhw(&ybPS#HLcB|;iC|1D@Wr(IWx4nopepH24 z&tO8hT7`yzjY`y2H4(&I@ZoyamuY9Yz-Wa=9H^+PF)_YkVgNqQrRvNS8KJoA%kwxW z?EEjIHh-&zJ#i-RP3?1B)2{?bY7XT5wxW&h%Uno+KE2Hn1^=c{x90PnDi6@Yy89ZU z#iqw&XF4P^-2y7~BxvtQ`-s`59_JyhfzXydxEp6W$k$@P{<P?6Fk^5~0)-sa0Cobh z6AIZYUy2;OXP<;_W>T~1Cnp$HpCcaX9ChF+pBeW^6v^(+BmuPRJ8y&|T9quSQQJ$! zc`o3H4b?<al|0{9<|`1Kfyc#DVTpVF)4*K~jgQp1;l+IQhGP2J2*0X6Gje`93q#(M z(_mRVz;DDpD1t$kBiK>vFhJ$h?=El+<rMsTI$&xW`hY<u=>9s477b_jS=fQ|a&}i$ zaFy&{SqP8$Z_95hXOYa7bq0czHY<?0N?j};6axKGy@fJ9v+Yya&Ex-CamYIh`cA@P zLYbcRb?D~kZJ4{5#6!#}5FE9%t7#z!Bq&?<-jzk-0i1VmOmddaSyF2(eNT~;JY`=e zyX1+bY{zA?f1D1_?v7uE)jQ!>cnV%7bS!Pfcn)dSDx8Ge^<W2^PVwOEH?AM-Os`5- zN#$O7P66i7Bt3waPjBZ}Cn8)Y2bmJhk9y)EH|#N5tn45>;y@UmC6f!!;9$}skE?e$ z*-E?dZgx3`$T;@BKY&SwVRC9e+I2jI-M#c)R_C|l0)UJ#2TT2C_MPP_8X+yyNYSJF z$ZxSCG)$WlJw1(Cp1s?vC17(P(t0zX!&n`<y3A&OJ~w0Dv!89cLK&gNw@Cx|F3}%s zKY7|E1dtCPUQ>Q_#9Yh<dd7@VX#sCf4kM4MHu?I@+jkjF0nP%BmsUMvzY^+qKS-?x z;~@j@GZp+VeCDn(hf4uZmS5p}S?+y7lKZ9=&qeCA^A6I*({nFOOsoCHc|VI5v$irI z3=$;t>X-n8R0S`LnIL4mPx<n_V)4X}bUgDfFOV=?0Z*NA+qp6Y181ZO<iZVE6{>zV zDef*&{ro=1ABy8M2}{Oen7i(E1|5mVtLtYC=7{WN(ALNRoB1lb-R647d2^=Lq}v!= zPyy*xiYlmYKIOg3t0W$X65$BEQCji$uO_m<xum&TBH-q+PSCz6zqW45ImDV^Oxv7L zNJ^Ei#TGG}B(V9wQC)-onWzLz#9_J2r&O$l5cHV<%CiCHKP{0LK3YyN;k5Jqs&#w! z$vigrVJ>O_jK0{VviODtg$})X0*}+}wL)&R;n0q=&e5rx3Fo-YW0-57nw!_KL0=iZ z&?zaKz~^qGFL9sGL4SKU=kf3*2=Zs2EcE!^orRMRoo`WUe!eICt8%fKTN`+c`ql3# zaBui7xL0-0Q9<Ys5O55?TRWbgPjj&!^6$cB>|}L1(Q0ifmL7=u;Xkv*6^VWOxxx87 zE)o)4@u9ATitH7fp=$LEumH(=U<w{1_HMNcli)2(5-cw4#d+5~w~+Gg&bzvUaGDs_ zkyTd`%;#<sETyH6PV<xS7Ay<*du3XEqS(RjfMPOQ;ukUuDyCS1QC#ZYgk_++0j@1% zOG4MqW$zxgIIf&~ST-<S52Y#vh9OPiJWeVYWCf*K^*{U&=sqO<dKq=fr>QiY3*5UQ z*}yDI|0HZD6n@UK9p@}<$M1u`&L1r#;z6UkEq1a$4@zm}^*(n#L#HF1<)Odo4$C60 zq*C~Q<`23892qQO*2%J8m5^8J0qN+v@qW&{h*f)x1w6QC+pn&laDC4!8Y@K4YsVys zhS^Ije&2>)lye`#p_AG_7F_zs_ej+Ujk1-j)G@$^VghD?G{K2T<X-%9=Ttx*=y7;X z_Hv>Leg}ovw@$q_rzyA@78V4(n(&D!7kOw^#0bU49oHq1J6BK-tW^}N&9;4P{5cTX z7>nQvcV@ZnV+_p`)xa0HIuXx(9o^3^h*E9DBtxN#k(-+sJK<WuanV(BwQdR{0z5jK zlbSvQlDlB$Yozxih-RuEk?HYAyXO%kz5Lz|KNFk4tQMsZD=INCQJc_{8>y<)#lv;1 zKwreN!GpR_;?$6g!$C``ZKML=%Y~?em27aq3$1PB*Xy6jeT_&44I&3yNCz#C*pcdh z<Q|HPyAC4|ytjc7G3HKojy3^CM>u^q3e_<CGeIyZdp~88XE>NTlxVI_KZ`M0P?_J> z<+$b}vhf1;$-yxosdn?EnuWZ%<Li&k^!iG9(qdyX9cj^o)kFrl4s=)S(xY1so>=SO zfqgdW_Y><NdyQM^XC*LRg?NC=V=^@dOQtSvUHN$dGU4~oBVU`^Yw<2&saV19=o#dz zq`@mMqR6IR5hyfMu}IhsnCFUD6K?}8Q0k}|8>$9P-|WV*qG6_|s&Ti+9h`d!>3}rJ zs{cq#7uy?t?@0898ZrZ=44AR$m_#c1132B`bt<z%j;<SIvCM>2yO;UTH*aT`=kSDL zy%~T+TT@Q&(OU+N3s~T70xe`wk^jf?_)F<Zfc4M+<(L2`JYk~GBWnKF>-Ik{)PylF zQc&z+6HAc<BUh|XpXpn?gf@D2Wy2o*?@xIskX)*A%#%F2xSIlBqYe|c!88&7eF^NB zCBP7oeNr<?>XE1dyGMm}s082OyY4zX<Nq)IQ-ZCjf~|F3*hjwyzD9SzH{kPEMPMGb z|MxQqysUn`{gK*%2s0D+b#ZDUApXCe5ezMWEsr8tA*lSB3*qsjVPAt84mf!T@&8)P z|L28uF!~Ay{^P|J?^$6>3R=0-H3cIHg^B++&qW^>c)D`)2A)R`@?qd>*ixE4Yhz9l zaKC)-$!Du5|3ikMSK;S!5x{2V#&WiKXinpEl#a=jW_mOF!=AT4>YstZgaD8A*GjV! zX8Wwa(HKPJ2qf5dkea@GtwpJZa9J3lD>}YKzj|IwT;X}d-{#gE%0bf+tEPn;cHH<^ z<){3-`%@OacJ+1(;I8ri*n7*UDw{8GR1ri#N(BTY1Ze~$1tbIsNh!%ghqQE;2%>~^ z3P?+L9$KYCx<Tniy5Y`q@D+b+-S~9B{nv826y}-Pvuk$E9@p~$^pQfHK#fY<r@g7& zwE49Pnjp&mbeexJT8ZfeHd6v0Rou&mk&msw>()(cv)}@n5GGZ%Q?>nEk!#J)ZSI3U ztV+-hLcd)>5S`u^TrHrmQmXO#1a(u1t5cx|B^1Gp%y`P@niw3Pikm;StZ#6&wSGmN zJv()AK9Q|EshXub(fCrN#eDpXN$cRDB0uY`2T~7|2JRi0I)l!M0l@q`JKDxx;yq;p z6)j$M_7SK5JxOyHG_zxD3MqaH(fT)3&BmXIbiU}x^=1Sl_98&nxHhgtk91z_t7#zO z_zP?HpuvsRMjRWuC<!au^|X?3bfOwxp-}(XwWfY+OAZvvh>;rezIr2U$6zEITI#>B zTn*aIsk<?#XB39#P7EiReRdnJvWVy)<NdL`H#|2bk=%S+J&`Y_xz2JT{H)VNMT1qp zi=r-(H)VwulnL}!?~8*n0Tv&;9AUpNpt)pj^Y%I_nZ$Q&brsjw1*Uy!F&x$9U2LoC zCvNH<dB7+L&ry^li;Ln9%u;ieN^ol4jlSj~L-X-ou+AEh1eBZORa!Z+Q6Ay?!`^;! z5G&l_B1V*4)q$9O2XlJEe*AB!Y0lBxj=SS9)!9!_9ZYLwSaZpDX%*01ucr3MEGP$b z(!lQbV|qjtovY3&i{T2I+Jr3lN0n$3TamBbj!6?q41r?Kd<pMs#*DqbG{1EU1?_OM zT|eI&2lo!<KN3H1DRZT4l}``#*qFrgdtwg)jZUBVit<;-57Me(m673D8tU+6Se>&- zI_4^MGxmB!)tWQNf3%~b6J;j~SWYq8gsWU<G9=-}F#I_Ta_SFRIh9u^H`tgLS0F_g z=3G8B6r@=scZ){aa9|9SH-{>rwMh4?*~k>AxITF<zhG8(Jf6RVdUVVu^WFu`94ULs zE}Y~qcR(?{>pI1VBBtNB0iI6uDO@KoLAN~>#s=$5jR|V`lbSXtcl`_9P}#kXoKGN! z9Nevp0%uVB9;Q|cxbx-*YU+`>R5+f_di@OMHpR-<WyG|A$##Jrj59CIB0)&H{KZFb z!_l8#+8HZk;ln9S>gn$_o<m0!KYZ~~?0B;X2Rl#xQ&1vh_&drP_Qd^F;;k2<P_c$e zq_SVU{pRZP=b4Pd32iY$N!aN!{Z?oFR%=~RJg2uq=}fzB_%k$}?to<1^LS1^ecK^J zwHqZEh6x=rAvJIVsq%TX7t7o_wch^AbK6D+bK`VbiQK86d8gssq{&APM&Xh(Gw;L1 zoekoz6ZAmuN<M2dbHZ5OqP3a+;Z7xgmNzJl1<PP?7={{WP4$<)PU$#MEEBKXwg>kO zwfXjS7Tia3ZLOz*rWvEn&yO%-c&q9ldtE{~dx514y0lQXo1JPf)guqGV{Yr9U?BwD zw=uqoE2R6Zcc8t_G)ml6`6q>w)ca8E@Uho>Cq7iF9~7%3UO#r*3H~M<{i0~TEnh@# zp^Tz9LE8$gy&d08thG<XpI5mPaUTYp1UL+k+KcGxdJ6R8H*1L350O~D?t}8(ueR$# zA|>RuD@}GwDyM6&K@Y`eZ+uh@sGg7q^?@+W6Z{Txaw+0rB~T>w!&a&8tHIW3ESV3n zFvezi)c_qO2uNu{caa9JK1x137AYNOas8zRRoJbKU!y{<&3IVcXp_Xo<XEa^7v^*O zotj<!hrIY(W1!F=WkxIJcEQ0it4TW4Ahh+C3ZpbxzqEx0++?o74O>?xL|H<`DS?!n zdZd&gD*UtRb1l}-BDpW##L#S4=o|`x)L%|nt#Aj^1G5AinxF-%_B^dq1^+BR;wTm? z-I3FY>o9Ch?KR~zIpP?cUKw@vnA_3WFHf5nOW!=H2yzD{4frO~gw+I~WFd2i|64<2 zyE}i*49d*AFS~s&<kQ;8kL}%=^N@?-ZAYu?@w{1QsY%mL(j-4b+$Z;Pckbrr;MfQh z34K#r)6D;L3r9fyjdfo^tb>ciX20C%K&7aEmvcut`QWwbjr&2P0FlyV?hg266z~LX z?i-H?S295E9Ls$3zm7WJUP44!>983GCsiTz9MJHS<FY~*`sfiRTp()49FO-zS%{~y z1MiL32!meA$_XxO5AmYUlXK)Fr3)ULf>OVB^0mD^aKSefg-hi|gM7;a6>mEwA)@hB z4b_mq)$wOgBS!LcD%prp4gBQ^kkesr8!)9eX4MasrY{={sS!}yY^2Y0e399o=RK9H z%7p4ZPTft5o8z>BTgBw}G?(E?(X`j&&F~M!_VdHIlCH#?OFZpGw2CvNh84FuR->xo znq`|XBOjV7OMuIMZL;)>e2g3Q6Ue6?87A?sVS54EzI)^pLaFi};>xszZa#YLJx9nH z==rMBy6ev2%rtlAWQljRQlam9x>csn;&p}sxWn0?*U^1j?(S?ijRaEo?G6nXP?B{> z?QAzJ3Mf_|yJi_Up)!c7u)X{;rSU$|qYXQ5$h4;vB!f*bMB%&z*;OG%$ZYp0`PKMK z`3nsT9$i&w@VRrdc!7cOoyQsI&ZXj)%<9yM^^jETyR%S7oxRdz|4YwnM)^N>xBq(- zMVA+{J={=UpFos>9w``pyu#G9`tePS)4@BXK|hg|9L=vJuQ!4=>^cgOy75Z4U?hU6 zZ|0Owv1;Zb<PCBu3K|IqDl^shxhtlh-d)PcX#kg>N2|IH>p^q0gI5JoCfT~G_e<sC zZbtJrO6N&`A1eK=!#ITV+TC_&UcO}Zxc9SymK&xf`CvS_VGsYtmo$z|8Ad@LO&B$B z@?kS6(cYv1!y*mmTF9q|4H~;WOwpJLaEYH5AKvmb*}B}(_UNC+41J(#9e=)ZmZ)zV zWw_NFmzJgzb(N>vFY45YNnv`Fvi*sl;n+}qq$Ap`WfEz-h8wkx6wwXIt8c99IEYEB z<!Y<Km0#-b9T-4uomJaQOP#Os<8<L{U}u7ooHr|t5~3^&a%EYdtMqqzle>b93|jdy z5g$vvIr<%rbQ^fyXYC?KDcy)ws2epX{eJDgQ@0bcJLg9hhJmrC%K+ioZa%(g?c*E5 zs$r<TUe*oC<RJ;=0_58n>+sVG!(URRBlWhZ<wAFS2fCikp&x*3VJaxnNy^h=H?AmK zF2vAGLeIQO89g%No%E?SvBEj~n?J!+RX?&~B1jpU%b}QNX=0LA$_!deNLLcpF0oQa zC_P2Sf^fO_swk$&k8M54?-Zq7paPY`op5@jQW9LlntYB&j9R}oFrtKSk@<>%B7UT& z{~LzzliYn1_TZe#ZS|M1_0e7GsoM=VlUIOwV0Xfy;$+aREG|KhQ9&S4t;bO+Y#zj7 z1g<tZ4OTR3KhMlT52-g+aWPQMkWVyDTvBmA^7bbH(T03@cX~j$bT+;uszFaAXvsbp zA(<W+639TH+GwGeOLKeQ@}zBpaY!Xw?(bD@aC@|Y5nqPq9e6hpX9d#pyOKX~uXx~f zXuoGq)1q1X;G57my6Q6|Ntd1cf#<H9W1kVS&z^7w?SA`0?le?D37Rj?|0FezS40gv zDk1oWMErY`^_`>M9_wSy#^_?IAh;`~CJ3(C-Hj5yM|uC}G028900S4)_{tp9vfav0 zmo}2?R$S_$>Qjlv11-(h&P>{7PJ1hhd*O0;5AWFlx6r-q*DwgQ+f}r4@hX6x!r!hF zkJY3ZO!!>?IeGA5`Wjbf3$-jVTVInKo+lYGuV8&6Rw0=k2|07EoLZtNi@VBaz6j7* z3p8%kNS><G+<PCycfDNwwt81m{kg#1hMVv%A$|7NHZtC^QO2takseQ;g2wZfXofBR zA9j&f1YGxh-HMvBF*$$m|M#jfGW1SP!{h9Gh#$O#0%OCjx}B;3aV%F~8@&B;wOp_| z?&pCys%$v8k#}eQp4(>Y%|mc6_}#=!tA-kn^AK=H(AWz7r9fBf4NXwPN1DzNMn?b; zWv-`(j2!Y#a>4?S`6k-5Hg+km0sY){!k9atjb%j|g}cU_Z?3Ri(um8);kX0UB=j#g z9U7zIuFBvF{P(VCu?YCQ7g<@B#~34*m`b@%sboerKlcI%N5z;1u6*Gmjptrio~XDY zyqF7`5@b{Gbl+;CxF4OA+>)1(WG>K8Y#yL~W?o^nJdWi{^Pdm;|IiZRM3tquSnS8H zo&N$ISG%!u)HhHPv6dlB;sMJ3Ce3j+8*pek5`Lni5tRMucdC}rm93(58aMFc5&WoA z!(<N510}~bxfpBb0}082$Gk_o9&xIs+>DFY5s-EZX-)a0z(;t%u<tD}pP#qPig_)N zpr4$3^XhcKYIz$vD%ZUwo{&rxmj<D7IWKYjT8~_ItvM~`Z7@5`L9<?Nd5R~QTQ{d^ zoX6{O;&;<T3EbPg!xZ)<+a#qdDTfRf8yiV~0C-euZKYBiNs}oOEXB(ALUzC1HMD>8 z-+j%ixWG9^fqX!mV<;TzX0vYn4J1*XN<H3d*Q7fdbisPsBmFgDJru3H73H?CgeYD| zwP<@6{k*aU%;!BCO8FDyF>HngHPfb_>r8s)_xEmVa3((7i)3Hp?$0s=ww>C+m!9O} zYPr1Fh>d0PB2XRojs!=GQBLARoz{qnF6i-!)Q}FEG0G=P;l+E*<Y!Z0i-e|&>l79+ z-IyUyeiQSa#&?1c9&n~&S>$P`y~d%1<_ra9;ZapZ{A?3eGy#?E4)`6U<8{opoy)_T zr}?QddG^B|v??rStr61(0P|k`YBJ-uq>7JxhUOI7N4SHa$+OvZtU=MMX%?p%DFZ4E z`!Yt%o-P`bs1EZLW$x8^G>oTaSf0E6UseR27&V_vFdnfNDd?Hw4Y{3Yka%~(JSZ5V zor)92cN@&gLcraKMi}daoG$-`i*E+K$W_+)xEbN%Q-BZzI=k1GI~`I52dt(1@>hSw z`BJQAYG2n$UU-MMW8vnM=md51|8Qcn5M-uOE7;6#I^s|Qnk6tQp6&&-6S#*3`$nYI zZ!=5AdCjb4l>GLNXZ=Vvo;@RylvguW7sl3%wW7TPt^uGcM}NrCHq^~b(kt*39?$YO zQ5Nyz5U2JH$Mp;5`nCP=7HDC3&p90E^rS72B5Ug<)GMVLYB!1M(WjGd1Gi#1*ly6r zi^^adxEXAOBe)Qxva1ZU1&?@@1W#|q6fZjJpxvVC<&D<eZ*|C`cj);@TIo*;-X3V$ zeKn`OYp~%D*8?>gZ}Wc9s|ZYo8ni~p!q1!D6DtRLywjvylx@cCdjRgAfEcs%jmVGg zI;4s2hjwc+Co}W;rBpKvmQf*9Yr>5szWzDd^GnXw%}9r!e>?5#*0(<~Lqad#M8r4L zxQ5_aVo0k87O(4efH+@3#jomX$+(YWv*Yn_XIi;q1m|*51NtbUx1Ej`z^e=x_Bn4| zlAq(yXK}3jw5oUTQ}bJ5x85AW;(eK1+rH4<7I2@h^os6sv!ixpB(<<;*xcU4s`HZZ zO}66W&wPsljPop_H|(f&_Qh9YjWtAYwMiSJY<{~c@aOFRMrL!^eziP)*r`0)iw+l# zef#-@%zlx?LeNn!ZGPMmbR1rQypI@@IS@JNg=%AZFof>d^PI}O_e;n2+jyB}O+u$s zT1I8kD35(Fn>MNi^oLTn^K~fjSiq0zFZZsn=KCJ>8PwoJ@~PJrj{1I|V9T$)PsYAM z`#$V}&6hI}U|>~Nol2^HPJ15v0roOYE{^3pYdBY&Vg<(0l-|jc)==KWOtqICv0uDT z97s77*18T$y7u+?RPibvAH1xLbkE}Cc-8L~JgNOv1lJIXWg_BLl!6$=DqM5-5bqTJ zXbdp@iR<Ne!%(MCI0r;(rr#-j%eE=t^j?g3tZEJJ{#fPb`3x5UbHtcKJ4Z5P?blb- ztz%0gMb9G!XXUmXiC}-=04bY46VbULj1yhJmTtAcw!04Mv4ICqpHT5W)--vG1TNg@ zytE!)XXl!SjCNbZV3%~rhx<)?+BQ@8mwFYW-PjVrgpQE086LieelV6_x8vw1ZvV=| zRzXe>u+sqP%P*XfEx36v=8eg!=?Dg$(X$hD+On{ci8WxiY3%MGf2|5y_)q!&2`0`B zLqDkJxT*%ic(+!pQ`A_tX*_X~gos{-zG1GV4}@dyW#l=HM}d${TiZxj>c-CZt+yYR zn(oBbKQ2Y9Zxuc^=qR#e%r5?s7JVDv%il=$`4sg)nr*4Ge5Ouz!H2DT2z&w`lOA$s z25fB#AO^mOCH_DL0m!dc3cY_BwRPsKM5~7KX(f5a*{v2h{(P6qs@qfLx#A*aceijG zz2+dcUpX&X3YXFolNsuXb&X$>2{ea#uR>M*;lJB_&?BPcpA@kW0@DaxRJ$iwIA>dc zVfdQv#9-h4IS2^b)7>m_;sY;(fUZzUh*xRs_?nL3AYwH4Ax4bVpc7Zm{JMhh@aY|5 z@>GR5Zt#c=L$q&G;eUb5R7mpJg<gV92qImDUNx(iR^73E3^H<yNlj}16QhK#Nfzov zawA?6d=0F!ju5e&(`OBggED-m|6RjI$TRf1>#r-*#*i6&LkM8%&1-o|)(7DBXi}5u zADRI|0h{>$e$JBLEqo)c>t3#kv<cq=Z<{`P{S%t^?;o5G^~}noEoIx-4`$O2w)ExC zjJWTR8zNX82f7IOHjF6belDMz*2{Z!%u3r&s55VE30%e1Y5AYk--OlgVPKy%PsZ1e zDW0B_oktr)vU1?Wh8cEli-%<jDGJ?Jpw;ucd7q6?9v71WPh?r$I3XmM&W05`CS&R- zXKb|2Esyt;O}xzpq2}@V!K6zU>7eFH<=!dZ)>(dO8RM|)GtDZd^4V^@MDtXo8|0o> zki3wP4ZTRwj?G)kC|Q4&3tJmxuF7wwuA;37e#I5Maupc|@s~K$@AclVq0AXAHimBr ziIKp!g1R@r6I7xCr~_>awF0!qSpHA6MPmx@gGmuHgTFjmK9Gpb6%V{iQ#-t8ZA5s+ z8)Jk7rWC((6{+~y(h~XTVIZLizkAPp-bN|0*nhtbZb5Rv`cPdr9X#6+DebHoA!)st z*z*45@eh0OJq!3AdFA&`?D2A=@Ybm>rTo(Ds<p`TGk<bE<{-+yAbiWH-#ZnX+vX6{ z-=0j_(*OR+y6|vAj}zLJ*Ed}7VArQwtg1A<v&Rnv&TXDa%(hb#MDKQV(S}QRoM#Xr z_W5T!R?Gz}^>V6Jr(aIWHvR5#)61Waf3n6bW}dE{GA0LG4%PrqXI0*q%B?pmmiZbG z@y=l5e#j~{BZKbbrx@=a0U6i+L#7~k5Xv42KRq1l0mttgijmo~E$%c|oiobELGVxM z5CSvNaiY^CzIu6!J#%NL?A*bOCGLE&BRtQ0f%#b!`_*B(I8O5tuB;bHLbH}FBnUF) z2%c$-%=ROVr0N!$S=-B{Yj3GQM1Ic*-u-H+ksU)&G+AD|DS(jm)emy@@{wXAF$0p! zJAX-b1!)ru#_z6?979a0D-U)>4G)y~2dN=fyj1VmQA<CC*uxJS(BfcGOwH>r-ErW6 zsS3v&`tLzICYM3E(J@`3XN>=M5y1|?I^U&~<>U~Acm@$7JRv>)Kfbww)I$$|$mJ8R zK>%^<B>+MPp5S{4#IyGRh;UovfeRk3N5hHcMsvUXJ$Wz(077-93wFW9;!0o&B_b>c z9e<VyrUN5l8at#BYvA0_1S1?7x}N=IdO`AAU_{7^G^q<I2om<JUw!<$;?7?}Uh!f9 zJMp<1Lh)yg3;@+*4myl~M*INV7vh>hznFsp+L)Xsy4XLfzyJnq-#@;J_{ggRkDR4l z!*bpCFQA@+)?maeh3myf<o;mBqKa!3eE;plQ?L;O$l?j*6~s4!B>)>D>K|W_U91UU z^Lln0;JAb|qXw~gIc5OGF6<e~rHD^M?Ayf~J&zFJZ=gSy>gp~bhv;88{g@#_FUzWV zLyO>`0@PUjc+Y+THG<IT0509C<1bGjxCEa79{#SE-%EW71jr___@_-K=OKVVyAQY$ zXF`hd5{P6K0K^C0cUcG^9;5>xboY33{{ey&0I|4)Ve$sCQ%``anWvX(N|4+X!p&eJ z{DKo+2Y6z16dANrC%SI6K58Dh<Wax|A>d>G9Okr(p*Tfgu@-%eOaAZ_Q~{GUwmsLm zV3>P#SG-0xiRF?0qWua|Go_%m{S#*m{|m$;Bj4zt@RSI-_yECcZ^;3wk?=Dp5EBOz zLOhq$!FRzg7rGS&Hf={7bitazRA7W^W0&+LR&aiUX`i6;Uktz@0Rwt^u=^oRda-qK zD*))N@(p8d1m?p5=0%K~Up)Np_gB2$LLwHOm!5=xtdIi0#K0%Qce%ziBp;eCk@gql z&jHBjuqVO0WK+M}0P~NR#A6XS#1Q~MZ1zQ!AYTH(36Z~e>I(+~h<kt|zgsQzvR?ub zjRO$=-Xb{^0mS<Ufc)Y$((0E$B+CIHh+KSlF34X3fDm#J2>b^KBme~MB68LRl{~e; zE06gNMt_&Mip&ar#JJsANqIpeV@Q#<etr7S2nr^!c`IFuizx(OfsG_NS#kbD)L;1I zEEwgqa|kHNUw;B4n$cS3{vql;Am<}iNY)5^UqSlc4-gaM!~)=7X?_a4V2p17Hxx1* znE#Sp0YEBEiuH#v4#6rhmR%&5*ueP>P<Y635O6WY9iVAMp3=NP`MVddw*XMd#r2F$ z1ZX|9U}J$bLh+Z2+<Fey2yu=pLoD*`Er3bN_IAXjiu2QhsJgBZi_kVWaR7+I5zBs} zORBm9-kZ)~Hbz5G^%JDHpNsY0yaXZ|5~Gp4lxYMIn~;9X=aYnA0--AcfOzTr!5IO> zg8%?Tqb&!f$fahHhp4)CH_HdwX>bJq;{Diu3F7oX3?k<NKqRM-eL=aP5>QStVhyN& zVYrH1^bD*aIb2D1F(4CQF3sXA?mr_~f%0Z#3$H}I7;zuc+n(Pu{~_u#K`;X8%boWK zeAjP+5hNc;hc4)YAnNNAh!rfw(JsDv3&~8*r#t^@>J~8KewfwOi&g$t#USPghBT|Q z(7ivpJPa@fE8(ZH3tqbzAqA!-yrFf$2`wf7IXUZB*FPe_LSSeS!q6G=4|iyPUSEJ} z_pf3~61v<A8a0qDZ4wz9#2m?xOdK8i5nXQOTRDKjb)Pv`7(%e$Q?SO}Rrah4R)f~S zQ3h*R4aBowhggA}6yky;0^@5J`vNNoGC~4wy&>m-0HWd!0HWO@P3{tiM?k>wh2uq( z5J235@c-y2=H(?2&+-8fJA0TM2q45?03eXp9Ahs;1_1<ghRgNgDPCw9uLoC=*b<Qc zA!-q17?_?`@?L<|43SK@5$m53tPqV9Dk}*uMl3;$Vfl^l63QFTAPXn7D*TV6UI#m6 z^=zX0XT)1bEd@m1L9nN%;A1eNF-iR2Il!?|&p2bRaltx*&%g+ZMu`h$dodC6Lx2o+ zJerH$2Xljs#mscTE~NxVTpG;5LAZBu90iMj5vs`D=Ktt}16j}d0I@%20xd-KBCI5y zOF+?R0M%c^WV|n^4jn6_E`CRsto02J5MI5{vUWjrZh-LU6;}FxsBQ#NeK3g~6+!hq zNTb(b8~@>Eu#%t|M0NXfITr-gA463CDwFp75{O4sV515XA_f;!e+q!Wb&lcuhw8-u z2=;v}y9=s|10de6+7talb!h8y{e+4r7g`89<GKIe&Um1UA3a$8cZ32YRS|J<7h4am z1uGtje){M17kmII<V?wt3&8au(ZBXe<HD*y2%6wsFrv}#`5%UU3$gFL58?mL(FCnx z(HwCx$1_L;QRLkDfcX4kjvrtrPZg{vE|@O{7z>fyUE_cDg}`7xU={f=9f6Axst<l5 zvDKWejuaPv!5_!J*gs<2H-H7n@!E`wQ%DZdvS}p3rI-A3OB5`xQx)rr!1T8dV9Pl> zcXt0#^~wO#j%%u;2sXwk2B19}vs3&>%gO*)rc)`6uS38Ho<OQzR;%~cUup6hhREIN zo-&Bou@&SfG{;N!T>_yi1c0EfdM|+h;z1|?f?}7`{2x^>3V?XEinsh0LQpV-AGC>6 zcGAB)B}WH9B!9mB<&Tx+4n)50dg4D)(BcZF=-^rix<I=llKEKKE<YSwz0546)O2Vx zAH2414Zr5TzcD57lUM5!IH1N{!Fs=nQy*MxN$A$WQbuB`UUN{9%|e&wN)Ra@{gJ># zkx@74YpJ?B^ort!-3e|T?)LvU4W5FI0Kmo>ak`7;^~Hadjah_gxgUy@nUCT3@6NSF z+&(4qQhlNE39(0nO7|4>1z)j#YZAHm>aCdIH8#w4c6JOy`0k*3nr4l&e4YELw#>r4 zzt{#SY4HRIy#Ax&kNSI!7C<JTNi7wP(@)DL8s#GyZ~f2i#MJ@r&B=l;Bn`*POIs}F zQ5EAftl?j)qX1yw$&Ej-7YeeZblinSevkjQc8TFz5@4dFs<<Y^OwY0fouNU%8xy<m z#t>^xLpqVvC4v(nV&6Uqsv!p1Og#6woJki_i%q=2%LtwJ?W3RxVvvBFl;kDgx{zdX z9=%t);2BRr2?R`6NMvtc&Xg<z1}#^;MMo_6ph$2_&M<&UayG-O>_T5cXx5Ja=z7^| zqjiBpd7R%LSsBzyzAWA-27YeFYwCZD62J`d02Lvjf!@~GB)Es0iIhe9$G9UE3ttu; zF5&|%7>ue#&#=h&8M&=zgLS`u7&#X?%Ns}NUviH;GiwtsGbmJI{W8SN6Tu=63IG5| z4(@8w=eC_XF`Gj=BR1mC&J~ixC(S~=&ke=>1N1IX4-J3BRIu?w?r>jQp0bN67PihD zCESKLFYrdMT0N5=l8*a;(Qg)h-xtw42l0uR5QFdARHx^85f8(qi?*lGHYb)!u>J~8 zJ^JdC*NTxAy!3VEPbkDX6U4eY>6D||g0K+M{06v$R4VT>hQ^2RNkca{?}bRv7)3Ei zY>{1Ga<ckI3GkzotqIU;<#1EP?UP1O<Jq){%-4M!LD1`#7@&gH5zj=V`|&=mlUvXI z%KDs<*Ldf3ua=yxF8v}wSBzE8J~jarcX8{OC49d8HY)Dcj%oN;V-f(&PjGT=>pVDW z&YL0&uQBW_Lhb4bS-ew1_IEE=t}+zpHp?BF?doK{<bOA=>U%CTUkEAqTaYa&=gbpt z&ott$c>Tor1$|tI0YNOTq1h#30qPzg!c_tp;!zCNPZ3%@m>kkIy**1DHuG6uw<fo~ z%5Ue~lV`_fILTP}ZuM8}Kv?LYixF0&y^ZU%R5c!&*uiMUlMqSaDJTSvJ}TS&bB`IC zQz!9S4Too>b>uXb-%dR6`7h5{a&iwdZi=Zq0bQ}^cs0-SZUa&@FgZCcPyn}*qP$<t z5c^}dD$ffb5yfy_`8VX{S$~efIO_bYmh(KZJ)WtXDudtw%drO`V)9J}pls{hh~?Ws zG+}yL{}Ntg;8aU~?n;MD*?JF1ywQPG6o7i^m|yV<2Pc)hy$&<LK2A3nvN;aFUlLf) zh|DN+shh=V0l&RnwjhF>SrwpnNs+;#P2Lw8?BYOc10U1D3kU$I{EWbG$Mp%EQPF=K zTxjrva3BUkc3gqb+`yA0#C`+$2=S;o4c$|1phLR34|C-nb!+59msQ1t*b9XP2?XT8 zc*l!I0J-;?0cQ=bOub{}6H2d<#BKKwOGD`}B}x3&pxn5*aJH`StpzRG1(<c(C!3zR zO`(}my=~v!tycI<%?6zttxCn0afOZd;aEQATJTYamb|qwGbhS)51J91*&FI3cP>@) zYXIO#A@wR0%$kNwrb(B_Eqgvh(Qe$QFejS>|IA2thY3;n()F1~Tt~*^{xE&eiebdQ z)KTY4IM3<L{6L2D6XBy0=Z@oq-gjXV2^4%<Zp>-II%8tm_?!~xLXd$CvF;l{CyWp8 z!@z&MDxg3}_Iih-o)B8cy!)|F_4rJTXPi>?G|@<fMc_%XMQQo@p4x7zv-kYuzRCeh zg@TRh&I)!}7EDx%HD#XOaX^MN8jn%FpLd@B4zpWT2{B8PTVRVNQn6owJS`;g5WRx4 z03ov2??KB?L#DC?oU_Cig*bXiQ4rH?+`f}!Ut-O$Tf9r7axE{PKb9V}LM=?(P6(FO zbhSyJ?Ft);XL4~kXYhy#QDeaGisCz854T=`(av>c($e5FFU6;j&AaWVnN1b7uES57 zcxN{TJX#yIYNICRV~ZIyDG0PIOho4X>jmIa%vYv=#=O<sJ=N-UdYoEKm))zxC7w6# zwEAd&X0eBIYd@<_e~HY-U_wiBt-l9-K`FG?pxue`C#R)~Q9F3ct-SijMi(k+NHR@S z@HR7CO@>}o?Zivf$;M5sVR+fVe=h$!h8bQ7zw(s($LyevbN3*#vFaZ8A#0h34(X>k z<|;L|g*hraaI+(q+)OPe?wUsKGT7wzd2az0;=Ny7_Rq!vqeW7_!&dLcV1l=;U^`5l zxw#lL$W9S8;}x0r3CK@}vvS^iveDa?;FS<Jewt{PE#-Ltng$2TS4lZlE)QmfHt%G6 zC`^?yM7AYQxre*C*q<xE%btUe^qJQ!hWD6mR5SnNcd_3nTGQY;S5{Z_!C)#|>rnpu zx$YJJ-;+!ZM8@#(CIqP<UXcQ_)2_I_Sfg6t72~TQ3@<6cq_b*U>NP$fzUmcCdUr4_ zVzF_jimxa0BY}nSm)<&a%8_!1k9_mzNl9av4nsTkm`mf0*Y&Gr5}X6)zM`en=pFqE zQro4ntUcK}TcNzRw<2CnQ&mmNZ8KkV%+cXwUcbHd)rbt`Q-&7(l)MP3`SvGFYxnbC zri7DgOL|>BB(l4A9}q<UU<|J}dxv&h#hEo%TI%$%f!Yx+!5+PGqQbGkUqd4ml}v+I zZd$)<+Sfw5k9D^HtI35d7?r4eZ;*Z))sSvP!>W&DO9j5JDOhT<L_ye>t@pt|%XT+# zZ~TdqY=MJhH!S0mJ?SGk_L~FJZeqJ;%sk6K>WqVV&fn!%mk#>6{VpY#HjW<7c5*vk znp_AHZXOH%F?Jah_#aC$xD_a_n46@z<rthSaff@RtZSvXWrPlVJJI9&;$eERB{pOD zmbNN|x6-IQ8ib0BPl=p<WjbqmruGqH;5{@Hb$z&<Hx!())#)&@arRTxp~90otw6fn z+W+TDW!2uWQ<UQ1M^=MX?HE3T<kOsmc-WG)s)cB5C9Y<<Vm4(dQF&2H!zAg5)=MVU zSKR|-L^l$G>pc6NlS>It73s)X7+eOO_V_1T&)r%gc+0sO&AS3JS&dbCU-8C|j*PtZ zKl`1xr_kMosbo4Vr|PE8Rgkh+a@X86lXb2d*Q6@nFy*DS=`PRFBki{_?nB#Q@B_@F z64!pys2~*v#o1@fKl)cz$`;9oCL-=kQ#lOHw3aV_<cb<I9V~_$Nf8m_7^W){@b#b= zwkCQPGr*`TI~Jc26fEz?`$`9gNhU^~l>G@k13gj%MJL)9D<47n#Q<PcRg?_Ig~iFz zSK2=QD*1Z%6K3UcTca14J-E6=d5h_N)QPLTLqVQL%QO4>)4`MWbxZ|QcKu^}#R3&p zlnr|uV>C6LEf30NVtrL|ow184qnh@X9@KRl_G`LWp3<aAtX(gf3eY!IJr#w^idosQ zKMkn;E}wz9XR+z0-1TF>j`O~9Z+P#EL6%|^n&TwPmK?POEBJm(2xV`+v@r8QZ}3u> z1I@6W(I*?Pm|pIez`AFuJ3k1#xdEOR*#G!bt<Jjg$y93DM7~#CUS@in$J>$4eZE)M zYWU#xtu3T$`Xg^D_wK=XWrl*sYD_a+S<VdL`6`?8*6N+_0$n(HAKE_KFS9>S8=9a9 z<e}d6nu3W9@Qsuv-s86wpCsz(mMQGVV5&I-FHR&y)eLG8*bN!yyzA`$!R5Pyes^F2 zsTi9kfVI1UVQ-WOQ@zEUgS0HM?sQg|P@v%5Y0mu|-@?C!jbI7z$V4P%UtmM_#5Uwm zbn|3w+%eN0tb^$>hZ~QU)@eA-?LI9v)jm#7UNKurr}UFy@F+WV-C2}#O7DuFyixd6 zp{{^3{iGrAx?~l9jKxL};e$jOY4o5I$`tn#*Yp6FO>UPibB<++K;MT3hmB6u1t$9Y zOV}h0^PCTx10_ra4mf&n^{RZ>So*eXkzpUxpHqI_fsZ8+AEv`nr0R~vjQPfk!eATi z@#VG+mPPwhuszf7Z9}8MJ!JAVzekVdbDq(;oU{`v%y#Oytr?M2rc(}|#N`o}8GLny z2aO*oF__&*Z)VVV9a6Fu#XdK%F<CUk!o5GuV210zaQ48PhrKv);*clwcMQ*SN+dh| z;d;8nF%}uw(Hfg~B~HINU24x>d&NxcC#A2g-r(nRKlt(VcJyxl$K*bfI6Jae0^5JH zaDLZ;{qkxNWf?ju_4B|<IqxjCHt`0t<HRe1`1>8Hl!R&qI>#Ox)VLQNb+4-hq-=gg z3sfv|9+A*Fsj0dU!-_!<a?R32;<638sEOC`)0~va@i<<X_TG2YjTUvoSOaJE)4}-8 z<dgl;z7sDQHOtoirC973yP;4^nwa18hqjHaDX6VR8&p27&SwwzhsshK$4e;_EsY4q zhBk7K1K9gSneE=mpR8eb_<9`V&bGw#B`j!Ztsi9#q;)B*t+~Toq{aLWGvbag?5{?z zOLk7w?ZxtXR61FC2Yv4ATr1W(JML)<ShrCSfoacsV4W-+39R&wxLbB@ZA`QMH4@Sw zA#UsLgsd(`kSA!rwYLa2F7O(W)#?aUKC&;w{E-<Uov`8{rMI)@x9xE<x;|7|Xv_Bq zwx}sgFmY7-NEn@fcf@^=tm;ZN6LP-#{-dz8Z<z-2m=?ggIk6ikvh#8+H$OD;Rux`* z#*4RoBAFjrWRz%$4J?tg{YWVXGT0rDdNx_-lr7sjUvrqK%ocvbkX@H;4ckNlJ2T@C zpM$>96R#1!r70)oYvOb)+o&+L^q^D*oB2x4mZM)DWMg9oU-ModvAGpC6k{uw8a%+R zv}GUb<FG7j-g$@7!P5^PQx{lYe@E;@*}L<}V=%ACE#FdOAd08pui@SU<wPFxJ%ute zULC+iX@9cvrB(r;!Iy&}O??Knx{Q6jU!AFP@7y~ieyps`=A-hyD1K#+PX6A;e6I08 z!B-6Hk=^HgyY^-ijNOB|y6uv&gZl!!Yb>Qdsl)4~uHAK0NV7PaZ@J$6AU7m!3VU_R zXo$6sE~zR{b?+XxEod~~QFbaO>9fsTbi6A@YuRc`;C9@KI;kjp{^sD!uBtYHTYB>D zc)RAbrqE|CTg;O8h7Vcdd1AS=`18+_kPXj_V~Ji&z_^S*ENl4__|?7=EDyMD=$u=} zg#Bc*WZd;!xNOog(6I;O>h2SQ=)=To`(}$)6D9FPEGF8K`GO6p@h5+yMafV=44ub| z9x^0jp!9L;&mL#EHi%k8H`VP_yY4UD%%`i;9ONc|F^x$d?DoXvJDPegHSTp|-81Um zekVk9Z@_foyDtS#jdkmZ)+#r_%UYROdtz;Yca$uBztijj<7$4(T#NV5uNkRsut~xE zGEU56S29Frz%YD#q-FBie{$-z>_c99NwF7)*Vs<oQ@A5t+m$~DkK0<xfBvGQJ><)= zc3Rp=;#<9r;^;iFt(PIbsUQ;u3@?dpL%W(}Qj^}7ZP6jx0&!7yD>6SF?Y$_!9w(40 zVfi-ZTsnRM=A8EAK%aQTRMVM7=Y{MBW?J*@=q04Hs^Ge@{<)uLrwNr~8%S*DHlHn5 zbNZ@l?r@}FdK^qi#qd~s;hQ+Wh2+3j_g{FzD+l;0?GkS5Pz+N)5NMLY3b6^8H)co4 z^t%qEHOi{e5T82NL=)bf-0)79n|t_WLp}2O*ozi#LQ;)ATbcXIh0BOpM9~u+L;3Cm z)k|UY+QFgoJ2IvhJDxR*8T)ly?RV>ChUX1|V;LAfR)luW>N`jz&(z0ywg>iC*B{mJ zI$N&0Rv3sCN1ubn3|o#)mg6igC#w=sG3BmU<$bq|WMgO*ny_RvKkZMP>%Mt`R4lzR zBp%gl>NuCW^UD#54U?*QZ)vr(FT!KM+@XDX%6%tYtE!%{W<k*VyAf&){ZW%5A}Rto zGvsRE>`6|e`2yuqf}J4C%7|pZ0iBAa&v#LGJxr|T!Bof5&uIIl8~5FW^}K{ckb+3S zamau7yl4^W^gKu1{Ns{eYB5u_qREp<4H?FSoiZ`LpV%GaRfbjPz(8G7r`j#{=s7=} z>JGu37+bc0pDC4ImmTRo(i;Kp!60vb`-Xt~%vzte=ny-6X%e+&Dcbq>kA`c8;p2}F zfuT_yV*OC_uE~&&dZ}&54rD2aN|4#Q6|L({MTk%{5{*B~#3DLd#6Y$kU1+#=+!;PV zq#4}*MbA=>d18kOWy)huu73K2M`Gg_J_&F(Lq+SCoH9`i&s-*7iB>L&b4=Ktf4g5( z!!cWub4+GV{S{tT)~tZhak73^`r~9_45<S<{;X=a8pW>8iODv;gpEP-9!pjCxiyj- z)+CvL{Jh(7+Wx&)+&szoRZdtEQ4cph%yIpvk8$t-Lgjie6`}lYiy}Crq~?Mq<A6t6 z8M!#U9ygdD;o)Ew6($i`X6e_Fzz663w7lWrU=dXVKaU8LjE`OB{>A@#_P5T+D^ZQ5 zlIO7B0yWZN!b|v4F1-v4;WI=$=3E%d1Z<Y${kzqTrE8;wok50@r23??hkZ0bk4`Jb z@RKZQ9&Syw-#hsQQ)3qYVb{KQ0`K!ptF(TXpWEC=o%c6p5DJAv2!2877o2kd_U2Uc z(^8fc5dHMAksn82+>wtr%!&)bf6`rTWxG)JuxN3JteSJkWL#p$;QFOulLM6sm~?|D zF#vPTfO!bZ{j(X+I;OLD*>WGboJ~*x$CTd?lTF!c88f!soMUB_E3wM$<ZoSoE|fbN zbPhOzVhPQlxWWF!*Gdk_OI5>GFOow@EAO-Aaevg!Sq2qxi6}bej7VOEojL4(0D!U@ z&e-xE5CFCS<v%*7x6qM_D^_0;U1WVAcNzz>7+A>Vk)eO<0Wcp8>V#x5d8wxA(?JoD ziwp$$APBbT)}UrWF@2mWpbZuct(;H*EcOh6f1?37H@v}b?&8ovRQ7uWCce8;Ag&Zu z&nRCMeo;+;SaJm{Q#k3}4+Y0=K;a47;)?1w^^8pL2+_Z1*NwMOn&qb79Vk2TEgDQ5 zV5gFrC8>>1r`q<HSU7%mU>TK-r^`?(@hy}FuTs^}e5#EfH6fAm2YO)PH$oK)*PnfW zwxbJK`%OHI@&i;X{^5*se`7T_zCk<7*(cwH5}85-Aht2V$dnJl8vmOF)B!nvjB6Ak zh=3=^O5*F0;Abx1iWZTGwEj<jf@Pqc4Y~v$K`w9-Acl_xev~qxi9vZU?SF{<pJE%Y zyHNhC5X)cD4y}WNO0x1Y{8tF=Y=uw!6f$Tos&D+;89-{INZt6d&`IzvP-2aq;<(^L z!2l2){jTy(B&!Mv38!gXE~x@`=D$mtdIk-H@~%m7-{<f^l58ZG9sQE#H^Fao1XFK7 z{L=&`*3J1!p#(!J7H^ej0&D=8nBe=I|HEeght2*EoBjW(&2ZQ2%|IW;BCAs(I+fXH zbn>gSe$i`V#p;8(8cg85icmJ@3KdWhHa|g=re_*?PFHEY5E7aZvD-ECwGavJuoj+6 zd>*MwB`H~Zg7T5?zQo==K4-<qj&mjSM3n&kz)=1qq6>NcmJJkO^&aCIq|@brnvk5J zP6~R<%c8#5tg$pzRyeMiouFv><H>5IKzGbTJr{?qHj+oDysZh8?@kkXABCkq@5@qd zTf*xK0kJEWq0L_^UWv8QRQWq=6s2)|lw9Mm?cv+W`rTVT+o@V{qM`e5U3<^hWFHvi zjmyaw&tx|xOu%Qu9Gd*kYUj>I<)`~um+%kvm+SPKQkC^958>@0Y*FC?8$y}#@TXZr zqDLpT3>|+eSN@z4cL$)u;X>bkbEKt+L@G`Z`zXFb#MREv_F=77EVd;Ek2w;3K}#Qg zzf?~w=-z!6jIF1E5%p#(O_;s#IZFBE9qZnz?i-eGbrK^#N|4@8UaFl{S8EPUjKiCX zV*X_L=4Vim`L?Lb@jT!4_QNPxgrSnX*~kMJ@j#IWHWBG^x(D&m;h}Q6L{c)%bq%tm z?Q0fI1Q+Ue8cL|k@Pryd>hq#r2d_J$NnX3m>8tb7$?&QCuQ&dw=5;++CqhRbAr)Uq zXRrGvp`9N1*!rM=tLO*Eaw|NKMZwEoqK?a`5;nO}668c<(12>*+-zA#O(t{KU_)~{ zyl)N!ajBx~&7PHWvVzd!EEFUyBf|EVQ99M%LS~yb%h3Qq$u0dTf@^Ab{=eb#C5m3+ zQ8ra(jDU@m4;yQxwS|X$GNg|lJ5gZG<{Okc)2xvn%-5j)E{*eAVg%iAR7J)i3zP39 zvpORRIzb3XAWW|fVaIXhgh5swh<{72KXC#P@pk>r4^&%f=N9stA52qC;(a+;dNL!8 zNhF!UlWdmxi;JWaZHekVK`)B5dJmHBH=7hQjiMcH#o5Ojj+P4zo|X3JW!UR(hE7H# zg}Z5=^B<aSB8|Y0wYW>iV?a>G*m5>p`KJe&{n{`!gH-(XJOeq1nAPrXCf+tXJyV5C z*t(U2usi#l26<idQeLKXYrh<9RCJfoVDJrwVby1T8lJ6<k}V^bfxeCGNXDYBG#P=f zV;+k?uO4f}gHY!~^$aR-(_HOCf}+M29wTFRBvbf9YL=SWD)kP~)n)!zmp4>mC~d54 ziEO0Mu6W0IH+o_yYyY)+-lPLC95YEJ&lvO6%EZeoC-pNp@afe$vg3{m+y~Hm-04B= zG}*1ddXubareG<*Sjr>9I>TqjrCq>7GDB>GEoH7NSd_%M)ge(PYD9rLojCfGd&7uu zxHhvUuifvtrsEg4snhqU=8Ne`MZWD-B1b2|M4hdFggh7s`As`~C4`_r1x?f~D$Ky0 zU-iePX;-H_LSY{{6Ry*<q%(iRHld+yKU0GpeMSF0qdLnFua+=)16@q=*SY5Pf-YoZ zzS#Z{5dYUAwHWE)l@*X88WfxFh^CK!CtGqg)b~o&S*S-}rjd8LyUw*Y{XuE?(Z;XY zj9+O%{>+I;+pbdf7)LeGT%-k4=LQR9xm!~4ymN2b^CHbgid%s@q=(JLiWL<rNE2RE z>TZd}TShw9o&7-8c3m%h;MC*w_D0tlYxPPWr^@!wLM$(6YVnXqVsG`iolHLHuB^%6 z&C6Myl*%CH-P~?bA)Q{!%C3!B?2WoN#(!=&h`O$LtS-hfU^yYOg(rSGcZDx@uuy0I z=YDNC6ssRH_8Tl#g%O+8upfdh7^*vCAJb)ITS-cnC4t-4YB}Y~l;X_KP(G9cxTI5B zY4y12W~+Id32fVtd!<owZmE`X&U=%&UCc5F<4dCZ67MGEq4g0wla#@A`uSdLzL89r zgBW&EA0h1i@>I^T<khv6(Smf$WKyRiZl}|<g^A2P2N17ZrDZNx7~^(Xepa~^NpHRW zUAbbS`DF03tXQ-Xm-(Q!W7}}UQxwBQkKaMnd_{GyV0l$*>z>x7e3m!Y@qT;^;TQOf z<ccV22Nn9jh@1YxW{eIQ@u`3bf?sU9tAPwmP~-AYdJ+lRUVWIX<k=e^{mFfs)jKTb zZ;qH)6v`F4zL1x{%%<dVUd;ZO@ji%}ccYgfO{DvNBnL>AEWMd}CU)?u!qz^meRgo! zu9G(H_H53FC~>uBd5n(q8crd@*v~>@FJW3~Rk<I185p7%OHzhxW7SRKFj99aeC^Al zITQ3J88df#aSfBgPuHWLZY}jcA6)j$;*c%~S9U*KX5$@@`P#Ce>E0g1-q&LS#Y=TY zz7I_@M#Y{ly<%U;vimYn5s_=Hw%cxWl$%fZKJ99XuVxg?Bb)GHDXf;|YJdz%e`np| zc~0xhT#eONv&W7L-XA{<3dh^hf5s?Z6`?zM{WLO<E}gTgGhxdY%_sDz?)0;A+Um%{ z*lFL?T}t3X;*yZqcZo!~c*T$^qGI}><1^7mO*z8vJ$X$|&N5bDm<IB1@o2Zz$BJS> zNY=Y!n<>w1YB+ZEk(=tbH;Hu-Ixw}7%uf#dzy0oTXTCi-+BGsPnaC*X>U6={6Ma!t zFJ+nBEw;oum;2SBIAiSCM62?rBgo6nHg_BOoD{v+zE;fjuI|-wQZCx(2_E;dTl38z zt@kLUJGKQx_Gz)7j5MtW?HsO8{_!8lCm~wriWy@ATK}mPL$eYEw9fP2`SrHPUa@DO zd}?Oo<ls<h!~&nJuGR#Tn6$E9$~!e7UpfrQ=fkj#9lARF-FiXdMBGDRw2mDiOefpa z5%87ia?eQQoO`N0V;Xd3)}_r*?!V6?7|>~1F*MP0SMNyhaEQUREM0p=4MO4O;>O#k z@@K)%x-rwt%GYD_;Jf23<z}x^3>I&cdacE^h91_5$GNAhDt42d?H5*$=4UHbEnhF@ zRMNVgOchAoCuhXW?lL@h=rc4~u5hg{YifvXiYw)#W-tbW7L9b(-gaA0rq`%FjD3<U zde?GjIOvwE`|2IuQ^l2xq!EWiSC9f+c$9K&qYp-BL8OoMYX2=}lr4I$1yA7X6tsEU zZ_%|Czw+LZ79a2CMjh?7CvA98RZ3M_yj9)e?+xRi-T4TTFwtbjxeAe^g;=}mWp+Kr zfkeNT)6Y^|?82vN&k{>O*8G#ha|8GkiXnFS>YX3!g>F3U-CFJxc5&rVD`yy51ZPLy zBoEUq;XCIw!*6;IaLu&$dW)r3w|^atYt5$bnx?kTaLj5bY(%D?0bT8SetP()K>7*f z#3@+z;zLf@W8muk{4{-*1Gs^V|5#j;W96N(r(Cfd1NU)qdyp!xWTA$3y25R{56K>V zTzJbj@-*{3w8eA{l(acF;LHZI+iTCnq$Ui`7N#Z%cS9F<zS>`F8ZFH^X&yHs5MNSz zxrSOdZ$CDb6uu^OpCoq{iAp&-PsDEV$5Y;_=VC(Aq#Q5!<Mz@CK38t<lw+_Zy5=Qn z=p&VuEbm9gG-af;Cb(}J@bYwP1@X>fcQj$I(oMN<d0`H-kJE#o_9VN+c`2RSV2F?G zgiG#zs4qw(OJ);swN7Ajg`yY|nPL_1!QfZ*R;#ohuGdXYrWluBEu%F`NvU1uGW~T} z{u;%Q>Dz$2)R|#t<vd}Jr~F2)-H%QbLuQ#+!r#{=_REvpq6b@^c@WDV-vikrkU{0u zWpjd+Xt)v2L-BLd+{y*E-i3@o(m&OxIRCd{2#Vo)sjt{1&)N3Jn4h4+gqPdY*ID~! z*9%hK?0|b07WqouIqm)9yvhOrOA|dZb}NU=?7TjYvXR)hPMJSjx7PMMW!1pC8AmLR z2T<gXwywqlYzxtiNvpahE0FaVM;e<$sm{JfTAy~T>Z=T$17+^tI~78$gj-M&i>4{n zm9!l$r)08Py-M`N1vW^n42oSJ$F4h(ucm=0;WClMk~#CfV-r7{aC&Emz`_~hS2*88 z5TB;uL6V%h$^A24wv)$5a_t7K$BP9J0X4GB)uj2!`3t=Rzx%Yc?BVC%`dUmJ3NP&* zxr38^Z>w40`lmVk{(O13*SPw5tEXO)I~quPvdD^&*x%!slsVp+GMY>D7$zWJWPb^p zkt!EY+^7@yQ@ZF!0~peAS$Y&IMnVRKy7-?I#uM?(wCBlzHORzsB>MGwJnNUZJlQmb zI@<?|qDXA8l3LB|dOGkv?M@lS7F^_fmp@Y)F14Z}p}=al#tJi%8e-0N-G(_=mamQ0 zcIHGVJ!?zUqWifPG>JAZtmdUiYv94^^hSz=OOe!ktdj|AqfTtA+nbjoN`zWp!Wcb) z_*d%HwCfSZ1I)SB{4SQEQk?!%$KDLN?}(UAPnA`F_i@d8OgS@ai5u%{Y&pYG7VS)y z@(s;7VH=VJ$3wfTJGqg-yf7U~@B5u|9NjuO#>Ga-V$+>9cnGp@NLlW~0qbMs4*5>k zyNW2McC^`1439#uVQR!h%lYs~57kv(84%!M>RmYA=>BtqV;Z`ykl-9v23h<KK%7w) z?(7?Z$miix+g#u?JReF~W;c6-8Le>xx8p`J$m4|O1)88pk=V>(FO3V~pKaBEbkYP_ zUj6FF#JaE~n3^+X(Kw&?Y5YraJc$vDw|&y{fyvmM#>B5(;$M~z1gz6rnGhbQi*gZ3 zi<K<3aGuYLQkAZ9bkv4C$~{7413Zc?n4g0L=8=D)f(3`GBxy9tC;i&R#e)0P^?e+Z zn$G(klVjXri5XM7fuCSRisdU2oiUHT*N$uz+E19@*e`MFOAC_e?=d}M+)ZGaV4z6x ziWz)9lqgxdL}b|dv63$NNK`7GD0UKG+AF5^uC7!*mi0ic(p(59>{`~k=Ht&gv)_lt z^*;iu)D!*pMdh&9W2k&6r7IPxK@|)Jp{J)p-;a$Ot{d`Dn0`;5N{dzPdoE^J*@>bK z6O9$=Ykzse`0dCq;diMce9!xij%v8u)py*1OOYzpmul=7a#trkrDb%W_Um=pswRcj zzP8c%Me(e1u^muvMeuwApM;gYkuki;VPlleoU~$Ka_qcG9OhVO)K^vH^p$6cXW9HX zvSrHSU{uDa#Q$fUvE#ZAvU~Pb%kIb%AM_0FMk#_KvR$ELJx#~YDJUE#{Xi2{1&#h# zP<tM<%Kc23>7jc0wPSEBavNS_V^fn7!%r`@bI*x&lB*|`i)&6?(-CInJKw!Ju{M1+ zI!P@3>L8ezy+d4kFzYCAFAc?w#{Geh>=U}r5}71@3sRWG_KpqJk9IgWh)4+cOPz|d znWpn8$?Y(xSjK>3;f7m}5~Vd07`u%wJL8QZ5VwDqaY4&I!XZ7coLWo-{SK3vR)2xf zK9f2bhn-GoLXIi&vYMBOEWJe{$%YR;F8@?EE!jm7%a09+^y8HmvCxG$WT<2ZS)St4 zEpT4tIx$Qic=4HuCtIHKX3o+TEtZqbIutvCmcuWmSih*#Qg-aPmN!MX*RJQ8*q8K1 z89oRdqri<e9$C)~BRX0PFQfP_6Nw_=Vfr-;oj~J(c}McWx^#FR)4GEiWrfhgG3ms< z6(;yE`&O-jV5rQAq;x?`U}do<Wo!O-Evr$JBC+q{A`8iVj`QPuGx7ccH(8gO$gr2E zJD9;B4snabDmCdtWU<Qv)-h~qF!O2vMn?&wTaR}TPv2)5G4HvqxFMPB{vP-8A@-Gi zKGF)Ac+D`^KH~(Q2?Z_>mZR-YBV|lg0*8L(0mz2yfeA-un%bhjCX5De^WOKk`cPAz zH+^Ms*QF|nqn~3$Ci`_+j_O{98~)D7qG9^`<>DlDTCW%v)i=P`GE^^6m+*?gpe+pm zwuJd0Mu)k-O$y;|JLdeuQnAkSz6D_}xA$H$5qp*){8Jvxe@diqm;ujk_J)-}$^46Z zo7oloY~VP;wLPVS*+>fGO|ZYH0KR3~%bAy&EQPMWT$g~=?pCb5RZd)$scX^W@laTd zAaG?&Wd8hjgnMZmQ$pVTpzB6w^=4mVBYJZ;uUYS|?Ms{vdYLAq*-KqZrdG<6vKq~( zJ>85`C2IpAnep7Y$YawnE|RJc?#ZiDsOEJ+FC>k7JRH<$4bFk37^8<ju2rU;SEB*T zuY>!dxyD9wJ)gG~`4Dt)RG+h(mhx?!;)EyucNP5r(x9MZzWZ_DAzbfzT<K!%@1w0q z6T4a4V$G=Q`e*g4KfO*3t<HL={bF>FB^Xi~M$HGvBF9Vyr8yLL=*{_G;c(|jX6qgO zoQfYyTk|d-Q$UqI)Hs<xvPQ#HP*B^Ml1}T7l6f-Wkm|rdLbta*++d%}HL*ly969)C z%>(8l^Hr-HrgF%W*;61=zgVy?`{Y#yN=DgYoy*dj$j#l3{kgyf=9WLsT%~T`Dc^2H z5~#Gi{Y{{;BalLB$*FItl&5pBZfPoI&qzW&wOe*6VHI(^Ly$ZYN`_SKe}Pg&!BCnb zpz`~K63ABnU+legP*wf+FRoH5f`Ej8G>AyIv~+iONq0+2gGe_BNavxuQv?nT2Z=-X zA*37bK0JQn-ud2{`}c2tGv}E{X3jo)?{}@WUa$39tLzZY>!1wPv<R8b0=fG$^ZTn; zAiL%HTH|J2djYeuTuin;RfDH^R^|m{N}c4?`HjTtPenZnf|cqrN*=3rc8Rval&^mn zDl@Fc_XS$vL)m2J8(qTFE%Coy4%=w&o(uw>`KOb_6vuReokUu<4CDPI2y;4li_<lv zf-fKh9FE|)0d|og5`r#^&25K2vu!DWFekwlbIMQ()3lHGljFR2$i3SR)V%(o#Ys2d z@6v{90SEK{%0&;|1;qeWZ2S+^ry>O%U^l%ai*ocN+nmKi$XxhvsuJyAC<{>T(A{0d z<H9MMZNRYQQOA?^Jp*Iq=2)15pSRqNA6G3E*zI1~G0#1H9|^w!KuW+_G4nn}8{jvV z;G&~Q1jY;ofQKnq$fZAg_h+*uQ&j)T9*DI1jE~J<!LNDrL=mntnt}?i17{RaQ3FUA z3S>V6nAz}F{+yrxU>fkR{kIe)^er$g<~tE|rKo<IxGPRp|2#AN2MPg!VffB|mLvRm z{!ujn@=UDcUcF33<NtgO49^;QW3~_XZA9XLV43Yy3>OmAL5F(S_@D;;=Y>=N`a{&^ zT!9O?l9U|S!kC7DT=5y$sC=Xlo=yfl7yLdOz*Cz<vbw<|@ocz9km^B!3KKF&47!;A zFJWRaFdNxTo`r9Ki!d;OR|u{6sRt~q%4xI7*Z*N=oPdF2=}T#b|N0mlz>#f@334H0 z8Wi%f{V&bY6!18TP(BrSLVai`@CqUt#^nG_{u{<>`5%4Mf9tmV%;8ozf8keG6Y#ub z{Pq84BQZkZn!Un(;`br&0r^K4RH<$b<dOi%&!1MW*p(`Ng(>C!?pK@`Wgonc*J(VK z0L-qKY<f0`JOrI6zFi#@{Uj({YAo(+NS?>o5wA7N$Qu8#b%s)%|EG}{3OXBUP_!`0 z%U_En_p!ci`_Zh!{Q5^5gPF5Mm~)vadoB^bn-1+}TCL3TUpIRm5k3rupE)p1;fWbP zh#y87ScN7TE-H&OzdNh}Rz1&m?x&+C$U~`B@VL-?B8Ead*J6rx{p9(kL|@iwPPW{b zMi_{YQ=(6NKS8>@S(LzKmniYQ*x)f!g}3sh)56Hn_}U1izMsFLg|<{sDW~6RIEe7K zy<AlNHmB5Z%hF1`T*Ouz*s{DG6L4q~%lUoeo4zGe7JOIi^nJHA67rI$%Eaq_`^AjW zlLn(+>3$%1rk!4J*Y61!{z2Ox_Rs3b1*plyC(q$J4!;j@X6_grz0Mucv{R~gANXJ2 zq8s4wi@_7BbzJ74j+YT`5|>W2QW>v~f>i#jDI%C0wq)jtMs+HE2*_bGW@kz}+p$l% z9EL%p#kp+Zslsj7%VLf|x;Gh@Bg;Q4GKB`XWy<V4T+$zk4$SM;RtQ}=K<6g93HAW_ z@vb*LL_1I$;f&4$ZyNWbjOeT<;?7nS$G#U*O;IUyRC4rjSu94(OxLgxr15%H1TMfI z7_YJNrO~ZxDZREzdEr@>z&<sW#Vr+&_IH}#Dl1?q%fYAV6R^t3VT7!CA+$h-Wr7;L zHeIiD=G%e{2h-GvQaD;Wh&P-x?|w93u|CWK%f^u%v${@{%~oCpID2KOOugaY7hH}T zC&OqaQu&Ea0Psr#z@=5)@y%XiUaUdo(eVQ1v6=|<_4H1AU-POAhxC<tzJ1{W-OLmr zQ}EQ=B&?f$wTy`imri-UNjw3uNxO?7(&m&dniz@h4tY6ON6hD_!O(Jabzj~*({4FK zK2v4b8qc7-5d8Engk7IG^|PJm^5AAKjWdN?|6R~m++z~-i_%koCb4rAy7O1s|BkwQ zrJN<#uTs*E%NPw$V^AJUr|Ff72cXR=_wz&f{?aoI*6O-xDb04r6l{TY=VplNM}d^4 za*h!(zb;X0^A#K;eoDLQI;UFAq<zF{{^VdO0MoQ8V_!VRtME?pnl-U}99zbo4=c6M zZ(={M+4Y5h>PF%d-K0_+w<Ax+%|yk8PXWkzjh$jZ>y+5Gq#Qs7_IT^}7VD?as(s}7 zudGM*b*miqEK_(rqPfg6(s8v*KTC2oP8P+sD;TN=-li`s2Di6*_?^Ynm4G!j^Dkv? zs$COGH7k4WTJvxqGn>{$?hHdfHU^$t=JINK>`b{V#oYeSbdN!*>4lZr^8!ZGep>J5 zUo!~|aS(5@&)1iKPeV3fpNcBmDZoh)(ti#!uAKwN766GP9*bdZuCqIyE$$Qo;LkO{ zm1y<8GOX96^9FKK&qzk@5I9vpWVIPyT7ac9aU<ci{|E;e;j!L58;xOCGA`o__GMX` zW`X#QQmgJ%4!WXa70(p5(kS(^#F>E`<6Gz$!g^9U|M8hs=sEaHrK98aM2k1gl>VF+ zEUaK1fRRs!JvY}MAGMz^)*%B4MwQp6{LsEOD*>4I+ZVgbppZNsmtuXuy06d4^M5b} z6#`f;>nS0&eH*g(P*Xd0g>6z(1dB2X-pviDHR^8Eb<%oLN^ftPy^tL|c>+%n^-$I$ z+Xz@V2pjfpQTT~u%DHg;rIcL|T!rM+I^ja8*nF4M%4C`TY}|vb$QV-6^e=ebTpygm ztjhf|(>#C+?K-2#sX)hmW=g4IE9?lmyS;E}dAsIxyY@Yt;r#3pff6tO_Roy?k>$PZ znY(+_frR!;G-mNrFEt=tdrP;3#97K9VKz$oY<C9XteKHJ;;7ofuBm+C05J5GjHw?y zPC{h}JW8hlj9wbhQ~c)Fy@0)B$(^W#-CZK&#%IvCY96(tO)rSgxyDV)5Z-FTQE|Yb zx_0Xy5V)_{b5$li{lWigD3gS)Ak$s0qw%=ydYqs73Ja-T*TqVeIilirNxyryzWm-y zd{DIrO1fm%qr2%t0x*-QDn-@@mn*p`kF!c<`II*qxEg%hU!-0IH3u&Kb7qRbRdFgV zC<$<T$^zF<DJSRLzlEo}6(%gu6ITj8<ojM8GsdEseu_eVLGXibtg&3jYe*JXP@E2J z|Mz^b2|RnRtYI5{#zain;&tvswQXa2-Lnj>Ij8pTcV#VHH=@lQ7oe#fi85aIwR$`L zB>fGq_h!kM$`9#)gAxY$S%ZA1nLz3>KWA1V!|U>E=aRF}PBfN~#PLVs!Ua<5P*Pc$ zk2z^=WDW{_1=dWtF6~=8grdT)b(19LiUk*d1pB&80QzlL0W>mP_v0nIx9`>3AdydZ ztZ<{I_W)H$<({V1tiUz&nI7rJ%eOVe-Ay%BegPT!Yq?lQu<1-qRHBrLmQ4djb1#4~ zu&KzJm@6w$8j4)HUI-KqmodoEW(vP+=P`};;%2vSI43NIu;uI?&5|k>VC|UazybAO zVg{xON0*v}vp53p(?d3%)ZG^8@_1anxUMcB5Yj3`=5Z64|9uu^6|xWGyA$qDp|w!? zff_xZeJ^lxU`%*ruB&~HZ}Ft<RqZ?c@x~y!n|06Y@vqWljWgeV%#xC{@Q()<4mG{3 zx&0(Bb7p)yuzohbtgAie!MkvW<oUA^P}9JEbnb)e^{a0`-}*kT<ZmYctAZvJ_KT)} z^KQ<`*w-qU%Dgc?pSBt6{N_PxTgIjT9Q(bII6v}r{qk%{!l=E>Ju&X>S<V)pi<Oy> z9+6*RS{t;=ZBTsg*nL_!Dn-$JbLNpOtR#{s<Qt&gNUGSh<jJ&Cn)7L^^&l!T&@=xb zxK#i!Xcszl#XdRE%mwL>&VdU6Vkql^t$FVnv9}l>Y*AKjO?B~)O!{Ly+yRvoS3rXw zUatUC#)8R{YUjf1Hxnqxw+=*75}==;HQZOp4YCE=XI2vm&JG6)uVTfXHj)a9qqDP9 z<jjW>qop;m6`I~3f^uZoTE@p{)4(S^13&;1p)_7(G{Udbd$;O48b3l+o>G+*(;>5n zRn!?{%WYDgcNx-dd)Fb|-HLmzZwFfuYl9>B_~$E5N|%Z6kId7`79>(sO|MdegSn@w zZ<nU6H5r)TfLb<%-;FXpbmj0=%LV;)!h>iD4ndy+S(l*n^~C}TC$aM#^K_k7&QeVp zD1m^%5D%JfH!`zUa<B$MhWj|O%Qli}+R(}9a1_+8JAPBF&BWIV-=iIX@WUWutwiHU zrnCBd`e_?b`R**&m(Z-FejvAbX1f?44LtvjfnFLm+VpeijB9MPtR_XSC=E55jp@2O zFr2$r9-%w&ZHMy&(M#grj6(;_!*vCVY1O+fSHcCDBiDDdJV=nB^o7n(By6PPOAy~y zWF;b!#*xTRFs3z^YndGrAm%a=o1Cgs=7O?<u9daB>EhuB(>1XLIe~wpY=P%as0@*8 zp<j|Mz=@e4Y+r$R<~Ws_oCNNdh}=Ct`2sLP&Rhdt^RKAP5tbZs?XSU$wi7>8V4$>n z{F@%c76D%_?H%6J(s;gWFSje^_9TW2S>zheE`Hb*n0is?7wqL#Y-aaE$p}D70<XM4 zd)f$3ef(RliP*)BFQcRM`qj(S+LF6iLlUpOhzkcaXA1GFR(5GPlf)o0PUC2%`?-i` z=nkSF-p!~hEmJ^s#hE^y_ZY%ootTI$)r71XrJn!W!T=vgKYtv&&?h_f29S3xNYFzM zW;u>>NUWFEpM-5}poQQJm**CIPD57ZULQ(Wp1o3u5_Gl9a?8FPEvv)VG!qiSdwUsm z%ADip{3#aot<FWRq0SL|#>O3|_r6cHPjVzL0E!rUxG+(qW-w8t5uf8(iRVW6WC=G> zfSs{3<+LXi6}h<w+AR#+0b;-^o=+UTT5J}c9OCfRq{;Qf;}(HytaK_NrUNVxG&N;> z(~;piU^%GQy*3|SUeT=kMP&|=lRH5Lq;*2@{t|p;b2rooD=f9zZx|`9J-b)7x!ue= z<I3xhDPW59YD=HLT*`4wy3F+TG@F#>kU2vU1aemQ-JfmPb=TGpO0Yg3EoI)Q6Y#l{ z3wE`jnTD#$IwPjp?#~ea87|0Qyz|{__Z^ztl8aiP-b}@eA)vb&A-cMoUP$#h`%?bq zT&L-_^Tw5e(yr}5(0!>tcWKBsJ(6;2%z@$Jlw|d<oL~%|JBZf$kqGXg|5M58ps-b2 z41ra5t;M94aCeKKo+GI}#x4buPCq$@u1TV)j4N*f!^Q6ikJ@_|6tFI^c(2)wPt<u- zf^U%@-b=k0NzLSWxk`fY)hSEp=oU^qKgbz`etZd!^LDpgOLMJ>Mg8d9lY;!gb|OYx ze!8hw&T#S%+2w}VT!Vqp;e6&Ng%HzzdDIJ^?bCXx9Q4+;zg%-1HTP)@kbus$^u~L+ z^H`og-l$<+fEmPcv$<`Cmz>8&WhZkG={jIOqF#_9dM9o6j%lK(lv{PYU6xasprD=k zEvb&f<5->%lGlyiSK0*&k!-Nd#?wY;sTGsyMbl~337WiZ8yX~2Ej*C2R$5s~0W4)Z zQOf{#U@_RLJk5V+e=lEUN6Ujdb}%B=xlj2TG}20Fk)k9~G~IaTJYr&Ll2@Z&VB=<^ zLTlhreb1Y|b9EiH_?fKX{`J3Dh5uQ@!UE)=nCBGiH@-EP&1;XbWs5H3%!ko6vgdky zOS4==-xA=wG$W$PqsV8Q9AzK<n#2!fuoYOKiw7DgAewSF$re!VY=QhmV<Ekcrwg^- zCC~!9N0Tne07(ttmS+>8_=}zsO1fP8HvSCB_F3`7zJ(S%ZmGB4zb_8vnx|Waj_JnO zmz{Or>&n9b7+kc#|B92-hOLz*B~vDYRJjkWc0!_@O<8_oRtE~i|2oO5jMJ@t%$M2s zSpbI3J{e?)#?xZZ&2;a5QXMS8-K#L|Jh8|<F3))%%HB*7L;?)yuukT|+J=^^;IZ21 zktthtn}cfo<dbTJI}Sj=<2<>ce6DA8kajaOmVO1s<vygmR`NT!OVn+JI$5Bi9fg#H zl+jEBDzrYG^(U}+F&$zj79L>90s+`Bc!?>xkDH0`=nEG>EKD3wL<aX*36d>tp}&&X za&%9U3A5NQS{aAB4yk{~2yzqj-IG7!ljqd+wJJ|>EDDc3n5&+P>>7V0QsA^PkWRf# zc{~wuH1M?pOuxS`26j)7S_ase6hg&SeQDWnxF+?)GUg;$7*D296d?I#I92Khqt7JX z66NKXEb3KfsBkhe{;>lZM4>TFpn~T^>(an!j3n0M5dFZdl|AhtX>_UYT0eu60PMpy zKE^klQ@sH-Jd%?M24q>x+>CfsyXsMB*RVRDOWtV<XKNmOO9CWkRgCtZs|LgS<)2+- zC>d<z3ohu^$9T_~b|BGsIdsPP+Hmj<MW{$0*a=VH0+PuIN9f{Uw1v|yDyg%+rFQwo z>7Kb<dRt1n0oeiD`CI8`$PauRe6Tt1!OA_qGXUOOb68=ylCaM33+y)#UMA1};&3DW zE138t1&na%<mLi=gduRSC&F_QdiwcEby4|?T4ib%vYS5e%9-u8$Huj3=ONzi3XCWu zvc6fj;RoZBEu(yf4ch3MpOdpV(s^dpO|EPyZKMYEn)=%_O{%-+0E@c?D#%sdVWS(R zr+rfWAR#c7%G8#pi+XnA443eI+nbO?-<>nd&qS|&tgq!aBgEQdAyJU~u>??Czoq1z zE<@ln9@ntzi4)j$=rlf3RK9W@D;LNxT?p2MG8FDFh!wgV>9+%dQ9?b-r5HVfd>f17 z4v<!04arGyeKMe8+nK2;L*sLfkB-Czpf?_`t!TF9O-t=EhSYKj6qkwm9LsBGo`nu@ zP@Vp8vJoHx+LRfb47z8pNc&I|70(SS<((`|ub0b^!-R}-^5KE>0gOC#XV#oxf9GT_ zHs1^CY|{$RGp~qd3SeQ4n5MRCpH}5sMa3za=zq6IeCKis!cK1F%vdmgG8*FbxDX;e z?a6Zsb^>SL?%A-PP(8Y0`LgCa$UUT9u(PuP^zWg&pWsC}-u=aWUT-ZgwH(sM?sin@ z=UQALhwqw(is6DdA8|hE*nIg9UiMK19+IlMDd5AsI%PP%7Q}UB=g8}qf=t7h!pXG5 zZG2_g1&AJux-xO9{v;jPg#Iagrpu<|kcJw|#x&MpaImk-ve=TRFjzF#(u(GLqQf~W zIyhP64iDNFmo^9bzjnyz^Zy|G(T1baoh+rk_>QY;Gn%A5*&={Jmr@^ZaC)eg^^(A} z?9RzlCPA|MPT<4n?bX?+zbW@7kQ0J}_^Qdkr@eqsi-K>n=DFEms)_GchWs)?Sx#2V z0*T5vn=yT_51I=sf)V@W&QoX`qAwbECx!W5%NXQu-U9h1*^Ey|_o@hO>vJO5=q8g0 z8cxSgGyv8j<6r^3IMXGanTS4bIdQHFC)vV%e-G4D=zYA2GNZjkem)xv7tq2PCA1F$ z>D*&Y9I&%kmD2BQ-eXhLmk04rSo!)|m&a4A^vxH|Uu(cK7>B|cJs+nBX9F3DHrJc6 zFk5yqMcHe&ob?8?x2^3F3S!CIPkViA2xpqiM@wuLE%J?aSOJ}CmN%O8sZBXI7WkIl z>RtXsbgf*!cyeRXyDe-J?=carH002$d@#_5ZKgvLwCT;+cejY<{a6CV-_4g(GqC*n zA9za>UW5^6XZ{uZ=+lC4%+sjUW)#9?di@WJGHzDpi@yr(Or6sr4Lkrz5h2!=Krz6v z*Y104F*m!W@W}hZ1iqWeys;9-_ioO&rB2}OMV7r|{{h#)t?GFVGfb9kn!i>Abt+eN zF&xY9{i_zhx2IRj)ls>dB?mJw=O&G~_v*90fmR<9AR7xFDP9h~^}$L<w+uB-;dfRu z+)7g7bKu)E)ayDqXn~>7)NIrsnrWa6fGRjA5y|G{Si<en8(nknBKL_kh%ZMn9#TB3 z-9n3m9?7U~+qH8rz8~HNvIrW1b1PlC>ox~_8Jy|B=H%l5lv0<FK#wa9MJStnsgY1b z8_1tfGH)6lD5M;Zsb~TOj!*5~^rEite#xSU<IQ<p?_>rFpD&PZWZkiidpqq-@HG-P z*NghcSwCSGZ8ua1ZCJ-(L(C?xHaNvsYuA8POMaS?kZs+mKaH^+mloyaZvEqkHit<h z?zy3OC2xwHwwZAoZk!58oIJ_#TV=QTd!E`3nUOJ?azH{giY&CBs=Z%slWC<+4J~K% zPPSgiDDo~%^Uc+)@Fk?FRUaI;7`m#1tigJ=e{~m?qjQp-xh*^uOIOp(F_h*SLHv3r zJ2+a@cjX1k_jTiDcZ26A*iQ&_?M~f6cNdM~Hm_#vA$rk2qX0QrauSO3^Uq-+!JoO= z8#Tg|EOh$GfE^AP#s4m|77dptX=V`zoxk8<k}lr?aw!7JgiHl*cDWpL-I~x|_N+-? z9#P8<$!tH-&DAY1$K?o*Ls2%BG7+)NjQllL?`{b=K>5njE<N)pcsg$|T=$rB_A<&+ z*kl=N*$dbzCNPDXlD&YE6ca5VWO*Px`%Ea8O<r#ON3ae7KMvg+Ekb3Q;eCkclH1;I zVim&i_tZIkH6tfnLuQce@!Hd_#-b$TtShCbM1o1P`L~5ug?X;NXpr&iISJ72<QZ&n zQ*OdGU26B`p%VA~k{?2q*)%%qN~K2X-YCZw+n50-TEo3&vliQ@>GtiFw-wH0P8BBh z#DB41fLuwOS($$c-z*;R&2kV>SSZ*4fc@q&FI0pU?~1H6w`=5dzw1x8YSlY7rq32o z<T<ZtOxPkA{h(681g?8<GD3ZUi!Tn0wL$@cfD)cN&Jg=!@fnz<e$^##!m}Jb4oFlb zb~4t@DebK0Yr!Lb<MiaOe*(Y$zl+-aOD?9V{|tDZZ3J(pRjq2K7KW;_oFQl3BBg>P zX{YBpkjePwe`|Nb0N6UQ!CQV1^W(oNhazan8Q^ww{>{98ct~EL0!)<uOSqBl0*o}3 zjCuhSAVg}sKm4b{*?-q+#8L;MIfrZhAM1#wkKg>wql9}@A3NX$f8Jfd0nNYFA6;hA zsF<YZLUR6rsNmep|HmF`)E>q3`g}ho0+3k(<fT-M5p{w0@e5*Ty;*&!cB2kZR*KS0 z&%W~0VB!!h&Ho`q2><y~Jc>Ah>ex4-iyC+EsEL@jLLUy?#Pif=W4}|OBxtN^k;C_f z04r4rUeJI#iJzX%Mxv*U+j;B9O{--P_B$OFAk~_eE^<uc7oZdPG#Gs9=vbr-ulX^i zA^K2$t)~?L2It+F(ctuEB)|>v1XD7*{Z9gh!v7}$gXn4qU~MGZIU*<zo?YhN&t<|3 z%T$-9vM&0_!f|JYQm=HjJr2^r9n2xl;a}Bl{ek_u#N}jelf{fv1iUzj8&pR6HqZt^ zZhpsd9S<n|c%kt@#RoBCG63a{V?ZnJhbOW|@!smNsl`0hh;@M1F+SIoL4ZrWjNue1 znIxE~5<8iQ&qG9Ox<T_SMbY}4Z(L-!^mC=khkNolhaw^#$2AEsqA9_X6Whzsc<$Ek zHT$exgj}5NlOHvC52!81j4QCwj|*ET2Qu!v)mgH&2^$OajEl4i3@2|KZd$Jx@#hNS zbc~(^t`@0(80ZJE&3x&NdvbWilUGN0Ps6Qh&QvuBlBCGOu#zo~&E(yM^S=?1!}HF7 zjC#TM<hu_<VJN&v(qG|Vs2*?}Z0)7VfHdjgz|mwb030#mFf>Tao)Ho)#m`o1JO|eZ zildX9iGdrDEq`TaN&FSr&+k{<$d&BI?_edS5sQ<RBrzoF(eFsFYZw1AwrMtL$hD7u zWob|cBtWqGS7gn9K0r89h^xad^OV99GuF>j8L{7!&h4-UQtW|z^5!)_F+4-crxP6i zPm$;Ez%$y*eTPdRd*Hk-@9Bm^4b^0a^?6x(m20okOs#h~nw?Yv4K+|Z=E!pCC1&w| zOC036bmeHY1s&JG+vz3uzw}$JD>Xw3L%*vWVcWI){BAK!ziBpZj%U!;bKLBwh23er zwNj&tnZ9z1`>L*EERMIWB9m^J0;xQCQhj@U_7xewDS;E_vd%m~HZfOo%bmRk7aSDq zoC9T{W?<6kKEd_(fCv~M^Ilo}&a=X@(P^G7YmP5BFx0Gd&eF}<!eXtZGb?)#8&D}v zYddYq5%c|;bjV2ePy^VU<_yI<p`cdJEfifmX>U1NH#LPUEe{izmOUY)G@jhl`&~UQ zv`+T^7f^v}=xGD1%yrZNi*f-)2zo#MPtXrdX(>)L7BzZZ%EqI$7#5sicB>hxy?PXH z0hX1vg;LF0%YGqvvidq4H1>q?8BXW(tbcpn7AuOryGZRze<yr)03}l3bW>tlH}|#G zsTaHBK%N<@Ss34#p;ya}=iHGWJQ9F)$_V%Xo>&UIdrQqW=RjHOW_c8(aaVJ*KaxJ} z(`EK$Z23nr8ecuCiMCve#w)Om#&Oc#OYi-oHNH+=hV4^Tj%vMGsnz^W>3-$Gfui-{ zlQOi^08d+ID3h*$%pkPxl4N3f^RrdeHWw}KEdvU8F%hCVy(ncd(SENwh}(ZQlqRFi z7%wCmp68qjt#_JJTfI%gPf%AFNKsU~@(qbJlQR7<R|!2y;a3LGMw+~#Erna2^ZP}v z+jjY;VI9e1x10$OG~a4zTS7eDBV8}^#r~uY?(J1Tdht-Izg#q&w)nr~0ANW0wKYQG z)<KxJPb^E8!^+51W)aPr8ZiXL$4c2B55qa3V|Hw-Y2O-7ko`ZL5CuRYxAV?d<+x?| zBm>OWzgVGr<{3cru_wTAOT8_?k0N_3G|mlRGx>hMa<jbGOQHbgkY-vP!~jUKjQwMF zA79_<d$56wg_{7LcN+<h!2HBoZTc?YBCkKwJ*UTc`KFU7^9WLUwU;(xqB3H(ewKQ= z29SCw6^1$VO$UUml=aU9wWeD%hnxN=!5Jn^^E*G@o*QAVOGtyV9V^u#ZnG!cJ3G?h zYEl+1-WYkAI5EOYeVMm#elno<=eVAwRGPN2{w_`_&lSNF%ES0F`o8tPj(4&~x5Mej z?H>TTe4DFfP!Q?P$xPyW%c_eqWYf30Q161^`BFXcj=|9XH=us6ZLTS&M&G6aoXZa1 zujxM_fSO`q5OthZT)1Rf=3T+#Am3(ycggEy9hrNie%@@x_xtpUTR?3Hxl?%YegN6K zY_|-}YCr&{CY@d|JGZ&8H(giDqU^Lt#HRpPAJosC9k3$&^mweQ<*i!p?wJAvv~D1u z(bEcoTHj7jF$DDd<6gdYH~x#lo11m(Ibyf|0Mp6ucZ&f@K;4r9+2&gs(Dd0U*dkvy z;K}gB7lmsYAf;em=@lDMv?5;yrcKV=$z|D0VFHl*ZWTPugeuNcW7{;mXm%j$s9N1# zyQtN|G>Kf=WDH0>OtscU8h7fN5g=rqP8#}`jz6(^T5o{#b<}qhmw5GZ(AAN%=s~*{ zic7<`htgXj3V;XW-+&Ddb47}P<1XG+##Ej5cL!LOdSVpjM-5a$LPG6tIoSL~9Y>8w zS9l<VXcH27(^#uTG^VJg@69_D`F0{kklq8ma_9vWlXlY*C(l>m`<No-I9rh3^tTOe z!QtZ1VYx1JfRBHlRdWAyirufQTnP%<9Na#b#bj;bNGyXU%${{#B(3Lg_RVZ7YT-fW zxrMJ9|7t8?;wg7OU#IXpe5={v(;05ua&)*?m^LP0*b>NgafYmOF4%TI_BE)p>f_M7 z+!RVNESrwub<uWa%4z+Y#+_-`oe9VrIjl1HEXA&EUfrU&;j;=!BCy#c1wa4G`HX54 zRp+U@YLR`zbq(L1c9NMp<#m}Dm6=YQu;2UU^nPHaQNUfW3xELU>V<Y(`c}(wb|(d= zbte=K0oB}`y{I_#_Qn%FBA3rl-Sav2E}x9!B6#pNH`2uLJyK1k{5x|i_wO%K0eORV z4M63AE7OhZvxQlHG)xz#5f3JS){gc7N)<i!rLDs_?nW}T{49S<Ei<a5#n0Kf2Ix4> z#3)rgTN<(1VSwtWbR(gxNidZ+30tpVTK(#qg?SOhecQzn(VRSif|UXhfNuD><ZiON z&6^2TpB@2vnYwp2-Iq!F^a*xN%5|xjVGIV-c(&&4TN^tiBU|bBeGBeiBo#|_FDNJ^ zlwv3qF;uWsumv%&F(@~s#T<jF`lvRkHs9b-N~wHQL`3XF3?+RM%j*AyRI<V4xz%n; z`b_#v3h*D_sX)?xQCV4;M~TOML6m;sjDqG+UK~liT_VY(*0Q%H=jWCoOB_fbkAQx` z0lik;SG{&SPG8GctT_ZHi+IN!ZTh_KxqAn1O5cw$Zuqs+hH?czfmLPg5tw(jcaKUQ z-O$>Zm#8&tb`@95CC|isO%hvTP-|<#J#p!~33+JPL#hRDyau$Ke+q9Agoq8az<FHz zhMDuV8&dUYxM}}v5Nobrp90N|9@5+wH{V*prPveKM$4<!h0v!L_Qbq)2}8vwPh_XH zqUb1OOr?QdfxH;|(iLxMZ}ddjBl+u?MVyCH43M*m)Vrl-xr;UfFJsv>9ih{pcH4L1 z^|P}(sEJm|0o70ZY@MfZT^}FzotcAlF64GRW1>4WzLCPH5DAP`Kk-i*1{ryCWE`Il zweQk_!PUCG_}BI~3VwO$8ehU_^l2170YXEM7q6v7dQ}(2ChbiPH!jBbi>;h&B+YT+ z{(zaG+S`xLZ_Vg%8j?K*iM*S>@|ha_?o`E43;St;gKMLV;ylH?L;!R)L|XJFfi~=- zl;!J}-DzO;$p1L6>F;=cK6y7}W$BmGcj4ajvU(bd2_7aEc#kl(@Lr|rJKOHO{G#0* zvp+r^JK3YG4&8F$sg<iY5GlqMErrZ(Af0%<{&b;-STxKW#H0_N!`4voVH4!r+>0GA z-BeKCI~&{84_*G<{*#C>o2kc8Bdl|st<4cZRI}Z@?y8wtS2Oxd?TIdx?sT<}`)x*` zoMvwCTNR^p(-(VG0yg(b+Gd7r!Ca7<^R<^YzS5)}QTq@jZ9)&AOC0OVZ>O=2L?Iu{ zV;wq%YCK!3w0el^S7_%J6P3kZ_D9#9*21`Yx-rQjYE-kD9i9!{1t%B}j>e=?+><@@ zmk%Wd+8||zux1Lu`xyC&0G*adL1Hf=dv!alld5)-d5YU{Q<l&ZczhqHS*5yMh>6&r zk@4tg3U>;bZ{4s1U8;I<l>KKG()9;b);W;ttIntHOHH;<m}m3KOB~%<6Qv=r4}B<} zb6-WR`%y52-{;QqD7ERa&a__>;<UeuaVFwZ?i=@Scnl+Mc$@vY^Kl}3g6cO;t3wqg zR-wVIJpFge<XNDS_A>pr1rn9m%>;lz3!}$OD12(F_87%m_<MlFw8a{m&7oNaigd+L z2`(*elm@T>Zkw>c2BI9Yjr*EN{H*Ao@&Z4jlr#_w2|L%8_<;E~o_6A_;6NJ80-n1H zCCtF~pPHJ!pelik?FZ)jz8T`?{#ujoGd^o)-Gx0%h__ahxj)W4MUDm5I>9F^Z#hM` zoj|I5yiD-i5Y=;X!sv_<(Xiw->Y@2nG_jP3ZPp4ZN2QAu1j&=gbK&V`MX7&pX6>%x z0o6R6tad|I*X&58{5(Q<4Zg64$6hS{l)FY>gws<H2CE(I7@;Q`LtTJrc}3$;c7#Jo z;G?H_mJJ+u<q8N%<n*D9kRp$;cjel-FAWg>>E)V)_%69oq#G5OJW32Jc-O32Y1mB9 zJPw!?YlvV$ytzJwx`Tdy2<rfr%nfw%8k@old&(MeI=(Lzuvw;*h>LG5!^CPEYf3yE z8A=`W*|h9=4}R$5S86s1!k%qO{kJ?8cytYNo!Q9bkQC#P(>_+C?9TV7y-(3Gh{!TD z!m0gJD%q+Q4tBj}&6?9y=XSDz_O)Hl{X=Me;*9Tn{b~htqK}sy#G6HA)wg$}a`xjY zP0>0bE;Dvcsay^fL4v{(YhMxSoCciv>fS4BE8iX6$~C{%Z3s3>7OR;isl7SSr+Cvj zg~!kiF$!d!C~w>-Ug#Nh5@&MKq3wZMsuzW(5B_k^;1O59o}N^<niXsufYiF29+&N1 zL7B*3>$vot*5G}$iMZ!uhS>c)ota4`|8>`qSlfaNezPd-w&|DEmHOiPTAT`(+&#?L z0UJE(^QqwNl>ouVid_~<kHAVS&@H~SrWKeU=8^$4@dnP&i>OCPtoM~B4^6%?3V{hg z+8GczZUDv{T(P8uERErH(PfE@<Pc)ek9b}iDuSr)Kck&TD|L(Ei6`9<(di9FyzO)H zT937$<HIof)v%T2i`|1DomJ(^e{ICyOqU#0c6ZLGL$-|d?7W&Jf$GHy9ocUI{_i2Q zZnPR_OF6_skyCpEUeP>0g*j%xfo!}Ud~Hy)qG5aEE~><T7>})7R)v|cg$?%VgeHvE z8g6@Ovb$Be3OMU18>^Z`WNsFdXvB4%V&Q(YpB~iyQ86r&B4ff86ht-e96|d@#1k6R zi1{^oH>>!oxG8;EAhY^uCtWhWX>$8&+wJBZAC5j+3VY^+T8e4AhOPw~XOA%<q|NZ# zbJMpFBBRRAuLs-TG@WHyk&IEc4Y{A2R$z6f?zA)SWcW-Z{hpT@YSAIWBw-S1+!89} zVCXEHt;%??QyuldKwc(Ndr|=BlAi%7&{poX8V$kEts(<cqVc@yf*K^8o}h&|TuZ-x zFAN?hWhZPf!*_nN+texhE6fjB@}69Oy6b%~!3JDx%ryb=<#RI4idvg%`<Ho61ynT- zs(XobswPbVwM6{8hR?<&O`1gB@-&$Eja3A}tf3x2k5>(YeAd;K=d_%N)lpK+{wc<b zn2ncQTJ()ph8w8+Vp|@o>ErK>*C`C29CHYcr9V@;cC~p0o=63EstO3uCe8NWa?c`7 ziQoUELSK>iy%JIQRpAW;jqd(CS3XNu@ow?(%0VO6Pn{{0Y6H{xB&fqPcB^G;SvPAf z3e=zIk<6Vh4$*n0-en<8Rzz{mn((n7<mDKcV!i%23O!vnBkql9c06qP+N{Am-&SMI zoDuuaS_}i$qJW=C{lot6fUm_rD<Yetz*>xk{y7g2kjB9Yf10S7|Jsn&x%l}~I=xvS zxqptq*~snnc)pM!6i4J~DCRpcosLEUI<Ud=SgotYZ6)d<%HwTCQdjyEto-U{<P;_w zW==2<^?Op>$@IDUnqG;x{eT*Y;JRb+m!oA#70+q(D)T2Tr+OixD!^*&9|u}XxYz#5 z`SE89Bjs7UM&rAARs-(<>ugQOb<BoDF%_NDy_cO}ym{Gq9`R|mSA?iewa9TH9}%P} zLz8Tfzu|mx6>*q~D*3JPc3V^hsoLs2v)H-E`tH2Y>BmWdhXvV&3toEv`S_=c4cXV- zdQ2Sn?ib&P4Qm^^NIQE&j<mpF;`yeOqrSwtNmLc0rIAixYHpxA0aL#p^)?XT`YF}J zA00v-RUn0v)trN0o|9Kl6lG2Pa|rL+aO<X4ib=Y3(+`|4@yOo@Xy=^3Bik$6^Es{f z6Y52?yF#NK56dVg_hnyCM?l)=PX{$|bDE+LT>zJm0Pjk#V-@r;8@ce=*zUUY_`Uz! zl!pVPic1tw^DB@pMLutY62*6xs^W<sjDX)<k++gb*z*<Yi%rqR^6YAuH;ArI<}kz{ zxXp0c@6oD#<)C_Da;^^xXqe+^C4<?*&A28~MHJqFV!i+?Zr^GZ{_3Y=>ZlDHoSU}A zt41&ZM4QuY!uMN?N=4|N+OgpNMOGwonGQ>YMnGkoR!#16Y(3qoHE<5XScr>7K9X32 zWheVtlVfk7=ew`Q_CC7Xf1*m-Zpo6D#}*tp%Ph7~)ui0@8LbB;zJ3EaiJeVZT^*%~ zx&#aLjQS(x=8P^Clk_mcfW*(yR#((O^RIkpWQbJ7{`me~ZY9cM;U#g?`wC%Xchcia zQm>s<?GUN;dzpAHCT#H+OF-I^<T`u$$|NbHh`|`N-Q4V|`$_dRgH{{g8`mdegs*P9 zi4O#vQhZ3pTnA@Ojw-P#hdfi66v@*Mv8Uf4v0CZ0hn3XNiO|)!(3!SG9%L1{nGp-m zv5zwR{)|V~)3xJ#t{zqT@{kwkkNQ0;qGanb^1{A#xbWHF#OT{77Sxx?wK_^P=i8yr z109~O+3SWR|9bF(C^5W&sS_o!;(usy`6LKvc==H?>@{&nvp;_DGcwcl?_pQMIPDQ% z0>A-LMX%-GiND;9s~DDb5#vR?s!Qae5I#l8dv+CNbzz>tN<Mjp`K_s{mT*uuVH>k6 zMU{!icc&qeaUBdAnyLj#(MXh3*>SsE2Wn4q5>6L0AM_f#&?q+64Gc`{gwy8_PmP62 zMLGnPW)C*t=+B&*__qV+{wn@YU9|o5bTOG)KHPo4_c|L+*DsxRQIlrXST0qmY(~GS zn5bjGU^?u)_aBJG*L0YwcgU<UO=r_&)NTB9mRj*qyEetlhs7+`TBoi9_hfO>bC5{Q zpkz~aO3gSj(R8ucx@?aBwx=$?;Z7>XEiZduA@sE;WTiOMxP3oq#!{aA4F7r0GIF|* ziB!8>>^n)UG-*@7bX>i0vei-dluKsRbmFrsw$WsSqFvbq5>O<&SE;?-YM-Jyg1HQu z9InN9jA$U7%^{m}In??_jH={_ig)sS`syw&GwQ1x<%Z5YK-!SzZ(TK&T9@QJ@`+vb zDwMCrc$J`aq43KGRW*~9H<;tUhr7TU`77L|&nVuvC8LyrNk<8oaeQBvu-VeKm)r)F zb_HzuWS^tQ=6T(pzO}(Om8CQ?s^QY==OV1Xe2Lnfu!QW88T3z>bOjHS_#s|c4`I@C zU?clgVKMWf1NN0wC6w-be;y*ZzS<jZR@}->9(o&L<-7Cc`q?W!N4uk9;g`0-<{<_L zG)e+l^z*z8>{@AEL8ooczhzKv8!u;|qUOVP5ctEc%;i(*+#6sqsv?tapU%Zjo9gEc z6%jG%47)^@gTJ&@77)v)aL=>%Jy*Z|Sc7yBZ;_z_30^pGGVWJx5<{Rb{SZl!??Xs7 z@?K7STBG8?04Gd@FsAjqV$vU$Q*vt~!bAD$onDrhv#el+Z-u%6)yBJlD<h1zJ{dVo z2jOiqC=X3yKd!+S_i!jZyx}Qn9p(=M+&MM2ug76u4RCwK4>_<xClE<3BX`Di7&7=Q z@Xp`oXMH<>%0(7k1RAS*;TI7)%0#heZ>Gr<MLwe=Z=!x(X0uqKWwJ&@Zi5mr!BE=C zWEvUs&_wkWFo=~F>|A(oSJfWwHJyLxy;1`j)ZSt_`9Bs}T8q&~Vn|R;t9kXw_eG#T z;%j$(Uugo@9m1#GVMni{gB<VO7JuuQ=FBTTf<Bd&>=6iX(fnka|Cr4h{bAOOaNu{a z<l@4AxI;K_hkB;oJ2Bu6_XJMCexgf%?lKX}Lfwzri7^m_s{+j&d8zTMH=c$binw5e zQX$i}eOx90{O^MR!kw{zYszLUuoAOt3J%%`yu&Q%EPPFHOnPBFtce%E%^p`{GxK0P zH)S5ol&5|Opnz9{cazlt!l);1HXm5vfBnG&zIsPa_~}-O3AGAo)&9Tx)n-Qk$BDV{ z5DL5r^2ZJgVEmb+F{=@>3~;M&E}h}m|NZ!NPyt?${d6z|+$VbM3ry7pW0fF;5viI~ zn+ZMXVc|aXtNs5U_5U`iVT1lNK@-%JJ~tP!9L5{Tx`_OIa<3X(j)QL;?n-lq{(Pg9 zP2`oUURwJ*N2H^`q@&j0EPsCJ*?bRdmdi{B1>R2O&fSeut1827-kEzS`X%ftGxl<3 zUa87b*L``)Uyxa!^_Mizm)ShdQVH0sMdJ;iB1qru;z9IKCS4)`n!k-IkfCS9qkErc z#S#B0s-VIO@0)-TZ2qp%SHI5tEX?b=*W+?^h0FbD`-Rd)>sO^*8LF^MQo)amQxO8? zQs%4gkKhNfVL{ttJ~i~nzmva004y~Jn_ez`_^QFkGWe<c<Jo8OWWI|{5_~8q?+sst z`eGWn3qfZeztpda{#fdJ*cI>VyOi<v>YKU)6#P2-bl$`phq-97ce2mX#@a#0p8M<5 zxvPKru^9EdwC4<X!iJo*f%pN-aT7)*c|1AcczPu2tp~*LH2U?L9gI_(wEXcn)o&%v zE1f7kGg=-C=%Gjd>*?U<sDnu!mpuIG;DJ;1y++SAt2|;gqZ&i(I%}U~n^14AZ8P!F z=VG>_<1!l-7Xx)ai7~sY*dO*ZJRSy_fiAZ1dubu@TA-%ikb8(eXRdzrjuxUdt3m0S zcWbmx`%#G32gAafmTSVIsiqR_a)oXPf3lp>@q8rFEF*;w!JzUst0c@PhtjytY#m%l z!BS)wiaYEft3#^kxbbXjy~*eF<+j?UcYk{M7cA2P#1joJKv(~;2}7lU+3(Gp+R%Y7 zjwMXsW#*ryerfif*=x6xlmH;0jI877U$5i~>M?K3yO{RS)r$;i-P_eDv|Cr7(M<xh z5@l4;2U&Tu0rE|GBK5AHhV;+-%BRSehZZw|};ZX`FzKLbFLN8!&QD?|Vn%G%D zc}Xl044$EcvT72MY#yCcJH74yX@Z*T^xL^V;T%W1*@a{%-i2iN1$C}2zpvO8TXFWB ztVXRW7PX{hbKl|GP9Cb`-hc`F+3wMwWnr%D(nI2RZk@X$?UXd}9ije}nIF6qu&5-- zHJ$hwD?T~VE!i)FCAmCT+-x2vNjgZJe$whQwY^8rZG^TuX~QwFq~UbP^*dvT*<R_* ze8yxuS~oZ6gj&>)=gT<t@c0Vo9W*h5#Z#nEn>@Z8GL+UB0ay$4y9Q}7`k7o^Kbu#} z&mQa!{IvT9w>HLGmj>_w4rK<;Orsax27JGDJ9p!;nu2RUj6qU~alz@=GqO**)bBUg z*K4z>ay82hU8hS0%Jf^aOU(x)7M=uANmhM2>V_~{Cpp;F=I@zgJBIMO9VLT^%c5;i zB5T7`&i5B$q6z~00gvkU>=LE#^2+!R<3eqRpQJv5h-nQTX_Qjmdmu0A*3b5Dw2+7N zFc~mAzQ<%$U6({)aht?^8pxP%jauy=x5!x^oRY3#C-`Y0HQpC1IVZTf_PhML7M5Pr zSLt#;#x0jx6!4kV+un`qZtk&iUX(o}opy7WoQi>4X9S5KOMUmXZ!CcG{MK=2cCNVm z0ejS6PKU=wk3prKKwl`@7Zc9$E@|d-9Ro4H*AGzLKD3cyRn4|?iihX0x5=HS9!+|q zzt~+lSMpr&$m4K)IJlqt@7g5wGk*(Mhg82~hlgn65wKK|I>-^bFo4+4=Xaf^Df%AG zaOtmoXe;gBdz={O0ZZ$S(shOc6GT?Fk8^-OT4L^Th<I3l;q<)g8QqAKYDq~@l<IN5 zrD2I_xZIKLx>l3EMsTOli)`(bdv8)V+k1Nba&3m|c&ozdM|Y6{%*J2BiFHm(d!JeW zK7KUfvsYS-{a2#ZpZ2=&$FoPCy03%WcFQz<UoRuMP!&(c@DB8%S9UXrfnO$iB)gsQ zGWM5ywkh``;cm|rl-@($2Fh+t=%L^I0kx>-w6$D8n%{@sx@)rf7T0|UDA|&>YccW8 z)!y`D;tWj(-e+nRG}l(}Y54H_cV0&cTv>uLSQr0-d3BeQl~VCotpI~05Had##YF7P z>O}4?QeaFd<-2#8)Gw5!F>bhi)J&39P&4fvP*0k*X=;2!%lNnP{G(?gB)~2o>r0g< z2aW*0kMQ>P3TouReamGAXWJ`WRi>S67P)kO&)cqj8Wi0?x?0+3vYPc1MW|G^Oh!1v zAlvkiGap7{&l<7XZ+cfjl=3l3nfGVMQ7mFq_@4m;VZ33ERM!kK+OJyeodbP!S%7(+ z(}b2}VQFtP4s-Ss^$tIfWk_{=v+sRI4d_o|WN)y@qezp@P1vWqWIv@wMM9b!7SpEp zVxJwb`M?ciiK^D4<F41q`yl5dzdP1MDUWq6%Rsyai(vW%bE}Zj49|9G{;HFEY7Cpg zMldC6vRx;ZNuFZTU0tK#=+*6XW^Dq|>(5FCQ|wJvsO}}sqD3U*)bk(>)>7T+vFf!M zJUw(bsLJPN+qUGn^<%RHn4O6Y{S1q9E<QmNeBC{Km?Z$%_(j)d&V$z!4ELJI{`j3~ zxFNB=Iv-33N~km-DA!qxeJB0itoNzuOopeR$G1!~E2AeF-M3W}p+$8bpOykK&TmSP z^17qLntVEccUBos5YU&A>S;lAMw+p=tcf(`zEvnzSrQd3>?(4|=-gdz=#N)kH8X(L zz6B9wJfr1N?|sU+H}x6R1&J-nSu0_e&YSpUaWpSoK5MJb;azI2SNmITlzD}MZe!uS zZvWdCv$jWnC>1tLBalLJF)vZuSJ`~hs3%xc1QtOj0!1@bP7T_4^29};>o?Qgm09($ zjqpZjle;jtDq~h)tCyt!*!c~)?>~?ruzGqP0qchOQF;_5eBHc;@6BL5QU&6JGK1h` zL)1Pr+!D{+6a*vD+V|J_xB{hWiW^N_1f4G!g6!$pdR6&YcwCDWyX2UJy~-W<=FHDU zoCH=faXiZXT_@OF$|5~otz#?#nk0j+hVpdM^IeCcCm%7yDGNYN@K8}B`1KECzVdPC z0lqVBkW@l_^T<{AQ$`F#vKOgFB_y2cGX4lU+FYL5)>>_&a!die(8<TTCmN&35K{mt z$t&G_FaBBp%|<ujV@jGs5&No$V<6`O4Mot@Ri4KR4M1NUc5T0UUu}Vw-dL-;cyIJG z1^w+gtdR9Pceu3K?2@@Q2_s}^u93`76y|8xglz*_UFZ7unSTQJaQVJpV#5!vr<8C% z*@E53o(=#-ex9N^G+MIVkFcN$i>v+GcNv_0rahd$21g1GWSE58F>ujo-G*)#y>ssB zyAvrpQl4&nK<_~dSUS@?y;7=`<Y^xK$v3W<*cFu!^c&ZNY@^I7lhYp`ImkLZbTGIy z>_gWqdfs22Y|Bah?pbc5Vf)91z*-Wza|P1pXQ>Se%{@EMIqw^N56Wsl8X{&?dz{{q z4b4~Vpt9bnP@N!$I7UR)lhvVlnYUWvs4^#m{~!c#w4D|19K3MjPk0D+VgcVKIhW+) zh^smiYv)v^RHo$Bh|c?GuijP-8C?gf9xY#Ftqjl)vpG|JYb<u7N=~^0k0AnG4Q`nS z+iQ?UIHy5<tbLKcen4E}MU_PpR44d15*!wtYRH9^0@`GYcX|-_k)zJln5rZ3nSXIG z=>CXBmx*%pD*2c`H$*sX()nAEo5upArkne@Hx*LU>|#6bhXU4lyPeT**}S<MwrFB7 zsAa2ZeJRCM&8L(8nY*6$eWP%BFpjR)pyu_)|I>WEUWva;udP)EL3Hyz2N8HoR(ySC zqa&*QkqkyzCExV^0h){Sp9%22iwW}<rWCwE<i7lP6kl|bV-PXbb~MM0x}I#3M)A$U zG*H0IAv==fN;EixZ5$#eya7t=k&NBBi|Kcq`R4H8>53jKm*p40T5<H2j4sA{mSY8x z+RLZ%OB~T6%CGs(U_SU`m6aX0HGq52gnrz1d=4Bil30A_p~$#TodVeLtAg6}O5?HV zwWm1dWe0Hw2Do13xz6n?mFlqTMm1QSA3)=B`(vfEd%|pN($h(i-xzgrl|@zG?(F(Z zx;NWqfQP1D%Xqcc52_|`+juru{`B~d>7@j=H-X`?;)CfWg`3`ALOmx;yqI884l+G# zk*b@Za9C;NqO}gFx9aFLRr0cFi2Qpbtsz*oHOC-$cr$W`v6;iS7!<6>sHcTREn8eo zn!)1^E=tQ={Q~#`@1Q&JW5yXRt|62X$)ja(AK+7Wth*K)?UfG>*X^|-Jyw^8&C+f{ zEX&X<*F7TN&$KNcv3G9;sceQ4dGsH<SAHaG^t`U~syw0?YL&fbJ_1UoTvb+l&;}Ss zmAfCc@t2Dzk;2c;dGa?+S0W4ze#JkV?mo7Jw&}GCMY;5Zz^JC;EJ)I69Wcnr{1=7i zzZG_f;-#=e5&tf^M$ByGsUPU4Xn&3+>HLO&<}Gj2PT!o%@h97|pMQD>M60|uKBVux z(iG5yhfoh6qYU`=&_{@T_$xaN_-pGe@sE*X5vFczpG%(tFd=jiR~gj>LtqCebLKQ5 z(rU~MKvhV$zKsTo7Es@sJ&ri`(NwDA1t`w|yF;+GDmiK(4|Y%cd)3;E_+mXyB}2r$ zh!5jVU0&%XuUzswZn2K94PkisQs;|+#jQ6ANhwVNP*6|6?tgB)tL|c*5eHR{-Klg@ z>byJOznSF-c2231h)7J|#Ucd1nN{w41Y3@l=x~kemLvtEk-y$RW*)>2)%ulc;ZndR z`TToW>IOgTaC}CY#Yw}i*@x^`C&59#Gk&RTdd9U8bUr<8;&GuAWzI+I`emsPGPmXS zTgCLSf8zkzA@C7c1{c8n4xlL{a70R!U<Mciv}d;hnO0{uYNEWn$uDeUlJjYxsSA{m zL_TGf!XH45=~q2#6UNyIBocnwq%VY9e+Q~7PnAV(vzkRs(vMnD>#23kz7SlC%afpy zjOE8I<3z0KJN-4UKxFqKu+i|P;73tY8F5+xV~3}-KNGVkh(ZpIIIC*)K6ootFpeHf zNPYT-$CxpAcGrY4Hm}1s5!(IY;@~UBzn+}b6mHz{_r2c3&-WfmxXqIk_>w_{(@FOw zm?ST&gWgFcX~XewR}!Dw*XCQOp&~)U`rA+o&>&tcU{mNTe8OedxBR@ZAdN+nuu<u+ z@kMXa@;4kA<~U2$!dQbP01tb&^r`Mb#{;GL;9iUg0q#YO!G`byV)X+apBJu&Hg@)i zcD|@t#`;jBi%>M1n;z+n>qLuDOWlpcTqDWu0N%o{U3da3OV2EiyEH}~_%TCEY4a28 z$NoX!+fT~jFD{yCkBx8tAi7Ha^<pS7RN-8>ROI;sGV=!un3NW)&54*k)y@|fFAwe| z;i3A`nlH3KCMSoLQsm<|)*f%x8KIoj#<9Z}4c9`WbQk;AV}}i|eUWvPqlo9JXA%rd zV~Kk%etdZJue0N^>=%E0&L<DU#|PYRWg(R(-9ZNz{QZd`)xV791GjS{4Sciy|M(^| z5^I5m#=jr=VSI!-zok9ws}*8kJ{;_Z(ub4%Q<UjS;jYMkK0xq_@NizCz-i)u@O|q6 zZ18tQL!$rfu1}FzZJ)p|3k`tVw@y(v4wL7m!-W<9LLC0t(C?BRUOuc{9r#hmP}UU3 zrOf2?BuMq&P9gjsKb|rL<JUj_-(U8BdIfB71IokfqQgfW1rItCPm|1*)AjFY=ixPe z1mT}Q3{8(HeB^+(AStX!tken`|IjZFg8|(3>myy3Mzn{Y+lTM0bIAPmH{j=byQDau ze&;)E?ujDNA2%DuHH2BIGYp=V`2GSgf_~$ElBf!LMzQpcgz(L^jR;sQjkX*dftmbY z?45;IRcrU}j|vKcQc?oaASodr4N9l<rliYFBV7^-lF|(d2m+h#?k)l8ly0OO?z1@h zzQ23N9rrJ|V{pcB?}OX5pIUQ%=VyKwZ@@CIJPAJ)sGom&N2=~;|LXv?_IS{^$2>P= z?-cl-ES0+pN@dd~AD9v=c<8>LTHWR)CTk8JWO;J2ADYPRA)=V!@lugLj#JA0Hw_36 zE5IU9pL*lj2Wp>Wme5z`8s(ot(8Ydu)pVZ=^#Gys=Bn;ACP=fAx`|id+$j%{ET}Sp zPQdW2c30Kc!kKj)Ng$p@29k{cEJ77mv=ZbZJ^&ng?_s3dS8`EH_wfzF=WmxMyRB`~ zp;{=)N{yP5sXQ*X-$?FwOadgKCLXSc^ib*u1!+#nCXr8+)zqq5j>jYqmpfmglMH{W z6sp(}G>}YGpVWQB9+HG~8`3O=G_gW%n4$~$_ug^Wg!fu;=Dl?}Yntzci@2jW5(`j2 z9if8VUE6Lit4Au<*72Z8h<+B?MOyL|Gm|qIfCBk5s8;7dX6F3nX=3!?$Om1sqR#{` z5Yj#LNFGalWaK`W{p7SekK7Gk9ey$G?pNy~@z0do<_`}-tU*{dry1>zN<Qg7l=GNi zZoNU}vshPyDdZ;na(e|X$?C;j_h&Z3Eb!ASqHrL2p=#E!u8uN%rkVTqk^mjjU9@KK zO}JeE64<<aX~f#*Kj|*#I5tgs@Zt7^JQ>SmY>b!M4dD2un!_xxgg#oUYio4ax(Ket zeHRhCZ6FkQ-B}sYZdg;~-|8u(cf-&)g6(7%^Jwzs)Yya)#v{ZxHFg^{mTDj~YVo-% z4)PtAP)UbL$q@^Xc_s=n<l`}YUBHMF)~a_A4Y?{3SIl(RC|Y(Cn7X$*ERA0Rm-)yz zj-1CP9?9_2=lq!ZeE*qbfU?;;u7aZ%HYZ=Gq{2mxZxwc{H{RP`Gw@qVwH@fn)~X_2 zAdEA4fMJ%do%`~m?8vmC2WL4HJd9AEaoa3@xo|!W=R0re1_47v85}Dq_R5_#hPV70 z&;QobetNtR(s$c?;^wq1zETmKNrwIV;ZtWDRayd$V#!WCwL{KpM?k@hv*V)$@#MN~ zmE+D-bhF?!6$qlD%=>G<7~HBnFSNkzdE>SvdkV5j+>O9+mBR?mX7)0ZnR-uoInqnn za2o0D=OK@l4?`yy`oACY=JwT37D<xD*XrbiExPVoIBRXCWDc5?9Qs9c9Hp5Bjp<~1 z3NmtCUA(gE_{`x)B@^4B<>1m%$DP%-8~b=?D2}are4&2hGL~V7A-=qluBen_3sO6P z(uRXTUA|o|wJ#UTx%~YY;Ua$Z+NK+`7~S;;Nn_8Zy5kJ0PK~DLm5n4OE!sItTG33? zlVaCnzIUA$hJ94y73N;=9wso)nFqwFDbKco65aQD?)vq592%OG!zAgi^E%`YM^2U6 z4=0R!58WcQ&k>jUNuk#cLV0iMbdR~vEj$EIil<aO@qZt8oh8R|Cm6h|X<k+zNxQu2 zPaRKG@C?PuVbR>iw^u4@<YN(p?HDQVM6oNd6}j6kk6~3h9*3l>ofBIYXH1=*N8>fl zy*!r*(aM-uvuPeYEqc3c2!%s6S_R*SIR8E2O~66@_pJ1vhEDD6Lwi>TkYEYA??=gy zPH9&cyOkTIo&8hwJ%x3?`A~ze()lu8hVX~zCjnQB{x(0^V7#Qo+`()0YsMr_jXJGb zk+-GW^rZ5r#qddoA1}B;u&vGt_s)Gn+b3XMmZw7psGiT0>pq>2P~aA4p@r^<Ty+*% z0d2l^tuhjEw}Y(v+n147weHRI>H-Fd+JgOcpXFO8A5(Tdk=LtkAOG}h*&E|f6i0k@ zOlicVMmsP2r{$3Bd~0BS4&_29#_4YD#KZ_P7di9KjaF%f<OO>jpwyJ}&!2qNIdF-$ zZ$bVI=d!Lb?2Kw3c}Uy$Lf@Pyx@Z6CM?yw#D17yti@zr{heF6BxkKLaY6EtueJRZ6 zvWD-_bblVh9<;U(8g)_ds~~bdXSd4Ym!%D_cT8XUCzK<!jmz;B{A@TjIAlEM7Rrln zv*u|gv4xil>3zZ%OokNZY4qdT3|~XXQ9Lfa+5RV3^fCP@g|5m=VixH#y&oOyT^&7` z?3&NUx5i&mcYm5-EZX0<$7*A-D~n#8I+Ht@pY}c}e>?MN{GyYhy3pWq0weCJd`9<| z4gOM=jiK)W#?NgIIeEX%Ogav1)&i>;S)Sn7>~`asN~n4~V^w}%gh;7CZ~`Ow%Lg;1 zmF<B8&1Fy0j$<89%kt%Gr5%*BWX^(|=`)_dQM{y`g5pZ`W6yBI&PBgmow0bmYd}Z6 zq51f6tafK^(qRrJnMjpoxv?$KI9t%K=0*i%gP4y=XA{ZAPJ<|yT;CL%RW)9eYg^mu z&;P`AuMhZXF*_uQrV1T$@9n_dH_=Xy>E^y}3|{E(ue-2t$s@DB3&!Aat7te|I-K8G z%M4c1?+v?jT?}@VlD=wcMaCmNd3MHiRg*3FS;wjLi=~Nx`EHs>6i+y<&aLt5%ZH~f z)3!{JLjN@1qerB9Oz$Uc`M7qSS=p=dO&t#%Ju?D$oyMFnX#om%(KYR<{u0Hsc>X7} z(pC(Sbm^|Dx3={m6|(n|{0erP7?4Y&?j-X{y8jS<>isnAEbHCg!cp&8^Rx2DYSoPk zlqGC?ty*OCLQY-DwD~pKW!X<?VGZl)Kv_}z>S3pCB2cv393~k1VCRQ~r<0E-C{o}a zGsM$reSS@5P1uM&{>U$@!V*RVGie+|Bb_C<2Rn!_XrMt7dLJif{3VzJDfEt@8yx3^ zETD?$fOMf@jNBYRb4?5)G%$qy!-TI$pIg`*l|9ExPnFibh?K18INvE>E!rB2+5>qm zPlsz(w*3-5$47qNx^OGbyfE;q&uUa9lKW1A%5=SP6RXb$Doz^dGmN|C!udp{m_p1o z%2)=44#Zd*{%O4BiiKx6##^vz=V(Fx_~0!}IRBn+n2AnLW^NG~CwHBoNhoO)5E<b# zfOiZ_?CM%CDZ;#X`3Y5wFXZW}r{Ruo{tVORuht0hw+xlx0cZ%(LO=3s-F!#VbBT^4 z5f=8EBx;o3N7_LWEO*TPSHaGVsL#91n$1Bokz22~53YVN2>;b7IO0es$pVBn$=YfY zqthOnDx%GA_&U!Rk!R^B)qc(@?33bsess%D{L{80Oz`I~(&8uI+h&n3razjZF)*X- z#<mY-*D04)Wor!gCuCoX4J%(`ElqcozO&li#(~8=uox8zv|6_EadzB2P6Oz(9MdS` zRh|1rhOe@#xMstH<nvF5*#A~>UxEcn(b61v6Z<kiv2RrO`e{12;g@{q_dT7k=;O0E zRwFX(>!?#K5K%w<VkdZM;|sUWAs(swiYEe;%FKiW3{8J7OdA=(;nGUHsow)Cg~HNG zOa@iugSYC=`(bbGhI|;v1Uh~X1czEF>3aPmjRY$h##l4RZdFO79tu^_eXsH(=xc4s z(|ghcln3KH#feV5_-+SBZ((a;!;v4>04?1?MjfCVf|>UFmzh9O(k>UC;@s?aY0pU% zy(o?fS#72&7;-U2I|eRnguCtP>}{{2I-7kY-hghQ*KKKL6jp8V{=x0+3_F8eufj@( zSmB^M@P)nmj!Y<;D07yGqK8){nwnkQ8eXyPEbp$~Qj75y$=TvI_e5M)cnmPAS6hG7 z$I71nqQWwSb8w5R5QrW_7Z&)8w5p9?c1l9G5CeSbng|>~qnW#wN}x{S6o|nX{Ns)J zmxIt%3?{!L9C=L1o1V{<!xP8|1On^@6E;HHJ3zcQ%Y<ST6In_ulA<i1F&x_N;+U_d zy3=mMP&erL3dkivFcf&z$Mv0{Qm1ofF{)T$nFmRGVZE}CDW1>kRTdxT8a!WF*8bRS zA%8eSG&=jI_Vsg~SW9l9-P2IJ%45L1R6J9<_F?Tei8!gMgjEl}-mzTqjVn?f&*0t% zr$1(PjLbtvs^}O+$zP^jKuUaurZn%@foXT|^Izfo=@#xxPQrY|U$wT#wI0S+GQM=Q zoq`9PCJ1g7PiF*(-=sp3Al8(?D*QK*;eigk>2NWNm&t%`Z1GNrEoD?<4lej;#${Tw zs64_tGFa~{%XA^5aUOid@|?6RME{ZRV-(ILF}aA9H!i~?$lg`22w1%zFj`iPFf*x+ z(hU))qChnbRhi3-XRO0=$i+mX%7Ud*aMpfS*X3kS?U}^N=5Zbs+L=D&JbR_64P33_ z8MW);=x;;qrW<tZR{LRFLG^`Q>jHA9{*~7_m;<B;e%&<_+#X2B6nwtRn(za=d+XVf z>g2hP)Lu=|-OVp8O>ile^*8y6N+WHegCqkDEi4RbnV*Vc;d)6K1Yg_zUy3PLI<ABe zl=lE??sZ=14!oObL!%PAhi^Qz!1#_JO4xA12Z`+qWb<u(hdcIr%_w&ZtXaJa%$7SR z;$3^NY$nJQDUNUul)1OV)T;rg1rOx2!w6Z{P)26gsnr}|{Y6}OKbWYa<*ye2)}5U) z+sY5WQNs*=Y~u1Pf1&jz^i`R{=;lF0{aGEgnroh2XTQS;%#uIV3(s-r4Xc(P2Ud>O zbQ||iu3wiXw(@#Awnp;vmUWZ}Pan)`FC@c9c&cXf*zq<7hOfAGau#Myi|c!4@!>iz zLQPX*%L=?dEYawj%?mzlD6I#Y=CW<NI4moPZx0?h!Zg@74)SWU5hy`}$3)}YU>N@r z*Er?OPV}+AUg44=M%X|KH1Xkw#){L7<GOxl?3Q{$2a?yVmp*jMKIQEdm1P8m6=k+Z zBI6})Uu9X?@g%I9U)7Md1FOnh(xN_u+1xgFUS{p@m9bGc*ek~Temts^9+Pdvt(6V& zNUGFWko@?RAv2p3josv;SA1@jIz<|Ys+(2ks;DpbBgqVr*irDe80sxX+r}|qy2K|o zF*2*ua@Bl11{U-F0AC#`Y1gHDy%~m<@4Vc~o|$en^p@u#f&SQ6D6J$bA{@=JNO4cS zGtunwQ1x9##my93&n9xzgxc;*=~oTk{*^#IGrBvAO)aON<H*PAeQRnRWTSA$@rcsa z;1mDZ@4me{b+cx6xz9;JN#j-@@6$j*f7d(4Ok6P*?QzX6W-;yG?;F>-l(Uebq&Qi5 zDpO!CFT(&TPWTJOvddbn49{w|)gOu7WJe3OV$Abq;Y7YJno5$i_#yS69c3){5qzB_ z_nzlWsW#2=jv-Eu{yd2TC!N~)tO`G)34mtxa25wLtIu2bNVmit-lkq(KbSJC4vNoX zj3-r}kK@E5(L241_5#P1J+-Vx43c|30#xXg_rIH)7mApZN<*wO>M(nPs)-R4pJn?r zFs0G#tvRIQmQnLT6|1~$U~7<X2<9R4rn(vL`AnRygNV@{@$iy@8w-pM)-6b{xv*6q zmb6`oQk>;{9%hN4o>SHMJ$;<}>c<EJo#MBVUBqYVem5Cj55ol-6yRZAD~VtYkukC6 z@}2w&RAbCiP<_yGNaUg~d_$-5(pSF=cYwvVZ31@JSzm8?^~isP{PP*84ZQmd9`Bn< zmNO3+_Ld5x_)x8dwoIZdPc2mA<W2D*{HaS_brf>kXql(}qIacvP1G-~r=wWj!;=wA z^X=qt>ify3Q{DONjBz{<zZ8834J}3fWuVMhmIr|#mR6VdnBxk@0<)o{7|kG_yCn0X zz@j+B{kThW=lhAR`aA4kWCPOHi+})KW0CFS?9rHP*$MT`?Zeq9U2JGt%6u+&FrbO4 zWpC`6cGpi1-TZnIN>QNd6UfE*2~sh&LJmw@4tBeSr&)1q(z%Mkp(>OJflhtQ{IqhQ zuHpyXqMHM)lBz)I1!s!SaLlkX;@Td5{Hw1PH!dhDxVT}K%StPzk3)K$KuBBs+N2V> zu}5Ky_0|`?Z@LylmJL|S;O+{;kwxDkTiW1^0Q$S=Z>p%(xmm_<eMsz7yh07u@Mwv5 zA2F0+DdhX9SUs<;1Rm$AI(>I!JbAIDT<iOA>LFH}#vi;KrG0p!Es-cj#zfW2fBHjE zXq2F7uC`JyKf$r+Qk5UZeDVYOF%e2T5^>5{bpsH-M_l?Jg)9MSc;XO)S*MnkNx8|) zvYqCvoI&p&e17#w`DK~#;BGvB4g#Jt7Ca9zmRCu=@jtMM?jQQs-YZu@K?Rvx$W0hV z8T94)h6XQw?)r@0I}S??tzJ!`eYUwqcK1jdou(1pj*~cTgM>3N7c@YCK_AC&qaM0& zFiDABp}2*L9X2P(GG`@jsM@eBU}uQ_XUWS01$EySZxTYtg*GA5-T{#|TKxNiZrRRc zE}M}Mz9t{bzRoteF6bc9+@7gpB5$Ty!!WTDG-uU=QT<vNXEo*b#di}SAJPRXw1v6M z+!+?BJAGE<VI*BfN(R55gjczA+*=Y%&vTQU^zU67s2lWZt&^3pu>vQ$zJc^b?x_tf z@Xkn<q~~8P&yzI@K1s0n6tI_QyE8=#J@3w8+niS1tk*g$>{S_|j}_MP=~#YRKrY!l zFG)!yoyT`u?e(LD`2ec>ACae0X|l#NI-=K}R;?xwLk(KlSc`n{ycPSv1bXAg;n=zt zZI^>nZ;MtMr?KLmljNsOk8Y@~?(+0CM_i)VJSc3+{$AlWf`pB8Rx-MemQ*FdX4Gx= z(a7-~HTst@TKIWkz-+l_>rQbV*8Hw4=}&Ew&!teKLyCd7pFDnl7zbL$+HNIftmu)P z)4pZgi9^9Zk9j<EnMUF{2;2Y+kE7Hd*Ri!#y^7X|ZiBV-_31`~GFBC8l}n7QyCHKN z%+Ry0S=R<Us4^+?RvcU#aO_|bu{wfumpcU-33PAq$hNw<<(i_JWos?%hCETWTnO>3 z*A5<SsSv@wXWaQt^wS|Y&HBKE+KGLzzlm_Gq3tuhv2nENMjlHJ+xJ`WTO~V0X-mwv z26~5}*XB%-FxowTF5ik*PiH*FQZXtvgEz-u?Jh3AO1V@%UlZHHjC5Zis9?$DHvjk% zE6LoIb9=g^a$ANv>cf8FVrX~Vf;bgINoiUh0&(p6rfk<c$e9%9uCD0MHP$`ZFXHMD z;yOL8FxW5W)BCtX{gWWGV?tT(NzmcRsU#B%J&koR*89jxa=rCFNNLv@lX2!z4?YFH zzPEZS0c`XGfYMXEUf5dSsmaq~)vNT<m+g4(q1WquTMS@Sh}_KIg@Go{6tQ2KwuOn= z6L&6;;I)E@>gaLl=uXVOF<7J?kNY_Get>2%&N@|7iqqj_hS!KlJOTdkP{82hR-o$h zb%diDgG$ObQ659qcO{4+jPqkswWjxG3)>U=wX`2Tu1?;MqJKsgg~qPM5EhKp3YMU+ z*s?n8fqwpqA}ZTweoV{sxy8H>aX>g!+lempZ6+XhdBfwVVRHNvNAX%jl%=`K0l@G5 z1M9ZI0=4e=wYWV9o7O~XLzexvf#GWNb<kaSar!XywWnt2WtEQR%*(`gDq35fYKI}- zglG1SDXZ?&6#-Z0dh?-;*w0Hk(-_PA*Gw)$$@|*Ouc8}yaur5z^~L|I$R~o}Wcd*q z!wq_&iUT5*oS4zePLKk`y`RiOA~HPk`G7V-j`rlfz+CsRB}J<;UNO*E*LWfh2UUJt zzrn#XMWdaDT^dXc%alndg<|l)n1x|Njm2cE4MmJb@wBq+?!Fuod+H0N9JlEa(Z+yv zAKuN-Z8hxbMh=t$^c6KA)nucF((;!fW`mkDZ&<jc*GEb@FITh;6j(R1J5JKD80&SZ z9TZWY%JW6ZA&+{zssqyBT$^&Wg(FE!&jAKJDT>}_*^ATxIOZQ>`u{Gx0@S)0lWvX8 z+U>~`%y(CJaNm0#`M*AuRv#9@J0RWu2y4fy_3LiruQu8^i>h)(5Hg+vh1jb2Wsv|r zp-7kowz|l#qBTV9r57AP@s7uo+di)=Z2Lrr?)jSl{gsX*-p^DI*CG+uKy;aLwcc!} zZz7cmn+OrhJwxO4vZbY_{rGK;FP|%Z%YJ4Eyj*D$^-&PMBq1i+169XuSgh#5jhyp^ z$HY*FM0Yv}YiD_9y;js%5q2V!=*wURnEtmL%Iz{~F18P^3G{m2X8F_1)CIPQPR5V3 zraJZ^U2-vpwP`<3t<}w#&(@)g9X1_M_<fb2kEGI$sJ(LW3qJgs_SCG+ASCao9;@fK zV+o+mGAar1tqk-&qJbP3Cv$*Gm^CRL=2gJM&WhQ~(N2kYq1U^XCmO$Sc_9n#qmV8g zSPk!w1B^<GHb`3Lw<jihr?eY;wlgF=8t&hne}IJLiH|~AAQsSp-4k?+nfqxU8)+&j zOr89PN0Uh2^k*5Js~1gY_ifTYQ;~bP9kbU93kko!Dj{TJfESK#el%wM#N#I$qvfnq zSpAW-V4^aw<w3>3V;g3zG*1Q=fr?QIXCE%)jo+iCg4Cr&V$WY@8+a1nhqDR&5Fsu` zS1;n_EIfaJMv6#ax7^+qhc!$oU6QSN<=Bk#|MVf<&SMh!ECQH-e@?^fJ%Ll)l8dXX z;#04&@fyl%(osrR+8X!r0%q&l4~L;Si>TW{LnW+nd?eF(y@#<pj|<U0JrBsa7Q#$w zjoB%{s@5$8(2`+^Z1d=O%XmZZ(iGqjrd{e24;w*=$*V#Vs>f(K_QgZ>7-$ofkIr(s zdZHMY^*LiRA$^v`^4e9vd?C<|_ACcI#IvKnx(wb;4I>NID57ig6_O{LTOf^Qn6-TK z3@D>N^P&e%e{FZd#yQVW3CIuU$0Hne2U0z$j^w8b-qj%#@e)9za%n3?i4_)cy-dbW z7ZMlAFt#WkgCCYb*w+eQJ@?~(&1+WKFeE@Fw)d}D7HFV4o>zr9beWoVnd}*0&rAPM ziFZ;Dx#KrUG?mI`-22Okj?6RV>asu%>3w74%R=d~APH@QJGK<Wk-$qsm0Vv(#bQuy zCg;tpRsU|fK?VQL^)?Zxkqksk-p*s1ieSaZF<_p%OVD~tc5PN2(6jl=rEH4Q6Hf)a z%Yp3Iq7UW0yYsehq1()yH;AC|b>}rWkc-EP=dCkeq8FoO%0I>LZeNVstbC_c)Y8@Z z4Aw=tQ6x{@t7I1Cy0$SeKSIQ4{-W<1g?Q#Vq9K?^{%Z6(C#aF#@ShKdnqPuwv!8x8 zz3okPd!aqr8`13F9IJZSey7FE@TSZn50>+OB*7#^TnYV#=I8>x8bJu&L=|)R50wKh z4HFGo_#Xnh=x!=9R3N+;z5O&#D?=AgO*MZ1e-=f5gT$U){!<}8q+}uqg6e5w{YQbg zTgdDwx`wpS_x$@)_!&q5u!`@AH~k?ZP(_;R{d^ZFjs6F3{#$DO{}=jy9nmnbJR~ko zcBw%TUO=x)PNHEH2^378dTLkG_b+x(@6NX#+aGU+{vk#r8lYx6=7s{WCEp|Ivw|0e zPB#2;@tNM^y&I#3&4bQG*VQC@qlbdj`^A;t42P@snsSh*T@dBRI~Q(AJ*4cflhyjA z6UZYxXpnp}*^QzZx%VaVYVs>M9uBR7=p1_@Jw7sTKq&jUB|Q6ZcdFB%2#87CXWdUE zKeL2EwKByr2Ls7t8P#fsJP#+=>pxiG(w41?MW{sN(@%yS@vlOuUqx@0oqbPs;uW*n z!T8ASQf=bS+I~lHI~=ukQnFdj{oCuTJq2yc8S%fXX^tFxnJ2dm>~6FrWGMag7CmTv z1mrNNUK|9v`#~u(hcpS)Fe`UgRNSAinHMc~Uc30d+B@=iSEswFJ6>oO+`UqxQkDAC za#9<c-m5!_4OCT6EoL#;%zyDXGoCJ#Y35zpNS<-zke6TAQA?6aXKeJIHjFb+gL}F; z0G^y&1<%YurL?cAY<T!z62ts_Ywc6H!NHM75Z8~D)j<A@9B#S$55g#=YDlWAbE%}0 zn}(K*UH?J_UQZ#k581TlrWwaO5$q5$9U+|uvp9{Lfl+~mK=lpJf6baTndirsOdotv z15c148B>9(p$7?dB2Ych)^uhjG$dGe#PWwz=kA5{nfhp+R<w`cR`l?GhN)MM$}9`N zSj?T%<EDW-=TrzgrN6DFdECAw?r$w|*STU8hu1qn4c=&|=tI}MdGz8lEiH|YAtmsg zljh0u0iDr_Z>4114yQ3y=YD&y`;uPJxYl24gd#`jky=GQdP|b3SVNLler(X=B_OV| zsjky&(;vdbX7QRU>t{h!jZTh&a&+P`bPnJAfUXelIKPIm#G6%l3tE-BSnjOI0Ck0_ zeNbk~BjtW~_1JlS?VzmP^SNHE`T1r9ir`{x?JQppit%)NU)ZMeUi@2kEq>2zx1(Jx zsMGJK>A=8td@*TCPaG9d4AbX6y`e2FZj7zVH*H6zOxDimNktO#v=U#e{7MJj<6TDp zp<ODbfH0ih%EM$jp!_&NF8Bu<q*iu05YPO&1Ci)VQ^F1VB{xfwYvI8n_u1S2v$r8o z)%GZAHd<!fZr$CCU{B&dD^d`g?HBFQ{5OZ|7fA2=&VbXqV5`Q$Yk%+=-Aq&GeiS3^ z1K#6eq9suI{08n&dm01SgNRX<YWT7rxCRg4jsP&!uZUO<;jU3Q4M@D%%PaVf%#=l$ zQv@k-)T|;zK1Ga8kXsVW%W((dmxAKpDRB76n$Adg#Go_n5PH}$c`mDih00`olQOKm za0^)MQ)`%-KN^^1b#bjHG$rdFP;(uLu_Ml(k0nvw?I9Kg55~I@ac<7uStu0NOx_HJ z=EmC#;sapcObOCJzXrx6XLrw6{v?gC>1tQC2^D&ZH8!KLTgB15B&E;*CZQTbfIIC& z(w_^jWJ)2B`mATXhe<AUH2+PQrG3|`$gNr$<^p)J)1CxWrdnf@QV**&wGwqO-NEQO zLqyd8X;QJ+$fWwb2g7QI>+%JT221@NK@SE{4>Uu25j0JI6z59G>jIq;TnlLbPKj+u zTL5;`bELO;fA`d_Yv@bp)NLj~U*dKc_!2@|Q3M1L;7eo%u5`pB52b_XUlB-4&A~tY zZ!3*#l~l555IMWnuXUpspFD_>tdMLH#YVRI8Gl5a%`UMXfO5s{dhs3ITxBw=bqWH- zDc+Ost9D1Mv5g(-7rmvBHt{3J*~*I}otd#ZV{M$5<?NX(x9l43#y>RTtM~AmamXuN z7H|xi2^p9??<Kh798xz?T$i|z_e^j|-|P<ydYAIQIJgIeP-Bn0?)Xi85Qzosky|w{ z2DqO?IQUMEXbRRIP?LO#Qf;h=>um$6PPuZ@OT)k{)FHgp%U0g}81|gK=%$Gm)_ObR zT7fsz!`~Ou5rjIBX*kTISgY=@9Rd7rIsksy)gemMWYjET+_8Pwg_=BVaKZO?^a;MU z_TU^!SnG-u$DrXwwN2j}FLH`MpW^;^{0-3mj=$i}l^9@|pDwLGA4a7^7k}~}MDGGZ zN<1ry)S#(BF$5drwzGYzjX4l!0M<zfi0<fkCZ5jfDEN5r?_bUL2D3L4{!k}iH=q96 zy38O2(&gPtzi;CshYX7bxUT361a;Q65R@^6w{6PykOGaS<sRItnv&6ZZabFR<jDp# zx5p|z;gdlLkb&kKB#vue97CH+Ab2?0;D;xQuXJW;7zDIIC>^M@v6WZ2A2<YoiM#u_ z!|q@>`EZFi^rA>y%-(GKd2x<j7v-O0>+|~^Nc16*J`#cUd?OUU6d)o~+Sl@Q9h|P^ z*#P}dr|>Q0r>Uh?7;;V~K(T?5NdWj@79a=r*e}qJ0kX0)e#2RbWcy?zdB;$zpL6do zguT&*n-5OvPSB@5T?&~SfsPlczeVxUfA>QLwubezp-f990~buoPUYszO|U`UBX%z} zap!Ia@-ZFGE8dRt(#LB~-XH(RYPteMO^=|~IKxeh#s*fCR}Ig=3r?__76ML`&twKY zmlY*rS;8zLY3vL|ynaJxEh9za=C5WywgMw2ZqOL?4pr&a`ToScO)mF2C%{+`{;Udl z<dL8He08kdMN+1~(S%TVB?vugTF2s%T3*&_2UzI36xbR;5}(RK)CGDku}Es6n?yq4 ze=c~Z%sg_O@@0~S%i|2bf6W-c9pWSe1eCLdhVoG0h{*_U?a19yr_PCB(Gau&c-m+- z&qDvf3y<?1P3(z^05aFK;Sak%2VU7O#?<l%3l+61t*SfO*5{akPSnp?`+gQW9W@G< zhq>VRR2!$nsa_WgBG}tRJ5AqMjdR$toUQ@>7pkLn>-=V4A>OCX<D&cyRF2@O6slj7 zrZOv9tn!#G*IWWL8&uo;L@HS4<WfJ~%p|CV%2a+;MGnW4i(!ICPpBU@GT|=xr;vGp zj+Mbdr4uFjb^`nrl(%(b#mRXFC&Op?#+Rw;MGn8v%_e{`=F_7YPkg5vEgkx*bNJ8= z#5*eKMpG1mqVpf!IPN}x@Ac{UnXmr9OCq`+=uau!32wzUrFhYQZV|-~ru>n>-cxAo z)LeCu-!8XbwAM!|PNy$y2t>ZWmQ$`QlkX$AT~vygs7QhugSN_xabRioKwSxS-<u%$ zHJ?N{$~C14m~Q9qS5}Y|kQ36JDY5N_d;l4LMCrm*QXC0*B$GXt9=LCBUse?cdcMrY z8%5|aXf%z?hoY?@H6_sjeN#I5B7Y<04`5r0Ga$ojBJS8u5OWg-t?}?^GAQ}Izec$b z)LmwSVnU_f|L(06Xm8=b9HgPWbxQ{9t$&`$y?fY;(}=#{-N9z{&XehRoJ~_vwugij zFIR76*C2d!S-zF^VQe)x^+-Y`_}=Soh9Kvqj@#CMXPS89vcS~dE$8{{4~wJK$~y^| zBr>JcQZf&s`HDDinEoBjk?8t7?WiAFflB~8?xD6>@z6A&rKPD(U7)6-=(yOlQ_R-j zXVQq@fV#E<X+tC5%JP1P+soSfiaWLujIOM1B_F;c(PkPGJl>IBiKkxLRW>vHJteQj zVQ`ZULGjLe3@GU$mvOiOEQO&{<1=_tjNe+8?#JD!YFF`g)M`MP^F@(IKn4{<dVuxn zmlI(Ro~oTKIUXH=8AkmZU~e3-T6@-5pZV6^4Scfj=B>rWT;m-qqr$st67BPSmm$X@ zhviuRoXcEEBzJ~&Z&^+e8fc{$kb&oHbp_bI+;QL;M+%oK_)ipek^;8bHTyCSxJOF- z?b!YFbr(o)fnUF#T@+e(69bA-i_cZ@W1kL@UJrYU;1mW)h?;&^p^sqq7_N>~jnnSs zZ{OG*Hs+oIT%!nU{TA^Q2;sCH4uT}re}4o3+}h*EGLCz5_Mn2ZN4V(q)e~cY>oy#L z?sX&6>br`ip8{M}`R6N4uHK+oor_OZ)a2JwKj1w)EX#RcD+oTG@B+sVqpRTP!8ey; zOgeSo38!mfq5j`=fEVl?xAt)Un{>4Rim4eHmR4-q>>JUYAnX{Ev<s>KnBBJCzWzMt zK^N{mcV{ez-)u705J~$7ZXN@-?M2o~>*aK85j0+VI|m1h4QN<yerjmJ7jXT!>2(tl zK@_L1CaYj@jCWj0Ly_ixf)1c9d;!17xEk{r=O&a2h31_=eY3{PTYXbYtuNru=Ru#^ z;0ms{#-!uP&ESuLwU1VYyFcVa7ki4zSWW$3;20al1^lMB2P*f?d%t7Bq=kmH_7Q=E zr_X)~@t>;%fB63w`hOYGY22Re3wLSY!$1wJCCIZ<WHvGfoMyJN()_3t=BdI3>^S?a zLMR=l0AbmebfW!4K1vK^SMPj_k9<wuZGZYd>1OMPGjH5ZetSV!C$r8A+O?EUdb7cd z{W9s$olv<lV{53a#(>}78L`GLRkTPa!|^w$!_0&<m7#9%zkm{?0H$?*wTV}55;X31 zJaaV!$p7A(V9=3e!`p7Dzj*w=bI9HVgs)im4!T<R2Xpj9ouP8Z2t2l%+_j6H=eWg$ zv_EQiKT`_O_)FioW^B3dSv|XO-+vbOBeRzH+`fK0Ti+)d%EYaz4^OUE4dHi>|1=y! z&|pARzGXJQ0O*~O8F?1ZBtASF1UgYWi=WjX2s4!CDbfA}KuAPj^EQ_06F-CAaDoWt zV{`*i3?ON>vWdMwy>E;)UszpXw8eJnBo&M8yeHY2ZBGRtP0G@);tiq8_qxyFPFBFx zs9G^kPN}`y^75f-<K_fu3}{DxY^$lBPUFrPmz={*`oi^Fy3D?>=DrE#pY4V62n^S4 zBkC6qp*Q>mz^l|ydm+J3-s(cnLgnv@H}<#(LE$8XddX2{blE+tU%m>Cj;6)9rhhfb z_1T-cODc`|l1Xh!@PTThxzu%Ek<}<?*6aGp$ZS=4s9Pu@nn@YYr1+-K8T4kJV~&OY z*|4|YgFXKb4C&#`36L5GUYXu~@;!7vO4mWMm5bMAQO%jhz}T6_l^LM!Ufp!F6%Hyu zeW3jmK^4`*ovdUtF!;jaJhMoZJ4fGY8Zg0zJ>jqB2BXOIUTSK}YoQGEPxF>jPue|0 zLFk5nmIr`%W-)yNqyXZa5A-Q<d#=Z(tJ>7D2_|TUQ@QsXF>Z7o=BSlatFM>HOOGV> z`z34u<{62#oXxP4`5MR{$s5i0t~<J|>MS;vy|M)%bQw4`81|fj|Bet^pymCl>L^p- z9ub^%>$=8Gvby4%%zpY|(KL6P<6=yTG9v#`IHmj>VRKN~Gt_?qQo2a0F2$!Sfv4{f zFE)HAaO~YGX|*a1-_*J(<eLl{c5cm&2~+X8Zx-l=8ou7$qOiN?i~G~rh_K9U#EHAe zGqhxt{iS9!|L(iHygA=HMsA&NwpiVRx^efK;8s$tXH3t_K|-xDuvUvwE?@CpAc8W6 zFOE5_B@n-Vq^+w=Dvh)2?N&u1zlTV2U!*8hUp#&?W~oRUwj~Rn1fm#<{>V4Ugf~Sw z?x3Z;zt*;sujJU3a5AlQ=J=};-uMg}3(=2xPg~HXzlP2E%67TeisbGM*#_PFzJPs+ zwz~9d?&f5E1_I7gWcY9U^;}&Cz|DK9bOP^)2^tJF#9P@(>472@3;P#;*<Vx%*RA!y zs2my&4To@F`WdXQ+G<Cm;TpQ4(+E+AtwFlNFPEP`NojQeJ{9X<l4xwBWQ#nI=0W*k zpunq-@?lHtZCxO>o?IuNw<qG49;CnZ<B`ciyB54;mC7nA&ousT2mNy&;1IFu!e;_n z(9ekb)n7P#cF<(!!lhSvgG#e&)Skg{yRebIz<jnqD%dJf+qD9{jF3g_lgBl($-VM5 zpQ_HQ9miP9QUZ05|MfiBSW!SP9-U#P1B!{^#ywhpZ9f8s445bhT#MIp6V=3<5UEc= zJKl_0%panu9kP6*VoMvUvVhMi@0*d0n_k+0B!jHcujP;ElqA)viyu<*aF(?Svb(E~ zK5=!}O~)$QDwgjta&K`dTV>b!WEblDy15KAR@$B&&zAU`SPISi<fR!)&`a+<a6G0t zt9Z9v60|cchgOmy-*Sr)6dVzW`~%D%3D2}^b+hphtkM2mYVhbTpdh9L0}t;@b0*zm z&=Bh2u9?4oBPD_9yB!53``UydfzaN)09t?C_1ImN{uvbGg4cu)i^vF?^cni=Vs$jv zKsPfC&9q<;>au^10dxFCNuoav3UXaUJCV^uUGeux2Tejq!my*@48O9{EYA<CmWqIM z!Spx^p(qa7KR{@B(fiYvnoaLWdBw{E19b0t<-GL4n@C+@pnls(-`1j$J(!-9u0YWO zve(kpNTB4I4Rgr|Zd}z?v*oF2GHK~<V(RL8LH>gLYADW-6>2$$H=zfa&}Cpg?qI)b zmM#`khLEx_ol^1F<bE@b>;9N$C8+wUjp*~E=|3=;+`hW6XAX9TPIAHVD)(_eJNUFu z`ij9gwt##q=l#0G9M_`pcxzdHX)d(-V1MetSw2yY=L~4h-imfwK2exFnN#$uOe0We zk4I_JacxpyX--DnxOgf6^ZOd{Ts+8Zp$J)H6*4%4njOLOLuS1%k+>TF^v(@1Y5{+s zPD($huhJ!S1Y<3&HHWcZAoByA<LIsP^NJ!CP0%%Ne6L)@KOV&`6l!V&$T6jG39NFx zD}Xf%?=7nUjOr-qsIcrgySWWUi&D+KF`r2R&2Md|Ic0C)f+t}F8=W)0)YJZ-#d!K2 zj}*Sg7TbMdsi2R~*zg)5d|hq@%k|A8ukz?A>Sab2f8M~0HliIF8>4i|R1p2j)${d; zN$x>%_IOe8ACmR|wcmqvjpzA?O`lLu|GZaAXI<_a8zxrC`B1Y$PHpFGkKJMki9D^e zC@26VfO&pK8o*6PgYj{>Z6IF0$Hdh)Eye9m0L(QgPW94MJ9aTDeZgVbs&*T{2H322 zV59y*Sqj0%-~sApf37!vOD)h4M+_QcUpoN1;N_sroCQVqAC)P+?hly*F(3R|9|uCr z^azSEaN5-oc+HAH<T(`Rdvo9*5$32&Se<)(hRk};bG_C3&z6ua{YC&oL)ro8Sfs?B zivAQH=F5c9pFOVfSFeE`mc}tMtI@miXpQe6m9*=5=q&Wf633++C#N(L%O2q!SAXQI z!s5EOcEDDogu`DiN118n_4@ktbrA4<2#mjDlKXl)FcfizJbr7cL=~GxPQMtBeTyWt z_~UWdPW=oO@CTzPSY&IRwij|$W9?yPIGz*;ribqq@B9q4$m@Xh7xGV_90{!ytc8Ov z6HZoa{p~sD9P<rS$K||IsSQ0~{h(CM?^47S{LkHiDwTQ`i(&u1s|stmyQSH=v#)Y= z9hT;~momoHs)JJnQoSz&xzXdwCybHbPBim#fi6YoH=nvlA}f0S*025uGmqMDI{fM= zLG<p(=u<^&g~Z-M{Zgz!5?&_VBoaFRo{&JS0_EH(EgkfME@!MV#`1`e`0D^mt8x*z z@DFPgSh)&<*6nliePh5<PN=X!@X%5Zdr;1Ddg<&KW2tp-kHvasb;(?*Or=PZXvy@K zU7qBaN>XtY70gl@QZW`duW6zXSxIzWX_QeCHfr#gUM<w>HNn&AZy-kOMKG#R8LZKr zoTLN4Ra6vUh1|_v;oJxMW|1|mzOu0kU`NQf!qR&Q28S$SWTv>TBJ!JCt%-o_Qg-(_ zI#63bCgdOcV)f8-m{Su)sgN>$dzncu&4k<Et$`Orf|%AIM}gXc40~|}+7e4^7b*cw z7GW>snTNy<lavD;E%#a+dy-LwPp&RqT`(xFQuH=t7N4IR-dGZ2WAKkBkTS$b!XkuC z-L}7}ZDm;HsUx#j-wnGUn8@=g^B}4;_0gW<n)+DbHN6p4j1x_{783JE`6DBPG8%uQ zIWp*$N~Qq?zN%AR=LLXZESOk({Vg`BHOxMGd4cy2n8sR|B*qM$E|}RlG2`-ZS2w~| z;X*<Eb>n4Ep%Z1S?74Zlwxbw}9fE8AGvJ*i^c;(uL+h0AH{M^00V@W=7G@>OOZ!(m z!1_~rlPrK6Z$Z%vsaWUOjdsDM%*NdKn5eKV{*ag+jwcfVAx9AQz#uwP3ilB@04jj3 zCX0^|E+c}tdrINbC<b)i3Rs31_9u<ftP`NIdv2owD6M@_CekMXc(@+FEb&`aUah_V zt3?{Fvpf|CI3u1fq-x&kB5=cGs}M03c5_+&5Xc{xxf1dRR<$N<ji@CYaIN^$oZ!G5 zH?kt;5c0ygfNG1w#w%%<bek8_Ec3L4u_n=3`(V(hK?4VahA}#7{slCy4USM`5@PiF z_z`HP9bvIxs4-e`Fhka3bjP$pCp@7!Fne~-<3#mtuR5I>JD-8Y<89(-#e&$EbM7rd zt~>^KB)n!`_aj5GalGSfxO@&(1AfYn1)sL1sJ)xc5tTCSeyrwAXn9|O`Ab1ae9@I= zf@by_Bf^fIL=cGLV$T~7HFB%tjn7U(=RMDN>Qy-VSKi6IXqkDL>MN#w4@FBjx3wae zr&zV9j2+K)?n#+b+F`gria@#_0Go+Zu<S1?X%XIpQ;!-kJ1O3lZ#}V0jBWpw;u-m` zGsP=S;Zy}+%n@}somOG;%kr$e>ED4R0>-}?MrcOBFLFZmxEI?<`+@Lig305uk+u-; zI9gY`8rvLEAT(J<6nyb5)lg>YAsc)7NMuT|<+>QpV9At`B6rjYAdxq6gcqbMk>x*Z zzY*R3sG>f3qhMLaelJwD!9$;dnG%lH0c$8=57nz2_s_L7lCCcK)yQnu5?fADTNUVz zC~NY5dfTy2gnm1ct}UE~28rEY0rOi>Ya(Gpv4M#9_@W2N(#rI#KS%WAGZqT8zF;zB zJo4cgg<A{zlA{CcHl0SoDXZtT>xbX6rQYZ=b=08V8q#h=KjzW6-NaF*2HdpsGW$eG zKoHha%?|MGSFqs+7MePtt*m&7_A|><EjrD~kBcw5tzUjK^Go}8Q<EAqg%PLPl#5e5 zTMzy=!An26c&%YC9F^N(v>Q+qA`QegriZ)+@W;IO0plubC?Klu%X!(368;(%SV$I> z4cv6S(V*&-N}?b<j-5@p_14tt!DV^b!e3JhB*-MqtBise14{n>h(`|tXu<Z(P2^Xo zmcvsXVGCBac2$zm-33)*(__Y;*T9H6j$lGqGyMbosHovX5y*9ca4I9OjOLa1`-IVa z$>S=c!yVxNwO(Ot>SrDeEPvX}f?%;*=MFhgYh5*rA{mBFUH4_G#zy-SW9`!R)})k7 zLOOCC0G%mk=4P^kC+Wg|J6qf;1sxb9ljw(HEX|czPk?`ID%<1emr=|e%vIV(GK?B! z*>X>G^+kTA9$wuT*pgYs12leuQzb8%M(-9IFGX{jN{t=O`iX%&)Feq0TS~~HYVzbm z4zs@TR{{pHiaohaA_~-b+8@ylok|&lqq&mAWZM*Cu{ra=WJME%I~M)Kf~mJxM^RVu zf@%jVjU}IlJy}?mDw}$?ZU^grV5zOfoFjq{niB@b*^&x}gNFrg)-TQ{3SBz?#RvSf zJ)@oMKk_-&z|FYY@i@<|+PSu9uZOXt>#;&5Xo78MC3MX_PXEw###(L}g7}JA9fpLo z|ElTX-?|qH9;6f*eS0AX<;cHDz<kvi5dP$MrUZ<=XHZuI%I@uX7&ETO#Z>2P=>|2z zxr#x73{>7Z1;ix0yGf_;uT@FM&g-T#rZ?8hHomr!KId#~9#{dkQPhxs*$7=mT5JJ} z&xc`vLv{dHuvLs}w-14EJ>%o(2O4DWb4azWUia{k7{tcyT4~oS0z)ZFk9ErMJLwYp zzMCK(O`1dAr3TbvFNy`*N)J<J2yr;@_WPmEciyi>1B98J^RXg_I5$3OnbBryX1!Dq zC`EVgRcb`%GU`fJ+FNwh?#t%*Ktw7vYep*FhTuM^WLIbhQ7bGK^po8Y;^Tmc>Iojk z9enNk{uYr=hEn^cri4l^UYT9g_P)u8qYFj!4dactL(p)DhkV;$W)y7$3F*W_3&-Zo zmoZnU5A5)v`S6G>%@dM5Zi7IVM?hDIsQna_Sp4yDgt;HHK-cJQ*hGbpK?)3+y_uk& zq}Z*PO8QD-wap5cGA&fxjT!e_zb=?nAbT!OQ?>T-&+{WdUg8f)2pZ`BdTypdeb=V9 zNmfHagYzGQzHD|jiicrz1-jk|_(;TV=4r9$AN;kAnf#ebuhP^tp1*(Z<^9h-W<0{C zJWsol5poe$H8Q=kn=7?Yws#Kf)Fiy$A5>tFUgvhzt*I{aN7N3mR@CQwxfUy77rtf< z@!XyrWsN`D9Meam$0*Fu+M-cJynK+rF#@`|jrOx!=cQO)={QDyw>^pBw_R#-j(SSw z*HsgzvmE+_jJF;}%6z^saL2Wj_vsX~?5zcm3B&OPl>8{Cwj7=+^dr^!&B0V#wTZ9> zUnrunGxXDUomu><Xq;F_y%ad!gKHc(Uqx~KR*G4j!XZ*N?oqL$4K!F*pp<(VyUzCZ zBiX__c!WlzMDfeOAcChO-<5}56ti8Xt<_<qy^Hmp!5d%34X(AZNOT@rR&FEi_!a`N zVcq_{6PGTyb*oj^PmpP<yuYM8KVIe8+$rx?i;#MtA?fXlYM;bVR&g>*K#F`Ro>c9M zV^Me<-B0|?)xol`wixnLW2Arb1W6o3nCq+Lgwv%T()$x-X<e;*^aHPm(T>QjXkgO8 z%jDiBe^qm#0>}%&oS|(Os?oeZ1~+8qeiqe(xjbFGaYMbO+g};J-S;OA>rPm^M@UJX zEMi)CTa5=+pAIab%Fmv=i??HK+gsoAC0LO3g3~cB+b+4c%58e{YG$rezf*%A7yt}> zK5pQpcGy!9NJMF<hf)LUFpXQgYzp%&Z>sBsF<W7&^I|KZ{CY_ShARZTNAnAGO5!6w z(>9V3caB)v{wY^%zb#$j>bUd$@W32{`)J{0a=I4P>)<R(5GgMQYm2rA-;QOwGNj+` zh-l{+xOyH?<HYo;^Nv8mIrtI!0@a;X8CrIsH{S0+8aDsXm(2kX-@KL+xO>;)c3LyZ z!^Ts+N;{B0jssiui7l*XgWU7b5U;S^u>Z*Z;13~Sc(}2DS=~r>m*7R~6^8@-<D;Ib zrp02h0;zMuZK_kZR`FkFB0)t^c^R9+7v6CHkK_kw=w7B!Q-g-hz8>_sn3&)*kXzVV zF37-V^*nCS-a06x<Pjr4^kV7?0uxAVL63`ALd+DYKezjvzXYm@#nuuqVd%X=$U+BJ z-?A<H**nfPaqNFe@??K}E)v;kAgLX4`iKW&yCu^9xE^4^D+eS6J7xqV(R+<S;e`1c zZH#5H#-2Mc^Z)JN%>=Pqpi*~l=gm?9US>f=!eDIMg8vpqpsNGiIT&HPz7M50*O~wY z%Dp)ojZ<J@{SzqrpF%o_2Iry2&e*)I<W`3+=m!`I8%k4f;#Xk(i5CBj85_94uk}?H zL_-mzK0A0(=xom)ON7jhp<77zN6EdJ=|q4SFv>A;(cYBY{{621HCBOs<>u!FRQ}+` zAGxVYnnRv!ocVe8?9z0XZ!vQI&q{^A5;VnGOUs7PWkdDLwqQtFTM+w=>{U$b5p%M; zAPtBRip_06j@g3p?O$^cUke+kfE)hQCS2k?xFk*D@sxr=)%Uh&ibI_6v(A#RlA~?g zcfE9}3U8`y)$?>NUnK);uqV=2C)eY=7%%7oaszd2`y`9Vo>Dx08t?@rz&pGc=~K8} zpF|(GG$!h8Y>`wXQ3a%y-<YD9orDVBi#<6?FpK`pp3R=^&Ay)Ptes;Am#Ul`%gY>> zlQX9sCV3U-8vq6xLJnvMUCy|jz=Xr3z=Sg<Ij!gcqZXQ`9W3%6IUdNBHe1;;u)@}b ztp(Sz=*w!)NSkg>>A^)hzL8qb>$(!=>(s@L2zeLz<efrG!)H$}woC|zN-HC?)OE5s z%<#vTr)Ef#3~b@PRRvcI7(hK-8#JDCqC07lPR<q^FJ*5J?WhtZ?I3RzcyocY2fNAy z?sYU^?E0Q=L_C0KBWS1QgTX(Q3Oa<wp6mb&JK81pK{pC`vPoOTe)nOK+3D@9!SAvS zNXVnn$Sdmijc0n%kc<5iueh&c4y;s_T#0Dp!j)5eSI9%|xfi@t%#hq5c|8IdI6^D^ zDKzf^w~C#imncv-u(!41Z9Qyc_R*hQ`C5$K#z8T6lg^}e*`l=6=G7q=X|xaRckUZw z?8*@R5cL?ZRG}ZSVzyHD6o10!L<Y3>*H27}zrEg794wlB8cOVZ<8}>(I~DxZ;#G=X z4`@-7IYIePawW=lYAmdtybC*phg+AT*fdg%<9331djj8VxIcmigLZaO-g{^HJ&_*4 z45Asmb?u7{d0~NXJ>0gP{3@hd0u`R}$sW77jhF2z4|Av_BP81p<4Xqv%~iEE$oHiu zf<im8DDN%RWhYz~(h^V9x}1hzxGzWMMX<^8xcAP;PdV)_{jSql>Jr4K>R0+eY^y5# z<AsNA%0+=ugwS)>90vnuvGH&|mB&##bb+q~kDhh>vOgb@bw1d=xV;ni>fA$t+>cHr zcATC5p4v<8!^K~BY>!&7B$EWv%D6!V>WBuh0G>8r`tZf>SOhZlK>d>!8BK?9wZ*B< zg>bRbi07j8+zz_=iGr>g$xPsB9OrC@SA5PgEB{DGkVh<bsl5iCuEZY$v(QG`A8toT zdtQHgwSdlLH+}WV=E;eD%oK2-4dsgJI>p6ml$rM@9xoT)o~omc9Wta-8n0Hc+u)|c z+4x<c@$6FsjoN~%RKB?1U2`4X{jk~N)q(-@%6+LS8FeZ-VxxxoR&5EICYeU8r2*RH zv+}YPj72kEhu@AIf^k%ud3+D|SNLcU-D&5Z5r$cc8$Xm5ER0<IS09I_?RAn|8szsW z9Z8glBO8?AJCeuYUkhZ8TInD5jPDO<zi?C)bS)4p?oz^!E0FYEq<D_s?^bUXh|qF? z`P|X0Ve(AoVFta~k$lMa$7PpU{~Sz6zF?`m*EPw!IXYj1HF-Ng(23_*l%?9?lr#?5 zIAGg3@S=Y)JY9#NvQPPYm~nf(Rk%g-K9W3(U5P#gBgPFc?)}nRJ2zPBR*_D389LhQ z+?1?(n}OgqoZDPlH@fzImw%q|=&7RCd`mE8y+^&Qj@5{31z%>%*N|bUHQ%LXd<2K~ zYCkEE!HUHXq9$^yC9?uJ4!MWqedI0|i(Bm$cZxF+u4HdZ37^XWSd$zx2S|Ws+U)nf z(GQ1D<W|M!>j|85<pNWVjj}Dez>J`RJOf4&ahV9<>)b7`$)&l{n0h`o{*Yqc=iaf= zz+zaD{bi;qKXHv>UHnh8A#GGXo5NG>$vVg&S0e85VN^l60-5cXpbOI^p&B07`R_8D z(<3~g@Qyo2ZdpYEyCT!7o%cDcHDau|5Xwig*0Bt|hF|pOBTdW?FOE3`?VbtfO@5@d zbMFETuPd0Rrt>U*0<4)kM+AE*@8^NmLdH|-X~Oh=e9I}zoH}c?u6)L=A<?g)*hvbO zL7)>@N2$Cz`$~8iRK_mXapQjE1QV;xtMx9Ee0w)sm7yM!g8AJTeS5os1b6FAfJ5VE zXVbHeHlvWrah}jb=1{@Q#9<GS5vN)QexqND*CS+)87}tV(Br=Xa~T@Zy|-iYdJhr_ z8t>~bw-1SRmj&lAC0PISjD=6y*nO^CZ}?&@oLbznv3Neqt<)upF-}12v4e+vK&?XK zSNai(5#*6b_h267`5P~NaE|$lfi;tg#aVr0Q%eT94oT2Z{4jQ6h;4jjqQB#y!RT6L ziVCdBi@IKIgwG-4zhU{7^D_(}YiqwY&yJ+ySW7aG_<TD4k&p6mb?C%rz*{30t)+eT z<B7p{b6-Yc%F`32e%zrMLFB+Xb5!GsyZuBZ$j|a=lj%}x+bJk$zT|41-Unqm)q$q2 zGTEFc3&D*#%|lWKdQKUS^EKX6=W{g;DkqiNixo9aqn_~IG__#xY#x)=yG#cJb&4;^ zYRrO0Uk^J@N<OAG6_%OWUvO~FfJvGi&X~rgcXw3|mHb9Je?zAxn{w{5WxGYETxY37 zQui?e<jAZw@5p%4Uq7+#kD*U%4Z$LL3>U_<JCG?<v|SoR7Uq>G3em@>)NPOSAUV{k zRnPgl=h01deeO_vanOq&Z&yu@!$khBeFbhbQz1N3^Ade32S(_Hv2eh-Cb-$a<K2sy zQHD^@*lH$zmH4tSP?}~<5J}b__8ZR!<sk8lw59?0-(-DYI=?hEI7OhkOdF{rzoG6P z{Vehc2mTBD*L&ULnR8OOJiAFF`Wv^1pDunVcd;I&e#^X_U-Z>t1AgU%w};2bi20@N zO}W-?{r6Ys<ZN-QUzvVq|NQc;Z}c#2{&N3_=>=1Ss)vDdx9GcI1KIaGtV>LlqNx9i zy|)hPa%=yE6={%=5)qJ=PU#M5Pyvxn3F&T-20=lPE<w7xySuv^=?3YuZnpdRJ%7A& z-ZOLlJu}RnJq~>1zSq6hb$#km>_k74^PhK~`-Ki(eBj_WgRGS#H)>o;66SM}!7JN- zY<ks(3wtv9mwNP`T2rx)Q$K~5XQsO&yr)h!$>c|~5b<wQ5Z^qVv<p5WRUk?{EA}{- z@4f=s)%O=>S!2aPH%~(TR&6itGc93xc<yV(&p?xk`{uf7q^iOTKWx#1v%10E$~=!Z zWH~#O5j)`FO|0S=<0^*q3z!$Y=$i6;?E^)cwQtXl<#9iuQ#rzn?N8!?V6EKWCKi&v z@*h2KUJdrg-x8Ya3}fX$%X@;!{E$?>K=N3u?U?)DEC9u)S}b#DK8@cKVvU9hdVb?` zvti)5_URbFvaa0tv71wWPw_tT5-}FAEnf8~&*Xodw3}0#qJr<a*<8gsT%+^RsvKQ+ zc=5d*n|#XzqCvmi5T_57BP<TRhTJpS#Pk<Cb`o*U&h<N9F)_40?Lvyeo*6T}I(2Z( z8FS_beh0&{PVGA4A_|LTepRPpOfK(=j5v(E)(&kYeT&fka3Iwl{$O1WbBmHGK_g)= z-{BZOs`j3k8f4!$k`n=_sn<sXif4(D^<Q(0JooE1iMQoh>ks*6I*A*u$qo8<!89-n zj&UMqj2?9-p%|+f0X&(fDRAR7UuO_~#1(=cz!oq&hQu&0RUa<1*kOJ{d|R^7TE#tz zvq4teen|IfkA09W2d>XI{D<;7ZGm)&X3L3FjdrA*FGtAMbw2-s)9BSS!O$#kx|$9B zOZ3>AKniP-(pg{_XqAtaK(dUmQ_5XHe_fv;B2ZM%8)3z#YJ3K8H9HdCGna~LZ8zID z1E~K|O@PX}L~!826*TL@^EV_Bx{N{i7%~sCd$D@C^ijQq8E6q7c8=MdCV8Z)B@}b# zqT21kEoto1L=7hL0hiNIluV5og8ZOvd|f?eZzvursh3S)K(sdc2-`0H{!<_ISj*x7 zWP{hEEM^2GX5F!7kr}x8_*VLthCF2wl`b6<EN*W&Jn;t9wmi;?ZQD8N+n*+PBN|{B zycnG9@z;az?C(pG3_+60bxrwv6P9~znQi)kd*Fyk1g+4#tml(edN?vcSyWO4ABM!& z2R^l?eDl=%)|w>kr<uzARNx7fL6j`TBdf*#-X9=s--JrQtZg1V__2W)X=Q>gnwmah zLX(6d6JJh(4jrA(MW{Zv#$o?AS0Djn;?pKnU52BfChx~Ozr>^>ix$?;CVP)pmDC82 zI+x%PQ(tyo7`DS&SzS|UaUr(0w;aq7dE5P=+Y9V`W-;WDitWCsG@Fmo9CF2;pMAV( zN5CVU#OkH8=9j1cOBD-4_)1neW%C!h*ocQk#07w??ALseEMBc1_r#n_O0Tlnx!yb< z&-Xloe_m!J5<SP%$8B@odo_)>{kNnR)Q7q;I-z-as834<T(ZjVK9wIIcJj308R-HL zf8pj;9PwAN7%k-QmpPyb<1g5MKkh+9q1*v31^a?$Bl(!U7}-;xl((6_3=W8Xpo2(~ zaOM?n`m2Twt7bz@6t3ia!o$KswbwCW03G0iE@k*lrn}-$`n$ibpU9(fMoP9pM4?x) zc0VR^xYoy5bXzAWlKdr8?h>91uhR(iIY!^y9;0*D4vmXNi>ySHo_p|oe3xXuY_7?S z_p)72&co^up=%y9{Y$OQWND*jIc-7O#R|f<tNkR`5Ok}0N4P~_ZuTbEr3%*>yjw~z z&^^j414<2li|H$Ces-^zo9jZS9L3)j23OQk9qr*5L>`@W8(Edl!m~o&XskAPJ3Osz zeM4H@o{*HrQ(-}yeED&A7pXuVn=E>h@$NRJ_);w}X>kc2XPuXwLJ*Pe$zhzAx=mZR zA}X0>-1d^2aDZ>di)K^iiZidr7_V(jb6bL!<~si6@jRiPW01A+_P(XLgl>lWy3o|G z3a_cu`dz#F>?WJ{(3FSSZ*Vp0q<CQ(uEolJ=lIQ>8TnHThszxm!Ym0|M^^HpaIECb zciQ>_QIdBgwL+gu@RUilY*7p(V$spbRvB4!1B)<g-(_lG6T58T*VUwhB^^hoJ@QCd z7q+%w(HCU>*GRw)@VHcE)E1b}S0-P&sJr0BROhdc1qjyr=ueAW%QR5Nk4_p+m!TM7 zMHp4c2Rh_8Uo5I1px1nootBJ=Jq5lRxMJOYeZsA)6y&$3ioJi{AT;B_lAZ+dsvV#n zD?*WbW;D1HBuT&UEKU^1(h!Zz55$;`OQ9=!0L$7-z}s8sB%V;t7tA$IwZ7C?a2L)e zV5)F8W4_jGvsPs|x4QHo&Rnop^p@Xed^!k|@2lajn`i$yiSB_DFYc@RT=hoiB>HeG zG*VOH6?6F68tuGUu#lGQNx_C<a=3%yAN(WxDKRwGNuYd_j#4e)@$nODZ|M!NSc&q; za_C{Zd79ix<ZgT^g!oLjXQyrTeM`PaTf_$H7tG%I&B2cD5&edai^&A4n405$B7=sT zN)g}<&~_5%s<61*lVoaXN0zYNug#&dPHwtFU2=LIS`BjaY;$k3TU;lviipXO+&7-3 z*(I{B@{~jg$P<O%(%eNedd1KdFoq+b>UdAK4TC_~+IDrMB+AqgAD3t7+{&6(h&MGq zYcqf@D9gc_%}eEWL-R*DO~v$yr^ZQsZhzI{&Pq9PiwI`omwp{TbRynRJsLb!FVR5T z%3nFmG4>$%15;(C9?`(@^`aQ3jM<0Jwcefdh;K&~nCJbX_pP?3ST9id=JXo;=5!4o zZ&S2sGBQ4{IjPVJ<N9^kb*^Qm$P>-&7x8~qTH*^K=-2`pV4U=kX@}fr3Cbj3Q6n_K z6L5!N)_vnG=N9qcbfKpLsjYZF-_VC{Bg(auFwsBe(7D7VyHUYFV%u5usWnCP62(Hj zG*wsx`X`kQ9W*Mn<*R}TH`1VtU{o^RCI;oU1g7=9S{;31mL)NGE7&tnID1?%&&Fa= zio*_b|I_Jn<cSv4l#W9C(=t3_(Gr*eyr^xd%xmkl4(AT>Qe#-w=L9^lXU}NFG@Tx0 z*f#BDEP%v9ic#}~*MiJ?ML{&3a3Fz{YS-;|kapzl_uOd|2IC{_Y5|`s(%C0%h(GgN z)3hAiX~D!`_ew4=<V}&Jmy>IldP2-gnXiAgRV8Ba{}4}dMD};i?|UZ1z_$|#-qA6= zNu-Z~N%vfN$8|F+`dV)i2}@T#p3}<qG%qPH)HhuGx5mdfTjHH}uxz_vH0}MqLm^LO z@Y<2r{lI}fvl+KU1!3L($c2zXbJmethE1hgaa!>z9cK~~i)(SP!y#@WYfit!AIXV* zg*6F7t0=N?{YpmV=LB26PPi|qfj8f1^af2ux6F3j&)R{Y4p@Nxc>e6?zu$ujA^b7Y zWAJ1T90A^6E8(nmrlO>23yNuvdK}cs9=t8kkf+c8NfeeI@@^TSmDX`bzyG_MzwJ`m z^F+hRzyU<UbXn+rVkK&qe>{D6;duE}aV%LQrg$xs_=f|sC>b_jU`-orw#Qpie@l%v z%7qqUb)Ds~eahQ!Aug#Q`9|V}O?~0#{nO|8Ot+PML;d4kKLK=<8};lwKw?K!l1^ao z-np=<!Sz1Eeo3x@UBPL!tlG7svBo6AgF!@@sjl-hiW(#Hg==0c$D9!4OZyqTH>)^7 z|0cB(<dyowb5?4dqwv7@s;2KU?FtVH3e5$pl%V)1lHZ7HW$v4yvGzWaO4dz-h3S@G za7}K|Gn99<O9t+i#iBCDEeX#`*u!$|p(waSYXd@Vsx1x@cO!J9YmlRzwdMY6BA{!) z&!pcAzh8$CtXcy9Wt#JZ`h$Jq0Ew0AiqRdCMxjhzo1K950GiiNtzS`&e4}lmBQ=@5 zP&opP=VQuL9Gek`Zt*awJb{ux*k;8fMkPp%LdV0emD9q~Gx}zExn{TYVqMb}zXH%Q zvS)7)$M7_+nPGStNU%>lF9m{$DR{@LpWOY0HU-%rFkklP)ME%xfQ2R#39;dOvbEtg zp)y`=<M!hQU>JTQ9x@OK5Mm(Ps8u_7Dy(~`Wp6%qdhi78dm3mA(Bj~uHPE3`Cj7A4 zI_iXY8Oh%L8Z=wd&%FjuMs3<`k8Hs$wO_g33mULZWITB|1P3FV22bDO6Y0in_{CJ6 zop`MIl#R7U6HKCxS-l43(~Y)xvl$@Q4)|TgJsq%~5`@GhRaksKM{vM%@WS8obik+` z{zg2_Xgp#VMwR3y<UUy}x&D<P^;l6Tl?g7{r!)C%`+TOCI-w#L02<u-L)=+tOi2V$ z{S`ajp~Dh^>YexwO1stgiP{Fd>6daEFMABWO%Ocz=@uYU*tPTSiw!!_Y}ETT&Cwc1 zUAJhjqC8C|k)Q-hd_G}L=^r^X`PLC`cVqAuubrF;XOBzFVVdV@ak8b?;PY&W_~O~B z&f^`3UM&?=A|Tx;q{&+JA|3~7hzlgJDp}2DPcYDESBRUAhm!5Lte)+)`&1unT|p2- z>Ro-VmBT<7W^ZHXPFUdFyTcvMeK`|Y(^0Wo_MOi`vkm|xlqPFjR&oC|&2#kntFvmO z?=ICSH^EGv?^$pwS{{ad3$0`MLGYCHJ=mbs@s)08!b#>Y5ikU0McZN-dfY=rju2EN zfFPvf3C@){ZJ*r0FWe{t;jl<UB;<{1G}*|a0X=yA(kZE3bOWz8>Vq+~eYhN9N%mu@ z9=4WxR!{N`PN>VS-v+n}$2aZ;QNPPL&C*&8_pVU%>qY$P(x1G|JQr&+lnFX6DEHLg z<d0WXwd<zWymq5gh3H0i;r>})oYb|<wrf|${sBXc!$v;Z=ILg;2d!3}tK^f+D#=vX zyaQy@dYh}rXA2_x<p*2hC)}d>J>A<3#$Ob}UL>#Nhhti>UcVw5DzB^CKLS19HH%aC zD1$}QrI2GflDM5p*Cx5Uc<3Yaw$*T(?B)XKYeNLeKOk9B0#xA&3PRv2;@RRhTZ)&; z!PnLjv3{?Zjp<~U(-bNFg`B3AuPC3&e$!PQ3*-r_4WL+im}%T);bM2tO{Ky$4iZw( z|IhF~&)*Pu;>=lc53L|lf(z_r%noA&5atKXAu0K?3?P9JlYhp!@k3V_YsE2z`~Unc z;8cVFr-Jg5_ue4G3|XL`@;A++F_t45NSNPE#3&p&HW}s-^4ERD{&HHV17jUcGE4t9 zj6959S+gsaHiTT}2Pd|_NZjAP8v>61zgur7Q_Nm3e@^=QEQG(b4k8Ztu)-hHji{hq z;O|^&)Xm73(Y_}qLjd&u2LZna!J(LT<fUTcdl38`^d)U|Q)cMEtRva@o4*9wJsj@? z&1_3%Q78@n9~tTgKU-<=PR^G`yng{qXde>9hN>}N%PF?|O^6!Wgt(c9%y6OM4*j<f z3jXhpFf>E;O$_z)y{_#R(zSWeiBwY{8<6n1vHddv`k&u@feaPP!j-1?zJD($dGSR; z^l$c{|9J@hC83fbKT>8d_U?V8!yVWW*99U;=AOWngeLMoyYGDdT|B<v^b;ddX9u|` zf-pi56A~&XV;d*dL{}YHBJkfA>J79S6b-)E-&0?AV4WQk_k@}QvA%>n@z?*p^#Kr- zkb=Pw2XStmFCey0f{L=bFGU-7t`#&e86kv`l~ya=WK+anxx9~b$(#81K&R1FudyJ# zM};B<BK+!}zU|FF8t`pc0}nzNs71oaRjdo(uNIr!xSalcPL+z&1Z2j>7~hgowe@Do z^SQe?-isZSCjvsv2iY9~+YbBl-&waDNS(6T!YJL^pE6S_dsqHB=c>9K5bKKMDQFGE znoz<?1i6KXI?z&aJTl^j?v*H%W`XDxWL)u<Dae@hncK>yQiInlkp5HVPCju<fESVn z9~KNB>jp^N>L!@McKW$UNRY0OGWYI}h{`LA>Qa18P2~E6Fy;20YwJ2!fC!Zm@(Lki zQ}00^Hg+;2VdrW=E^Dgx*N=Qvj@m8e<x5{1!!(B$sU!8k;o;o#gZmFVnX_&e7Q5h= zun?|mp=hb(y8)JrFfa<0U?lR!-b<e_p_S{ks5@;|B*a88l>8m!pDTi3WZaQ-#-A%l z_De6b7h!uUqxzmQHtgRDfpp}TS;HR-{jPfY^H|IT@v=SupG3jTC^vBe0!BPLy%q?< zFFRCCwf?vOlC1R40{D-1dJ4oG>FSXUI104F-S^)s%nOG|0}-)C3{J8Q#;&$B&nqo? z9BSNTcH!OhNz!N?&GeJ)mk1tLDt`1I$Rxwl5X-Vs>cGk(f|knz|0rnPzwm>e4iT#H z37t0hrrb3(Mu{(@#`-7AU+b53+|XgY#5j+%b$_TMM_P$sdvmf|j<a<ApwDReu)k14 zWwp{~!-<!m#SZu>jFV@!TeNLGieD@zI?%5fTBbQdQ_^}QSMnc`j(#ifnri=eyf(t6 zP%4hi`|DZ-hL7>~c@T$L8)ehZ#;)4xP19VOb<6vs_q*-$i#mj0sbWA&T$J*Q!Tk~^ z2G83uS??S3ZndVJ%Q9N)y;Z9Oe@d`hTkkY#a(SGU^5$s1#3Gl`C#iHj^m_j8E`ra_ zFF1^KAY4atn(_8D<3+}s_2*TqP2LFBmiy*a9_RV1?=B+WW%6>obnN1`*G?jr_#sgN z66=+kdNu&xwK}_rm95F&6-rSf@@ixv;M-S<q9dklPS@=EL_lMaYhM3(uTg5Vv&dg{ za;^oDhIRca`qSqfA0Wf4|JiMdnX^pMXYu$t$DX@^@Q&5i<eM-`5z4NJf}bE#U*(qH zMrzo=wRNr1_NK){!hTJ5g|2$?ssZTVp8sW)h}?J^&Ww)Ua~9iFXME6$N8eir&(Lyh z+iH&DVk)0ixcE-wwoJ{JUqbb*8%$R!GkgDQYjHZFhZ}D!LVDx$52OJ)T%jsdK6Es_ zJYq9HIgC>E1~XV5hqIzO74FT|&~hE~XI(XMhU8pGhOx!y_r_|-5%c9vSc1^B)91-g zjtmoc?1cL>ncKc%i5K(6ii1${Tx+tnA+zYRLeq8FHtPyp`{AsMzEG|>a_JbUMG`oo zET#U;QT72=x01_8Y9WNz8Z6lx6l(dkI*D&b_^svABof}s5%Oiuv^;C(<Xu~+2K?=F z<$*nPHFhLXRz!q>OsCgqIVU0d%#^?73;B+&EUdTNU-@q=C<^CS>Ur!$IIX}-Lfxm- zsoL*_*xah4Xd0lJH~Tn)*~I;txYEE~T!ummu0#j8@2%Plmov1$>?BEB)^Ep=BFCI* z?k=Htk4-S(?9leFlo)PrGA`YE=v78lOM)V!`tQ=*_4+sCJp#wBe`yCVem30t((P%d zcvxAG_}H;Rr=Lu9XFy%w!?5Mbtav71;m*5Z_g&TUyL%t+F<wlljK^29T0^OUf>@g$ z@t+uaOZ@ilGMz-e6|Vxi?xLMb8SZ$pzMIzh@mXAX=ieO<a;XX1E*r7m?|2}us6?VM zNU{PHtGm-@5xq}`MY=~b3pD*Ww(t-wpI!`c(<iFeT!+XIOe!@r67XYWeJcM=#~5)l z(4zk@%VxiAesowml%9=S8p`;)bKhk(d@4Nmm{X95a#gQ5%3?e(6o*)5m^6?JT_>;a zv(-4t&ep|#k73Y8bkF!kr*G1x{ygX5u3tp4IubgDnv4_O9V^Yk5T|79tj&k70_|RS zRbt`d-Q%Z8v3sW*v}-w%3G8!uu)|A-{(>w;4r(TvgA!;Ir2JahxyxK{E0l_bALpQx z6L+{Siw$CfP<$gm*2fLR4=3ewHnXpGA5r?pEGgC*yO@}>e(WQ8WXL$Nd&Poag4-Kv z7sPK_Sm54K&po;eXuj2((*?B&z<aD0wp%`2rnIIO>P0MY<k|b%nE#-hL+C;&sNWnk zU1(QoGslDV-LqR5bt?LoTb%HhJuqnQxlpITAeo4$!=}-csz_pbtIK>5wliThTRUKh zQ~g2p?7C9J-(Ts|;JfZIV_hX4;I2X&6FqWLI0)9&l=_ilAQz}&Qt5E+umI88c_xb! z?w#cHovKu32pGx``N(s8P^g%u`mRPv`35d!x>>SCf2Ms5?bMpu6K)`3DpOlOgYxpc zQM0Sw{6Ounsnb2SpiT~CRIc`^KF(7BC<#WHbG?Elk;f(1Ca-n%n4jDH6AD=xV21Iv zdwdM<7p^2hBrG8-L;z-4<MrYr*N!jNjek5r^%VRv++h6p;&{|5;R?C=XWh>}j_d|K zfwj{jr*+a}p6y)=tY9xoE)geXo!uC(h45#6+*|6Fn0-3yAy$sZGII)dLk6}{>DG!! zrC)zgtps)d!i0;%MKwGAmZ*GH8^g@)5zA}WvYuBCYq`OF4aJ~3gObH}K7L3Ss!jsv zKyme*;;G2qxe`-L;o)#_Yx}{fW4$82tghD+6STbx6(snNaE#{%sFQ`ZiHmEsb={AC z+)braCptZB#v`xqKbOJ02|GQVUQ=cBw)Dlba}<a0*W!y2fADNhSMQ&E3$C=7sL<*T zPUf?VrkHMI_=<486UO+SZrJQ3u~yJ!Ur_W?+DTb<cLAwlGj#!j`UsC9n@=FP@G~Ay zt#*c7lH~3AuE7n<wNG)*TxcLs-S=x|TyjJI1K6U)2oEuo!p0ziBY6^ZoG+NnmOH(~ zy{~PJa(y2ic%*-zZoBdNNq7u{?g`Y|c>M|AmuLzI*5Qc+4>CW0hUaNaIZwTad{Mk| zTsGZ;u+j9)t8G51z66#ZkztAArou*e_%tJtHoexcSfVL~QEmAfht0v<`hfmk^h5NG zIw`_k=5mV(IZww=0Onh4^TX42d@%NDN+k@m_H9XEGMo`f(fR7dI&M@_eJ~HH7d|%} z50mc(ΝYr2#<91kC2>e3kn1H>B&6`E1M0KBP@NN<<0Jp2jl1(^RRjjO5oUaARDP zjC`AyoKZTdKVGK6<c%<XaQ3*2>H77l^U<V&?`rN7hebF)yTO!w3F_<NoLFDCJRx2N zmE`K_hX-Mr@)!<<?XdtJ3M1gPb6412+1S-G7TQsw__3yZoWLy2-s3%#i-1^7F`u2{ zkbnIa=?$WBi_K?sZ;Y5lKL3pe$q5)qFNlO%{qSDG#?C*c{WQ}vZu+zRIQVS5dEfWg z;u~ld@p-0mr&IG@y#|t6*u;nQqBJ255yWz1DXi-(zqrxM59Cz29ID-kOtS8@4^$(j zs|n@3@M#uR2EvY+2@Qfy{f_{z=|Zu$)m<*&$1Epy;xIn5KzZ`82%UYKi(Xl2fH+Z^ z?E3Y4rl4NVTBr0u{>1OMeV?{AtOpTje{6hGy|H+@{`4+6bGl~tO`jgK(^l|<y&@Jy zP=olOg2he3s>Jr=zT<ceI;w>?e)b}uAahjNd1ag!9ZDK}dsV>R8UJ>FlUL+x^~nTd zFlR`iRBbBRY$hPbI4Rb}(3T&IK38bBHOYtLAVZEmUU4oB6L7zfgzr_)_GOM2fBAy< z7s&GerBx_9Hz1&rS(mjq1Yba=FuTUYy8FVl-N&gZM|Zt&UiC)_id#DUm7#tI%u=>P zOZO$*Z2!sS7%fYs5+UE~fVYnL9E?Z#e+cAhHhAR`-3W)%6LB$iyr#sl{EY{=H?H4l z<!RDLr-uxYK07-2X`B0ADu`~by?<QT7hj-uym1Eao(VPHf9z5j&S<mPk4q-n$PSd~ zYHVubbkSMXMyCo~r8Lh<PE~#JhQJwMGLwfQhp=~P{b~;|W&OH{!W;D<8@DxxKz_!s z`QpVjnvrJD5|>xio^>>cXUB!eM7j;aXI~2g4tkbQh`z8H<O!MXaF`|I5TPF0X$Mu* z#o9P<_1?bMnRO#j9!j2b%U~EKNqvfvShM-4#<(ccs>0K&%xjs!AAg*k9ryDjVga|p zVqU=MW0J{S9WCG)C;(y)#FCk04W<$o^xcJ<U%~rRstyA7n|P|xZBYX5H%Y<U^TR^w zY}=!IS^B&XxGt{gUQ&$jH~sr(!S$b0%B+d}4Da(KC6ajW-`8d=+gK)<1EV2y=t4fs zAxfx=NKaLo=u+-9|6Ix}39!&5a!U=UwK>G*^bu!oFKpCraTR6|S^6*R<RcH%zr!)9 z&S^ge<u?gfIw}FOPy6nreF7j!VI3iWIT=X{j@40@WVK5(R2nIfaPo#e<mjXjulBdd zY;3VX+}St_qjmq^jDGPt#t=N$@lP=G6pk0{UXN?VtH-DbDD{y!W~<z!r<E9z`3f%w zKIoP$Nilgx$yH2$EJIwXby*+asUO!7?N8Qt?|nxkYEM_ABrt+t7tXO@PHvHPH`OzY zzf#($({3h;s(dY=sHw~o!66Q4OF=+e*5U6yd?YVWqivu$eCS2I5ZSYvjhwSPl3C1~ zRr|1r=Y?>Lb0wm@2m%J3^C;~`grhDe$!fXa_vJ{p`d&{eH4vgluCd_$d(#M(_22;> zVlX0#OUfHeTDE?c#y-!CdiAehQ;X3Mwb%3O{%EJQc3z!t6L$LP4Ku0<xR;0;7`5rS zx9l=#EM)IPZXJJ}J$q%cTh?tsyI;$Iw!Spg!pdp9#zEv%jnyloYYPHPHXk%|!dg3u z3QdYwQANkN^im0LEgA<t*FAVif9C9DoZ0Gv4PKb6w0uvXounfXvp0TY7xcF@t!H^m z4dUw#-MrF;E#cbipN8OfhkTr08r)3R$q;KE?tyrE(o3H09t$j9(xo<~w`<X~TIU)F zkCuvfY>WydPz*2${k@sDq#Xyaebju$n}cv+Ufk|_anSH?)KViBwO<DPu5}tg*oHb> zD9mwaAf6BEaRl#O(kG~}l6PQF!64Qwct6o}iAht7o~)4A?J|Owv$__dLS%5j+0v6i zy$)Bf;e(#{9EA2Sdv`(NLusa&ifxyK9F$~Di@P|iu2N5VGWaR2-x^2;i0AQxJ#YV% zRCup!XgX=co<3W5oxU`mdfrr&dt(%ttE6L>C^c8k7g6!{RZmMVjz@T^>rU7JeF))5 zI+7lnSB#{LS$@rx&w;-6$11(YjC9x*A}k3Tvr$}#1Gr?XfMuSiKN$8D)I)B*)TFzK z?B8DRQX{XekMAJFY(h=WGK2rdk*<=2akq$nJ`=*zyt*z-!OH*K1t7Aux)|uP^PIC5 z-5pZ7^C_*LOuu|D7(6DLnEuVsXPcSIM^bOxal0wU#LD$s^#nh;MQ-=}Df1JZpL6Z} zt_-|L1`@puZv&gOKMn{5u#?e5<9pVR1Z>jCoN@K=f?%tGxSHdGC%ft;r4;<XNaVd@ z+%(@#IOcP?%r3`rP;h(1$>a+t+kBCI3}2Q@Xx!9$QPJ{Di}OzNa|Gi6_EsFPJ*VFT zM3TW7>o;bOtm!5;MH@TEXPV;>R-p;OQx4`v`1h4N4!DzV#n@Y?PyMwGnkf&hJvB~u z;HpY3(R36#k@V_*i@wokA<^&GiAt9zi<i+TnVUu$0X&<gsU>l<;SO94<9w}^NU^|~ zV{{+<&M7Gm?kcKC;)_)?780X$H^<D>H#Nm199Z_JY?=bVUx-z|){O<Tj8bb5tPM|+ zWtJ9Y$I-viF4?QxUAmc;imbvGtX|}o@p}7+(eKci<!nY^&WdICSAGe7zm4`*bupZd zazb`PwiuS-($dM02uz4jfX#unuuwgd<6umvxZChqhk{mqyBRF2%gbcrexLYfHM2-` zndlZQE8hb#bR1W7GI~I4vWW%P#(}a0;C~q`)D9_jnf00(zPkO17jO?ST_~MKwmK^v zCk0}!8BDHZ^`ho=#7XjpJi5Ai53vE#ZoQ{-u7=&au_7;!<Ipu)g*9e_^R}rsU~-Zd zmg`K$io;^B-@jUWy~iRvYx|R&><4#!5f*TZFD_+t>UT|0h`c@ooiSOLP^!BL*Cxys zc#B9B<~Z|$$Qle)s4N|K9a4qL(gOr=Z*G4+eKdUl?oF5o0XmNShuJ`?@e4~{_yd?x zAt!bc(RMDYt1{ZCH0itvrTn}*UPWybFzKOaY5mk>_-!Cvza8Jyo@-w*m)CNVQ&skx zf*3`|j2N3^v<1Gvr;|I{KxCy;`|MyCZJ!kp6J+K8Y9|}q^W7MnYS7BpQqW|1CAEER z-g#`mcm_L#dCGMB5si21Xd)(jxIpriSt4foPI>G~gIv-ei3eiI7VAe(T++T*R~gW@ zNT7=)O64+7SLzS@Cyy3;M3&(GPUtsRB~rhCr&@QufL7d2WE#xmeE52f==Nz-Z{h;_ z1|XzZdE~ySm2>s(6?m&njPd9^D`1v4=~D($bB?IPd7B8)o2Ue})99TgQ{elYH)?b= znF`<1+eyCx=Qky>HUMsH<zANtE}PHKCZO9j^+qjFmY8ewWjV3KP(<U;)VifFdIo7Q z7gfRS2^nP{9=TOM;k|1^8=kBhbJB3G4^jqbu~!0UXtk3_rtfovZXu!jos3Aa7|1Mc za6SJDy4bO9gh&aBEOF1zuUtfltK5#$k<VjO!YRey>kHSc!!2+b-}wK!+7?~x=b4Rx z-6O+ur_B0s4ifwm((C-BzAMi&W!MQ{TTB#;c9ky;t>%MMjdB39mDll&c7;5)BQt#r zyQ)R(XF`mYV<zthY`Ge432Mm6f^dn93==q4jLPn&zTMMstN|t%Q}Hc+i;s&PyCu55 zb0oP-`p{Fh;>5jtii_cup2^GCdAvd=x=ggO{0(dCf(4;H(t-QrJ<@Q_Z}Y}a-7e@h zXiGkr=`WT^la!PqEYOKu(`B}1G-u{4s;7T14e_qqM$3;xajdXwhG1ARhghw&kL^Ne zKV3YVI`Kbubd_NtyensoE;?rJF7_OMFZ9nm*dK)B5;*++yx(oS7-~&MgDDjN+A_P1 z+IE?F?%;{~Dfm_eQw*47nB;6kH&HIYHcmfSS=!4zvwVm3`jgf~rw*r{B|81n*NRN6 z7nM6HggYQTXRlw}SKDs`ts(0mxHZ2Z@k@<iKR0o|NA$s3Mc3-JH{ThTm^%mWf>94c zwAcoxt6RDZzg%t(DMgP&l3zsW#?~S_|923K?IN6C5Mx+oJV0`JYiXek2SB?}Xifup zdf@AO&`t{i?P&W4&3cKw@P(HThU?5$J`tXs${fCP&S==LFdbx+K8upy_NDjIRC)2W zmh%_)t06`f#D(s7vWn+#YKBQNFhx;JpFbe!zz!dMUUQ4{hH*K9p!5ehP48pIm0vs@ zW4kY|>0J`3R$H5HE`15k(WA>`-evS?$08kXh7?6>j13a=J{u&44N(wBlS+i~O$~Y` z4ft+8KmBH=)zG+~_S+>J*3FJ2)(~CW)PlO)`a8eRmbjD5KeW7u2HOS*Fi)TEB9#v@ z@D9Iee5p6S!)=ItB%55&39Ns;o^!apS7cMxpVVFyN9+1z?Z%D2EGTH2=HCfJ!o9NJ z9o1Avbg(^(W<8ISchOq<O2E}zcueN=kQ~{gUQvbXZo}B~kM5VEo`gm(=d(VPL<Uu1 zlr%(iF@9QX!=~Qacs;a4`orq!hyp*&JFw^X;@F$94M=E~6}88#_|p%S_&<5WveB7# zb?C2M4n&fl&O7~i@^IfZAc#rT>jVxPC~HjKoCHyve5+~LyM9mgLA;*d@gy3iMz){R z(OVeRk17eDYzxQbmshHy&ceZ)mUj3{b=dG*0t+#h4=<-}{;m{*BaUf~CxhO0hg9Ia z9IQL+Wo^(q#3CO&_5@tBitV?SC@}Is<(kr7nTJW_4?B_G2bamY(SW6=Q8`{dcL$#c zGH-V~Q{9sw)3q)<*o&F<h`8=eNC$v$VbwBfTdW3JJp%_GWv<fC2sNW57pz<O4<9l9 zS=+9qrJx-zwQh_~o!+w^K1ocqCs!bnOOGC;Em2G5@>o6Rb3W&(CI9u45s^SCu2iXN zcsS<MCD!uD(G=Flg**GLeS7&55f=v(>w`s#4(tG&vAA;dbKedvJ)z0r@~<YgEOvS# zn>2Qc#hdG@GL2OSe}19yax7BVUk|!V#_@+sao&14FU^o2w?7U81CD;z>Sy}H6olf? zM2*fT+5g5<jrR@(*cC!o!O__y%2Nw<?uBCXt`&l;&{sZ08cH=^{qVvzzlCqML~&V` z!@(%=ty!G`Cl2??D58Q90nFkTuJF7=d>tenCSj(>1)S$=#<Lh!0#B!ugF=n)++O{j zm4Dxtke`I07j-ymB}Zgo+Efj%>@$=i_*;Z$<kBG$Ht-3TkG}8^m2`WBxx+SYjS0ut z7)$PrgqKT)-Wy1|SUZAEs{Ri?o;M$tj>||GI4;#1&G6Q6+Mltt7+|XjT#f3;;<#VD z%1V8b%d-*b9TmD{v{3ng$Tg`q)+n$Q!@X!_r*+;e%`rv_23-+5oTJYv>b=`g8?hsv zFa}Y{iA`MzV|Ha62U)bI-u6Siym6VQc^b1U$=ev3N4PI@w`1p|`!1OtN2lJMS?(50 zFhoJxUc-~GTc<KgQAvA=Tx6+ES3KK&7U9g=E5#SZ8aLut$5$b*H09RHo=+78di$%` zhP1>PBF_iz*Qsd)mD@9NU3a{2Oj}++tKs;CUvh(W2Tw{Ncm!erJN=WU84sfYsnH5H z9+X3*)C<sVkz+grg&OGtddm6bJWxzbVzR>?!jryN#H&qx(c2SF6<zd@QS4>Dy@V*w zL9uYk%^##Ev1%!&So3V1Lw>SPnveAh3GB~W-zNMN5Lw?r<GD)yc9>t%kUF#9hO+dL z0_{aGyVbD%nY7L~9pYw_A}0|=Y_0VNx|r)&%P*f;ld15EQW`EjwBM?x;lN~PJz#gL zaC5B>zG^SanP#YEyJ7)Ufu-y2k${)}je%?b#O^}VV>>J`!%ATH?5Pc!5Td3^|MHKS zI8v*JPRR!{Rm%^{Cj&?U8NGD&*<Rb?`VZ3cE#Yg~(3e)xv!-Yfu77mdyAR%LA17~0 zD815EUdTeBP}{5h{;6jo&h5g?t1=p7VpTLyc|P-J9QaY-`etEuz@_B9NLGq)lM3{J zqCpR+Ap_Tw`_d^AJfIemZ3oV8Nut{<^@TWicTgvDgv8A7!elr4?@EzLco-3Fyz$U# z6WZyfu$f4hxIa5R=}r0NafWGwD0r|rk($k$)+`Wr`V5!tQ5<00p1*avX!kfuLmNSR z*5b-Wo*vxdG->ntG<?5d`lUU(!Czo87OLRZYllAF1B(#$@;w>PdoN8Jmz8p4RoI5; zvG64ZO(cb(`Gz#}77t*wUnM*w?p4l->JHn~OZD&m_z_+CajMST)(<7FS<Ns&-l3%8 zf8XN)?}rAHd@xzstkDhQqCo_hdLL*ac`}D#DOfw7VM)P6a}|B0_RuTp=(f;6#TG$+ zirUH>r=w`(A{o*t>$Nebt|!@+Wzm|U$4Fi0-=)9!g1>p7li!ea?Y_xkFh1}`f9AO3 zW|EFXZLdQ_^ZFb0O5p%Cdz|1NyQ$RDa~9Kr4n1&I4nmY6+s4=T_gVpcK*|N7);+al z1Yd<M<~cIdRs8+={Leuu$9^BGKmy2o6$u%IQ3A;(x-I+?1n3I={rP_{`hR;(@#El1 zAW<aX#KElZsb55FE-xN(z~*b!kZuelMxw@)NG_*#&5kbLYZRc%?kNa`VIpqr+)EK) zpw**&fgrmNn)nHZJ2HaE{s^EXjmn8)lsxN544uds3%?$~eH_c|q}QnhrVTVW<VgRu zL7-JY+A`By+oE{hYkzVOECkgA-Q%BLN`_}xX5xL((%qVbq^xQa9yAMXMRNT~JY-<W zMJ|7&=>jg7$VxDm{8Uk+%J%K$t1KE{Eq7Vq&tc3oOS6J@LTEFbf!3KBYyUs~YD75J zd>GkAyr!MoIv1957iq3ihl+X*u`M>QD7POVqgJY3cwl?w%^z04QD3Y?CF%e1Dud|% z9(_b9g-g591>nfM-rPtX3CMavC8wTao=MocT9gMA?oX^YHmN1U`G2WjU&Zt1AIPz0 zy(v%`lpotjawSjdO(X{fjPJmm16x<e7W!9%p^E|@#G;SsT=xMBkW{4P?nCPqnHFpb zfcV4KN+>&7+oYs`dx-*w368b_cMb{fC5AlO!$J|Mc}mam@aNL9eq<|3f2!E})OQC$ z{8Bwio~go`vRc++JJv_5*IR1x$Fj+|-?+z(n$O8fDdifEp{IFA^Ogg5duQi2S@Ih) zl{C9M=CHM)j@8lc-Yjc%-&8rb`JC#3#gs8KV;g`(!*zRVN3E=4KbUyv|NG*`K^IrH zq^}4ne;Y->xl{Rs?V$x+NgPMMbjPQ(6FxwmM3&+c7a=l$!r#9|$$V03K6`GGd*Af| zdfAHqne>y03~Igc=fJ@DmcsP;pm4`8dET^~U;RsN4K&Fs@?D@Z*6)g$`6w|5lnQI9 z)#tZ2S7BUNzaXdY#@?*^9WdB{+2HlI_<SPZ1U4O0tFep(AkX`!BoKv`YO#`CJ)at} zAki!unCB9Y&H$+KYnvsDk1h;O2aBH173Uq#l`6IKf%F5Jf+?l(Ob_8q5(s(84HK#0 z5>5Nzp5OA?sml@b<nk>R)1UzngJu5u>ACi^nO!vw;Q17Rx>{$#k6jnh-cd;t-j@~2 z0LUqw&^zXDSp!;5wTipKkH1`*ij$N{e>AMqv*<yG0wlx0FVPQ2gDZz`e)ON+>@%bv zWIEZo-<M$^+_Z5JLAe7Ii&Q@bUzM7<QA1GPWLiVZix;ksjWnebY2T{Go)Ya@_T2)p z3QBj-x-2zNUXn}2D!N~;qinP6<;|DY$-G|*1-iqcelqOh-Jge%5_TJB^cRlPMpNZh zWl9JLzF(rkfY;V3!Vq;CPqeTrr(DX*CotE3k-7r%di?R0Z#IQxGE@|}xXt-GY%xs= zX{78zj+=(hJG!HCB8@EACXJ<*$tBZv7m=(%jmmw}AKzgJ>La^EHXF3%zF-o*)!)P} zebhIB2hG-Y^J4@!oazV5(uKjhjfom8Hb(NaKk(AMWq+AJN(((wKPeOZVXBm$iAyL> zKmAkKVC91>ucM`wi`iV2oixzG=q5PWErvJ+?M$-M#n94j$b~z&phj*;N7#L&9QcoF zN))oH5241-LH{C%0c~2a;TWitz>IfOYFE<xY+PF1>HA!jo7%LJy9x_+L6K)kR1IMy z7(DpUu4l|RrFgRqb2Ul0pBj<sfY;@hGatQcZaj|$idvbfZRAAu==g3kD=LvL#@xJX z&G*@ACy{#18g=)#$;tUo{_qg!nF<CF37_t?AsQ=8m_|7suFNoYRg0B;?DL8l{!2Ka zP3&j|A9&fM<%~zT(zK#gwP=&IBgpdw^C;JQ9bF9wWcI<~Zq8)nUG+a{raU!cbn&>9 z>mJQz9H>8umD2Ti|9&7YpRXT-M)klqYDdG_?m4QS_vBQ*HIkV{-B}&E%U;LYpQf8) zqi#eu^OFt&dPd#?_%xaun_I@sLpi4kO?|!US5e+B?zjQQ)FY{OGp70*eZ+b*Iki2M z)QlxzIG-c{)jc=ejq-Op6@(drTq`F<2LGL<E(1G>1|IqW+=1_7#|u&I-B_8xRpwh% z4ZKIwCPvfKgbX4tt-LXFEKSk3XB)1>@fS*7$@NU8X-O1gQ=c9ZH{8m%XUW2AI+o?K zZSsFeVuk7aIhFL%ioS*)VaOEZP(IOe3er&)m4{3I?5#S6v_kE0zDH;b5V(Y(#{>^K zqX?4<?u8N%sVrNjmtLdmgJ(F@GWobD_nmj-_%5KGhU3stqpK8&-X^ELb`*+U|J_EA z=R7%PbMByK=gy$zK>j$gNGBXFxWZ`Ddbo1?fr%+bDcap_&ROxJM_S0W)$R5sB#aSx z<flAmk8NnzQEev7c1(d(U_>WB4(odYhet59-zfdIz}cmr0v2?=X9E9`55j=?Bq|S< zN=%?99T5Yp%ZD(;rgM-&w^nP9H#ZnzL=@*TQo`w770qiV7vS?Awpmus`6B7~?VzWh zEJJOh=n<7qB~=nV4WXah_{K&EG3g9P-uN=|QQBJXRjQh2|EErSZJ)2;u#I@uN3`b` z*uw#G$mL#X+H6x8^6m^$)$YvVkkytdpH2JbQH*qSg2GX1<L!RW3S3f|xfZ)#t*K_! zq|hMqG8yl>CXL$E6ZpJ1S6;-(qA8muosj_HHZ8lDhQ6q!m;?c>oa^$@Pg4}Vc|jT9 zhUeY-kC)V$i;vesiZmMR#keP4=K0hajygxLnUXgAbv^+#^nK*5pkdVgsjLl-3K-sp zO~Z8+02lS6k7g3xS^92(+KDrkJsYHYUFaefv%NloXn;s9YLra9*RN;T0Va#`fZ)UZ zSINo*go#Fb;<qKbA8EFjWNrHAHU?%YI<LHIq}wk0(p=}#(Z&k19tu-tc$ANj;NjA^ z-F68_xR|D}?OZ2fgZ{t8?TqoYqvs1mok8sFYajGBQ_IgLwEWR@agmIc7`=;D9B$gm zW++>YGr66#7BBa!Xo!w8lJjex(CM*~l(oy5emiK+v5>NQ<&ia<&XG}fdwsmhTvlQ% zTW4dEbAH={Z}=>Fx*>diX;ev*TLB?y{906Zo|K2(RC}oUI>=m1t)`CmJ`)654*^8r zVx;)9x<DdQPi-heBiexMb2-?jVg@-@*RrRs1qSG%ziE)T0HW!IY1OkZ9-ZDfCm;i# z_ARs&^flSL(22WTur~>RpiCNFLfA<F-;^f<f}eNi?9rlR9aD2OkOkfo>8kzkkFH2| z`wkqx*!F6on(RBV0rHMB$+l0~#S-bRz4*`8<e9!c-iB+YsJ&UN@`rg_c)kk9P5wUV z;0VeSP6Md>X?`m34<gwp-=)7Qt=ytke9Cbl0=xj<@#Cff$EO})?H2#1j)w*Fd`lWF z4e!Om7!uGemlO!NJbTZ?+BG&J5T*oN0W^?a>op|4IT~u!S!ST?SG1)HuHxAT1jufT zd^jDU<Y@=}-LO=mMgEL{JR#Rh0PZRg$wrLk$A04~kol8Xf``xk%5i`0ld~R-MR<q! zd!~|^;OF{4o~|5+)*E2yo}*1T1CZ{Iw||<P9fpBJUMfqfY)9mLYtrWtdVHA-l>Iwr zb2+JlKU!R_Or&=Eq&K&J%(FnR<|n(u@43P-H4846ng65v)zoVIsM^RP<F9MKZ0z6* z)DWJ1Zq5&%bQ{*Ny8mi)(Q^$B@{|n|Z<pJ?W^fQoM6yKeA<ut+%(4#2&nrwzI^qpp z|FMan+WLLFpfB3dK0RandsZn7D0k-g?TgjfZM?7nhG9Hp=u@+_dHfwm6GOMs@(8gj zH*8@7#Bclh0koCq!Yw3tXH*o!eLDBt(hY5;-fqPY()@+Yh3hmiu{(zAR15<31PMjA zQc3Szsi|jfG%Lt|-X)>vQJ&cN?y$R;z!S!4%mU_aB`ZygJ+y8l!HYk>u2e0^-|MLA z?2lvAvoZAUDma(A`1G#AOx|J4@jsH`7tbKQ@axH)`#c;kC|BV{On_kAQjj)$5z<A< zi!dCZMcUu0G+Uvek5mHGf@0=G?~3ItlHOiF4~l`&bqn(&{f+g3(uu+2v~H>9*1%JS z08qnN;Tvq44nj%hE^{1ZeSc3@9)K=7#F~C!Q4TTyY781GbmBo$lG+LQzquoC7A~aq zeenXk+@2|o&zs8gDu^c1ypu6ssHN7*{g7EzfzKp|jO(W2fD%#yG+3cDAVf3Hg2S4f zqW4@dq;HkLTW69QE7{+0tk$h=56f)3Gh|#c+ZLaQh}H^&lq76HWcEZ6BPA$-1$}oe zZw5ALj;H?n3W5ioDfy6btD>}{?SvdSS)+=oN|u6fzzi;6QLm8Sx$OGanMjWD*FsQw zueVN#0~hQ!B~j|l1<<u3LYBAnZ-m-wqs?3a#NYt3LL6yz+R(MHvyD2r`zr(<H#3+J zykdxPFkg6=9LTyxKv67MDI#>6!;xJ_W-*$Hm(S#N`TF@LP;ShB#M+P%`a{uvKEk0D ztx$afbYw0OJiR;4n+)V7=j3a?QRf1Tyd_ted>qph6f&ef*5}^}=m}VWT<w*bz5V2* z_IRNtH^K+`jIXe{aq(zgI;I$4Nd>5+3ywDD$DCP8l}LKhz8#r0yu02$L5cx0%SCE1 zRu{iUF(??gC`EPLMsrwntPel1dlL8dMa2vv#q;!AWlme=sEz-enJNCzHCXD^1Q3P9 zi)hekprlVB{-g(OCKFBy4sZ41#Sea)@y~U$TJ?^a{#uk^iZzwjmAxAymP<uM@hbv} zF-kLy`Jg$@yW8VZG&@&vmwd|>7!CF(UEv4pB2O+-sozNfsbq7hpU+t(k>=Cv<FDX5 z<dPs}PU4SD^$a*ZZw@N@*zmT0z2@fIoa!B~lX@Y;p!sVGmi9ob4zrug>P2JIy-T)b zCal{UvQ|jyaDTQvg*LjDgt~i<`vru|k>9-70&KquMhr2D0!Abk;8nRCE-O*vDkxv* ze(*u9n1f}`v-8yjqw+q&&08>MJ)gb`3f>!51PTo`pwN(#c@{fU<XWQ>G0mx$2_)T3 zO+kaZk8A&VL#WH82k4O)iorh<7lp(?wedz)d@(ris~5fz8M);LFc1SA2HfW=iv}-r z`N&|47HY=YoWutL%5c>kcK^8M^%8JMz<V5k;dx<vmBB$UCcW+W2Mi{xmaf<*jg)_H zicazms(-Tp0B!^c_C>o6`R|JwXq2h2!`}5LCsUl>2Zx}nw+7QeyOe)BHL!gB)?P>g zV;xVjA2&tM#(nwq8$ZS#ickqC62-(F9_~L%&hkIIA+7qMZNSclSoxOiAJYN&?|tTz z@BQ~Ig1X~LjtvI!=W%eXQG$;DhUr7=<1_3nQ2GZ;-{<M;5<^|JMZd<kXs;M)I^KJC z!+#1`;illA`@i_#{~Bmz;qO--E+iNS!+6I+siUh>nr;5`LeT%b0KbO#|Nm>?qnZ5k zo@CHRvl`b3j`<&h%70$*%LAKeD}3nu6x;!347#(vsvb;`3l-=~^`FP${@Z%tPuzoL zt&rJ4cPX{z84Db%vZCX+zqrEvFTwuSX@B+4rGo~RO>%BCw97R0!eJ-=&0zYAqKSPb ziBX4#J_N=U)X%%w>{+*B_p0kIN&efI-v1^@-}?J<_ft1HH}n0#%i-W8f+WdJJ%0-G z*~%k{GjYE6cN@(M))&i8l6MmCCBx{dI;txjmjS&H%JjB!*Kg(xCyx1CCBueXQYd!T zh}4!XesdP#MbDz;Su3+-B1?pRUj)_B<JUiZf9sma)VQ2~i+{UK{*Ra9wM}f&->Zee z0=^$Ei-MR4bZHtbzyomc_@fbrXcJxO24m>>dA632xq1x^)_c=KqxQ=Xfgei8>C$nY zD`|@;j83^OceifF8n<Fy->H&L5_uQ&<~lWb|I7@qfM*jVqt`2%3^Nq>Rn^@A{e1Ds zP0`WVR$6sGVD70B#N`~+1hN@Ksnl5gP&WFd_WdGOtDdB#sq8PUmkuQTZ!oaaD*E~R z8v?#VJ*0y|p&!4G@`#2Y&fe$_i4anDL<2HzwlFYaS1JQ@gtRZ{F_bw!<w3_wn852Q z=vSJR!8<i90U`+~&Ox}7bpLFkxs?-r@iwO-TnQ9iHMvK>HaI^Qi^8c1U@>Bc=ln`X zIL`AFjEVowSoMJ38vTo@H?&%Wp?Ivp*n*cO?BK2G7o3HHaDZhznB`khkq(sUb=l82 z(E}*ye@pB6xz&{S`V&@}$Oho-B=0el4K@4{@xyu7dF!BG%fUPX3<lU_8UVxkBmi)z zj<G`hcNGQjg0=tMuoL(Gc^QQ^^fLS)QWr3X`yOqSBbs2$lvGF!0|-p_MM=h*3eG2A zSA}%-dPgC&tTz>-_$*TWyz){YJ*$`E#_6y?0In(?@PQPFN<5jhulE+}Xsdoq$RbB< zAj0XWcc1}$GVsYJ`{LW>?8f8O)$fnE(Us(Inxjah?fpNc7hn<+Nb9cYz*}Hr``EMv z5iev1*?%LPrHZ<}l_u)BD@~ON@2_6pY%JXmDT11Nr@`x#8khfeIM|{Uvf;Bx=DWhJ zU5@n}oyRDIv9FLZbfj$)--;A(xT`~z<^J%}pR>F;>%|ET>+^myD8nmEYVG<<fcDv_ z4P*Dxf`?M>fTY}*9LODbeu*SM>PaZ>ekTi_ko7`?=g%(AgUa~j2$i?#EBXTsk{m$N z6(TDf9^`f2S{N6Ff;$gzYpgeG4I)&W0^3588msqQE#>n}R#C5(wg)~QX9Sgg@CzN) zr+(+Bl9RaJl*?){UgS<R8$1z1Kh0Nt#~~R>Q-<c*#=4yCQ^CG_zIY+<XNDmnhrt(& zc@$&~l95=KjWEpDD|=TRq;vd!+s^b-jnlZ&6ma=R(<T%aW9FuE$M;AuxUJXVNQlw0 zDrUYv5|E%HK{qyKUIKsv|CK1U=1)(smhy!6&1?5kxge@eW;QczWm;3@i31qT5-~$3 zkOSp92JUUWdYR~_iaknVc>*)J?P?HM@<z9l=tBRDe47_d75yo#JRbd$<zAc}t$rIq zI4t96y5I*BR_%BS>)gb<de?HtZ&jvl-_-xaK#nbsJNp(X6@D8p5uT7WkJoE6&!_;# zT)1Zikfb6AUBZ<m%F?>tb^A95Yq>QrN8Z2VZYS{9KO%oitfS9-|E=*yL8OS5XAP-* z;Jynl`{TLOvC66C;ukX0nL_=1d9Gf7$kJ#PT#5ES7tk$3FDhCoq;`J1lv<LhN3(qW zS0lgv<F+?V6u(yq*v<(1Yhsmy(M_l)u;6SpKu9$JbM=J_*sZS@_urK`iBoHo<>%LY z>|^mi{SsbC=u|c9t4wNI2r!mQ_p2m4O?q0{YV`4p+86}oYbzVa?MbIv`*L|)cOiBL zn_f)++E1MY245;nuuk`9>*-nH9isRI%sDEyPZfWui0^7_#J8Bvfl+5S-8fmZnGwJN zIa%Ft@xW_ku>3S_a(gXJC+Ze0s%Ra7tpI29+6e9I(Z6vtoRGy2)<+B4hhN}AFdGWx zR2<1AO?0v>B3#>i=1;!nL`J384y;0fuMp0!=`bE4<1kVjXifo5eE+La_uu-oTosrx zmRPL;t^?RZj8j?#8u?<Zh}vKT-O&8ohP!L;vtFS&<L&wp9+qXtzjF9f6t2H=_$NN* zLa6|-_JsUAyAqCyx9{ea6s{ba&A2yR<14Me0G^cT<HwBW5P-j13$mwD|LWo|Hscsq zkv=Me^eH}Pj0-(FfF+JJWBDX0cPow9d$kP-oP7!*_c=@kiocHj>n2lC|MsKvDTUZO z0LxSF;lb`cwL*r8!t7;;=OJrR(&RwoKIrw#!1EzM)W%S8mXBpTcq1}(c%d4@VIJDX ztuX$)_&j5u&J(@1PWCe7Oxp!?qU3?d$=(N-z+qk5p&B*B8j+zZ3}Q5N9%B}^swthA z#|dORT{>Oimc2**eQo>N@xrNN;;0^Z!rKhwI`XIRZny)b@A^Z6*Dt^0)#!4g7#u9e zjn*U>Xn~f@(<l&TpkYWOc1wfbq--abuhrh;{Hv*IdE?4D>_I!wPa;HQy2tIQ6*xp# z&?X$6^&I`y>kc2TaYD2pbm7q?i`J3RR>%^PUbBmSk18IYRC+`p-FRegw#=W4nTx|G zu$H~WUO#gk1L4`~_84WsgTBX0X0V4N<9#e99~h%F{})|v85PI2MU5u7O9&oZLvRfq z971pj?(P!Yf<u7d7A!!5Htz1)xVyW%%U8`k=YIFSH{KZa1A5R^)UI87uQk`4a~Wam zLeSMxYWN|YpIiUh(6=5#wrisIgfI3X|7miTOY4qu0mvx8=78(eI|n$3COnyKkN>?! zeJZ=64ypZG=V&(t$iekKNCRV++rC)9Gypw>3&Z{k{&-Ox&H`z|s@PH-V4TzkF?519 zVVrEo0|0M~bOoUBGHMW-e}7AZT*ZaWH@D|<U@05cuppO{<~@!hHKpbD&C7Li;N6sF zn1gfXu*HBg#%$Rq=H128a_`j|w!p-3z_RKBk4#)>ff=w3dFsALd?0io(o+u^`AV4* z%xr=?An@zsmlfa1-%7ggb+!bL1rOG0jqdOmvd9KdVOQ-iHtHkNiaJze)=xB`Z6Q=` zdp&Dm9ZXilOzd-HIR;lUe1)E7>8)kDiyGT?I-Ym6#}k1u^u~17a9b94tu|+2{ic(( z^Pk?|&1v2>CQlcC4gf3tj$%<?nXEKv`_j~0Is`D?|FN&4BLp1kklVM!9d!Z6z_%qo zwkq`&b$Ol>-%J%)qD{f(LI<Fddtx~HbF+3Oy0!AJ7OkIjTh3%LY}W(Mh(%p0e7Oj9 zm&Vovujys(!pBVaBob;!U_>bQgQxQrJ5bJ_+VZ>|E;FRZ8ix@WKDASSg(c29DXkbT zX(HykDQ(UzTEwJ&nAeHy`8QC5D89%)|Ga*I;dda>biL#DSFjiMz)GmfUwcA$!i4-N z@eMNzF4wFLw>|t|ON3;<S`Onu&ch;OfJ?xFB#a7^GL*fP*n<qr;xeml*V#sgGOC#* zX0CvrWB~&@fc@lOKH1M^Ow^Q6TwU=acAF&Fj9(i-IPj@1!vQ^LF_ng?cqI*QX3Gv# zA4%<Rfz&>xX5#t4csJah=V1gv1t0`=Jwp(O+_SA+79qrt=ODsg)vDefRXpxyVJn_w zEPFHPh9?2QI*&WlwxC}HTXjBH!PxC=Hfv1C4fZL!^EdHv+_b*?5a?eME0E`+i^|$P zj;hbc?=u`gHBtHxTh>oNgMvGIPq%<BOc%iD;X4DTfW4%NkSU5*1QBit<BO7c__o58 z{DD+sA7Bs}+EAeOf!MmLbR6(be6M(xIsK;&*QyQAfJzu8gI2lotew?~&kG=dH>Sal zR~I@RfL&2FLsHiTZ~Na=k2arDd(`yk5q0l_=h$`hk~9a(npzDVw?=f(04FAQ*Pe`_ z<SY>h%?qOUfx7M-UoHV}G>hT>(9delL-)8jtaATAd7oe)dBH38r5Ct@3@|KK50U%b zG61Na{Pj-UtAgFq!ci%dT}k*H`QonuBN}}(aU<4r`YmDzxcdL)Y#$<C6d>(;Xl`~o zFp3A11}D_LA;35a)ly0CcPwdS9CSx@)a)w1)h&6aGzt*))TX@IrKcEEjcpun&4r;O z^tOIHmpY?<8!9T37N}_~`C^C=mQvVBOp!!p?tM>Jk_5I9T-JSVI#5~t7JFBT<pPpn zbp{CB+Mx>DvT0G*zf=>U#C{+<%m8m4V^Wbivxr-;7b!y=m*Q%ou4Cyef=e$i0PchA ztgDx;_y!KPg9bJKz5$z7XN(aN4G@oh4fgqkjA5k<5WjH}4^niJ_OIl=`Xw`+>&-b- z{p?7g06Z_X%Q&W^vFt|UFcWAJnJmXQ)EqekG05Qo*0p{c3im#s2O+OgX6yvYNSs@H zatObR`kV4P0_&iQp$WYI<0fzd$rZf=`f^_4Wl>1H93A?@g6)wk2{1KvP5dL7Hq~ad zG1gvZs>@h(PNi<32qQ)vE7#9`F!s$txH6}B*5)u^_p@1P7}d=8Rvn;@p52aNo43eL zf?hHEy-CNY$1hd?<Uk0G5lhx=E|^9esmf#3u-_29!QK7M0O=M)5fa~E-8@8QJl9Gi zjDe{LXc+?fs){bki`V#UQae5WDie~&DeDIXMSqwQ_dKPd9OwRn`IyGzAP-klY>7yh zqurnvq~&@N=ZKvU+-QBm+F#<lY$a$EbM=OBHa`?CU)1bxu0eM&&k3>rG|}B5k4cU+ zdil5nI$|R_S#^5_Ds0`+O2l>}A*lK07-)#{?-*kq?HsxSLu?H|h0P!trRm9sq*5yn z$(-HslwoK(uGJYE26}Yidwy|x4&vx5%m0G6K9E?2DaI=JCG~+IOe4vKv{e%%e#wa2 zdw|w?LrH3oKfd?Jml|b(A~d>^1Z{biM{U3(x_-i4dvnfx$J%H}#LN!I*+!>@ny9da zAg8#lCuO+z=VlFwOE1u<<6q-5;)V;#B>f71poYI=bT<<^_I5+q{~BME3J`kzIz_1T zSWWgg$dL3~t`j#;TSl$xIoJe5(?Ie5mwxTse$x6Si4SezmW<18el=jWjNYM3ST&WI zUo1+rE*40nQP?y;#9k;UBWZeDFQZax>ImuQ@GVb65m^t~guxMkSCp}ut$!Vn_fL*W z=ye1Dx)vS&Mx-Orzy++^9T`6cqdh;KP48myynVac?Uh46l9f~F{&D*=bBCMl_G>2v zts+C+?)@K!9SLHo0Aq2An$xDWo1l2h^gd^j)#m|28iV7}?cTiI?;jX{uM`pf?1T(` z!4ha>uffpZI#0R(Pa*cZ9AvK#2#bG-55*zzAw>?$nLofyzY3dUv72ixRuy25l}3P? zI>7YW0Gv$5>~G#yAsdi~f_eimJ17v-rG}*Tqg_y-zWhTQ!Fd;3>%GxFT&ZTjnB@GA zsZ>Ym6yj+M&?Vw?p7w`T?adcvGznoRUqGG#z>G#9l+KC3aowJ<T>hYi&KAPiF*iBg zH(Um7?7JNXcItWp$CGGgmIHP<YXqX46IIAoJGQ5B(J>UZ0z!ZVIHG+QHBu04OtqW> z6rwZu8xHxUjI2+%UeNf=0;%Z|jZ=grohbn))%QoDAem?y<dzjytC5YA4w)-d0#bJa zB}CVtSr<)q;kUxhn6ef>PCGqvMQq(}2k8>@RC99Pto^FQYG_rf5Tkjo{r6`xw%-ZB zfB38tKaRHa{e@V~w!@`uO>nx?Aww|$5aoFRdb-;ukE5xwnsvYm^BmZ&6_7dSwzmPY z#+|yDCg78HGuV0z-6^#Zcdi_@j@C}gM=q@b$}bdtH~=aVI_JfFANZ?!EdSZ0S0F@` z<`DyJNa7ZP4?MB1aJTRa+6S^Ee%F&vZtf_PQhQ}eOKe@zQjN8z8}|%^7sy4YPrhZ* z*vTVsXXFuz`Yx{KK`s!_fla_nZrrVL#VsfTnsg({z<DJg@v|?kPE0?Yn`ews-{!=q zKWn|C(juCJnejg!Q4kICGCKO-iCxwtlXlNLL;CIxq$>d>(4QmpM-hxpv`Fm<H6SJ1 zXQ9LjA`4y!NH4ln3(v%35@mFCnT2ITkjZY~9QV?b5DmcAQO!((SUA~M+4g>SChPtE zT><dthFy|C2!{3nqOXZAe#wH!^~GhcqG@>Ny?~q0HQ5Ok-m<mkTwVH17xPD_?HUa$ zkH4NA(kmi_C*J;1LCqM&i86=Z-GMCr7v|J<>nzx*_wOk)&`jrugS*Bqe2}!&<UWU^ zU)rLy8o$BFZ@*caw~C9v`K!}ab4SzFnk?{$h)2>JoDL3M7t6%vf#>sa`*2QFyF%&1 z7X1E4Z$B7~jdl*xwu2Hjn)n$N)I|DBWgq)<c)xdz_$(QL6Sl|!o7RM1cb!+05z%G- zZO{@z2gFucKj`OQP^1-z*@^2mwFbjEP-5f7uK&z~?tcuhrzD>u+)r+Vq@sBu;!4ho z<CJyCAkp;pKyl(x%+q!HpEJAn=rl?LjBX}1l8(f6DJyj_!&q-GV1?k*jR9T&+vEjJ zXYOVNO}{{VCvn%2xwF}^jSw1jKz#Dqx6wC>fR-I%lEJ-sHV)w#&jCDRnCc{`W9k4r zYS~|3GLRw#>DtbbbqspY4-+eQ+1{7!z~1VEsUca>g!#P4HWk~UNPt@Ancv}~c`y#K z9r2pnWeyQ#%bVl_*l;e!G~3x=Pu+-mv?z4(tJp$|n$eS(CtEE}UFJwhq?Y6D<EoGT zS1hI{2ZdKQ9-CfC@-F6Y>O^`w2+DA+^hz%)<cJPhsb9rU9EA^YcamP{djXzNN~F~j zmebPLVUE-!>E}b))DPHe=$#w^cEVq@SWL^ikftGA^rd91wL%~vgD3o8a*|h$tGm@U z&U0nbm_DPy=}fp2B{!T>W!~-Ywp1K*k+R+L`%Nx5zcGk{m(%`)xg!YPUt?<2GB;xm zg;JZkM+WDM$CX@2rDG3T>CHNU+SKt(580P-Z}fjn6<bLlE2TgA{0qx&1X2jD-1^WD zx>RL8?|K!WA=Kzg`eqVVm`M1F&&>~?5EAyHYad#KIIkE7ir$}M@x8Sj@EW~DL?_rV z>!3zl4a}dlhB$VkF%3j<H=KpI4|(^Uecoer587ampIIenZUq>?XE-8wMVQ;|%6Hc= z2O_T4{p>#pd20)gkNCD$_XHCutIn=ig$o+$i-$VUC<o=zS*r@cXxq#s7?+B)n3K3U zXc05bq*muY-*}d?2#Z?PaBSM`fcXMyem-*R82(|h#3DAv_{3jC<Jyqlg)6?p{#bX{ zSqtfGG_7|xpx)%ZrCrtWFR1?iw+v+oGlx|wShdnX;XdCfxpU;@9p_JGRcru+TvDI6 z!Z?^{2)22EARG1z!i70A=wFjAU~v-?1%g!!8914j;<_1ROn-fX_uvAgY+gZ)6~h}T z4#g3knnm#8@$}q|yt->H|ATq^+ChRYcnqc&#wrSgzX~U7AsPf=%;Xb(=jmWvrk!^t zl2946Qg7l91WJaa6POj&R@jqN|4TxO`V<AUt-#s95<Pzj<B%YcG*n*Z3?<;al*%#p zKmRHv1`+W5#o&#vUOJ1kL8>@*gA%Qv%>T;?gPa2bv-CpBN@s{!ZjkjG0~YeV*Awzq zF@EZ_Lj>0Ur4~RIb2P{j;m>;iGO1P|15mp~p*ID5AJ5(S|8e;Z;54weFr$1gn*bE# zd#@~LIRVz~Thj>6|1vp%q(Bs+FdG$0_IjyR*g;rpTIZO|mcaK>_<8^L@(2#VY5Y9> z%!pr9gRg<_DZJu%oAO5w=SPM3{}^lmjT-!caBS!IfWXTg)<TB9_!c3F1E{72r4#+{ z<zXNlNrVK1*drmMkp*Fv(0(WQA*0BQ+@+p9_|Km9g0zo9ko9l!CL}Kdr3JaeX-}8| z0v}|P8XK)dM}UtC`2SEj)uSk|76bjm`hYIp8%s?M5TCVBESjH_k%g8tXorcA_SJx8 zRMlSBAAbP!?*#}2RQuU(ls1j7-uQEZxK9aS++pcEiSp0D2KP8-Qp)ZCFbYBmxopC< z(c9X&spn5u+MoeO>9bTp<*vaa&z<z+(z0^Pee=W9Z-FB)&^n&-b807(H0<!@Bsg*7 z)NlCiZ<D+kC7t)Zzx{%#+`Gx=A=CZ@HoU=XnKH;7t1<VzBZmy?fABZTYVXS$J?E2} z4C61;TZ%a*Uinjh(w57fZU(nmJzPl&7tJkQ)X1X?E%%<m2~(T80AP70b=-GebZ;=$ z2EdWV`*(7Gmn+3$IKC3Mpjbclkn|V0jsG2NJyitAWe<>B8vDPkNvPDmWD${xuK<+> z)_Q3+sZ`4RaP@MjH5y4g1B#Q3y8l<-l12;Hwg9qnKn{gpcfv-+lXJZXv_-dBu73|9 zJrrL2VwZE+@)}dR#B*eT+%e|RKMpOut?J-_EOq7~`GFe1?Q0;z@aGG8HWhdQ9I4+( zw*<%NT>-{iyULFohlBEU`XBl<OG`K03Zl}vU8awgTjbj4p)6DU)M0*iMQH-&7L=iJ z7!&55mER-GwiEY_$IoCD|0<+>N2)$LsI@x+p&XAuG)6_-vrSsr&~xSZ>zmh$f|B=b zSX3&Md~T1U5Njs@7DSp-ix)529IPR{g~j!L?)he&*_aY3z5Jd;>_@0B%8wjz1Ayuf z*_nRXzw2{6kCu)h^9`_t{HK=VDheqTmqF~;W?08p!`b3sDp~-xHIRQxLR4G(JIQDi zjxG^?$<Wj#Er6(uKqb6z=Bx-;S<;CBzsc3GwY%DF)4(@r`~`8!QIwyve_rATH*nOW zrgHucdnk9b{KIea@bcdR*2c!ia?5=HXW>?Rru~?|y~t2Hl)(>w3#$eGM!U7;@#gup znj|R?gt8$!ckt5a*bnkDt|SG2*IX?GQaE*hQ4#eaacF-eWPsaI8W3TC>}Nz+3ps5l z08&Ogkg@G)arD|_F?o9yGU|McBze$|$^Bg9XsnJq(Y9<V*Ei|ID3u^V5*rZu1`O$q z(zz-KaRQlJ{{yhK?u@ZB6Vi(VX_6L!LJOoLi_0U>xunW8ViE0Y&zwy>E`!vVk%iXm z<P91au&VkdPzB)>+niP(0@iKdSuBYSAa{Bj#?Av`4ZQoNGICu8eZSGzuL+l)?6?sV z(#W34bTW{BZ=XoUcm&wOI;6h1Jf*VQtJGTJ7>m?@Ne2AEA>_58G(JZ2G1CV@4^t>E zrOWP98V{pF+5jUdjY=sxg%}2P;n~E4`{rZdFhW(Yz=>C^ru7M=Cwr40#}V`A`Id;k zhS$(!DcL3M`jR(8?s$RwdZEH|xbEvU*YNqfdtsEY7+pDHTu}VLtfPCJQzWJ;?@9<a z*l2C=*0HG!1?@1lq9r{h^_DoAefQYz>$*M4*CPK0m;T9?;@&0zZh8vXiwV6NDbbaE zSAioI61c6?a|AcK6#cl|o8hAs_s)d>_LoGbhA{WV_JVEh#!0hwcRWJ`zO*-%<#D|w z$q@@E^jnbrRO+=yryElIv8`YR`}Mwg-FEW=aj}0A_!#}DW*-kUoO-8@(O_e@Uq3Zj zdifjC+01O2e@XtPVRq+Koy~-~_Tt{H&x*csq<Si{PS9lP8kbO@l&R<MQr?H#4DY2S zT$iOZ^}^cK@xcqg_}^h-psmW%CxvCZZ2vq6`HQ&nUFN|`Oz5TE=>s>n@l<WGn~R5! z_U*Vpc1*IGcJd(qiY(@0@KO24sXARgr%E>)F9y_lv>4t2HX4x}4@#9^mBd|`6bByX zo`Xr;g$uD1HNmyGR{XTr*`c_hRraPi^jA-pKe=&!=sE9RBac-(u=JhxVvRZXQs%bI zk(_j}VGM_}wLH<!4xaP-?+U)@_pGt4(PA040!f9>reSO&vCvnf+#})msO9!NdC3|^ z=zISpVjZ@|ip9LvDLu+Q;<P&`#$}lqIYK=5{M6XvaqKgKBY}T`ZTKCVd}~qHv+vwg z7OaoePpTahZqkT;!(e*cVCH(&(6iobOMXs2R!^=$6H*mqzvi|@H2>C-ftq@SpZnbP zp(S9dDC5S(U9I6x@u6z(Oyg1Y*Pz^+5Qw3q0W2HgOAJ#A*}_3N6h}N*KlG~gCk@<8 z*O()RgiG6`Q<@P9^i(e<>BQCURnZP_``<_6{y^)<<xz#D36Girq}Oi6MsE3|1%*mz z@aH|~3ByzLu}4^2v$&4}5jtFJ^O%vWuIgg%3O;Q=AOh04G(nq?h{w(1i(6LgzZMlg zw*tW}@la1aErwMwXCtx0y$qo?(Tnq3^K`ANq3vA*&-d`U*U^t28%1RNpietT5^fZn z2_VJ!G%8)&M`aVVePeCKOVP~4{*Yae)uWx0Jj=jO;v4ati(@054ewDp5QjoovmqrQ z-;zyd0xWZtE5xq=Z`k>*xQ@2$;kWVbME4r{Sj#RpPo34LpCjN@!@ng4Zz_VH3=ff+ z`pJdlTkD0tP`C&;Rn1+TxRxzeI@i&&7`)c{#x7)LF=hHkf;)A*mP-RIBWb=0*%X$j zW8kX?p2+wT1!d18I=6~p+8`nCNf66M?|szaytVllx3R}$38f(6X_3bHkaOC8Xu|;5 z#d004&S;Nzw#>fCWd3&dqd`5zyMTTdpcYl#>|61*_Yj|r`_cgQ6CccWH-FIGMqvBV zN2O-hoAqP%i{sY;s{kO>f9JC8k=kCXiH2ty$PhA6MECaeV<eMndgJJyN7UP2nO`hx z4&PS{5;)~f@71s~&2prIc6Q(mR;uzZvM3%%gKz^V7M@+vYR7R2r)E7G)ou$E>l6q* z=w|Lmw9$WF|MIZ4sv$HK;XL4!B=#vzV({K*CHCE%39K+p1)qXog4_Rom}}JAn>}lz zOJ*y`PzollB>1R9?0WG2(V~aQ3WW`13F-kTR8q+nwmeQdzk<TH5;MGaPWe0+2Y8>- zk{4?Hd%SeB92fwJOPx|rD^Aq`Iy1ra*kh_}+LQBr_(c24Xzez0-@V!EL8QBJZQHZB z_r^V@B;3;mPi<~mkyFIaIzpsYOD=Ke;hnG;>NHp8wt;d*uO15)@{0I0+s6x7dD0)} zg6@@$Q+fr$<i9fS!%Rh~#nw(Y$fcTct+rJWxLo-5Api2g?S^@9x!#*>`QtWgT13ll zc4VP7Bnnjq(N2IeO+;S8K29L)3mjDLw3hJ09=Mp+4$)B|sEJq3AMc&c@S!H5Pv6Is z2i$1FkzViU7RV23Tw8hi)OjWMU3ZSFGPm8z!&Csxcr*&*V1bqokGC0diFB%5Q+hRX zM59n?=QK&HT}17OFcI6&UZD@)F5g(rzS^$>1}`g2Ahwx#l~Tv<M>0$M9qglDpn9p( zdRLR>Vfj@T4wDZz$$)JSg3xQG1`N!zWL!i_HMa|E#Yy58<+5M)Qf>oCF(eFW+l`{) zZ!bt%&d@DXdZtjpL}~^rT}1Clp`3_#CB{r^!PTqM%u1`9Wq?H>fNIaDadVYu>nFz- zYsl#oP0r{PGPT_Jxs$1WM-LD~@F*qU<*_N|4`$s5xcV-p;Wbc{I!R=|^cYWKHof03 zop0jc+v<oa&~$cwb6KK#p08YLF7@+5K5zd!_0tT#9kJiQ`y>mFi`nYu_W-vr+MVaG z;=enkdHNHW4;QV0_D_WRFkT`AR$BcN8U~{qm}jRy0<UyvwF(-m3~t97C4H)l{k084 z;NhgDD7~b_nv5+uScMf#ZacM*M_;>1VlP>d?aElug>>V&JPC{a33Y3eK#DIJ(iHW> zr#F^O<0Hdz*T!`!U4@2O^^C->RAtIMsUf<&gSJ7YMsZ3<mWrjRG4x&eeqTDChd<@M zO~@vvYkk}KBa#U(lYSX1Fg$*O-<tt#G@P}I<w#qN)!<Q&i1ae`;U~qOgoI!tEE}IG zj|!9VaS@Anrv%DJnb-jWzm-w90eZ{XxBXsuPCL0(>Aa<1rv_3?jvMwE8j?7YMo&EC zN&DLktmVOTSIMuWQ&XwC*X6c?;cClQlgIMzG29MjaVfLh^{VDe%sCo}!O3cdr3Qd- z%ksWaLRXyOc4n5J>n$={lUySgmVJr273-N*sDn8By+Xp52I3N=>n?xT^k_e(xVZ7| zIT@eYHgU#MR`D}n(!EQEE}1Yh>p8!@y9Vt<HVQeax~XYrS&8;NKJ+-k4huY(m3eN# z+>5~6A%WRWg8$B+smFp|sdXXfGbQ+RsG^5Wk7}pPlf<RFX9i#w>37vQj~?K;UELGL zE^$SB;RKe;e#?e`yvCji(zcAR(%$DV&OEn`Y-uJZDLdAS#=$yj^qt$}2I_hL(89PN zv^%cl)cbe#rGB%&ZpV!$7xJgpwv!V}p5++n7M}Vv_UA4<lcx{BfaAsu$;nei!hGV# zxDI5EV6M+A@)J;`Z`)9&+uBj4j6%!<X2g#+j#DAq07%<1v4R}QSy@_N>qGR3;K^e+ zEi9NY<UO8$$%ePSLrX}u6|GapBdSmr8FjuB!rpi6;dw(iBvA${oA2y~X6PO6X-;Bo zVM)}Fo5af$ZwBOP_O5;+U-tEjmD`QkMP4<+R``U22YNm%MR$h2LEh3>3?nVSuu<zm zdW``SwZPP3L1reGBsd2jVtFMtzJYo?GSd9<Dy?~<zBY?qw4$iHol2^FqZ_L1T{(#3 z?N&FxtlGH@(Ts|-&E1TMo|Sf49yTfu9+<w?+CqcJE8NpeRD8f1nA4z=k#v`VcTKq8 zPzVIkg-N>$i|u#j{|FF)?38LmNZ#KVyq`z)l2Qp1HF`3q-_l4Hpr{K}Tzni}T$aCu z`;au-JxJoiG7u-h{7}VsO|!KVXI~Yxlj*y+YBH5CZvNp-xXxXuBU2DT$Cu<Ov&lhq zCNqz_m7{KhUFiI!tO`SIY@){#oPm+4f;uIm!H1eo>%+gBq@asrZ*8+l57MN2jLW;v zPX>|sv<h|lmLJ~kSKR>?dU^9aJ-YlDRLzT39Z#g0HbHsP86rm<1V8jRt>wRdu)FrK z%r|5Sb$60km%!nRRk_iz4rrUSm8&@FKMazwUyJo+b8Xn>jnbR)*dyQP39Gi#-P>YW z8n#W1q^-+%hBhs!6pUFRsABCr4G|-mC5g9tI6Qjp>-`x4+pgC1bDH7m<ai74L;R~( zHl~=X&GZ43NxyJN=GtzooRQoA&L$+<>nhtuSL$W!5LTzLOYCY{zQPwUZ?;man|b=a z>dkq#m(43TU31;?;fR%w+BXT};z6BP{-VM}umz&6b`NiZB9}HKllg&?Gq&LCYI3E& za7CL>5*gp<O%~Er0QlW9=7|Tygl|uk)HVmLN>XY<XZ+-tE2bo_0miY-?9vw3O<G{$ za2KF+diJ_3z;?s9hf@`>s{vW-<V0O`nJoN}kxl(ztu$O0h{C`9G~wWyEWP%F>bk)m zbx$=nwUr?!zRUtpG!86WuHXq%iV`=%II*ZgHNz08WpV6r&a5k9&rG0V&}@hA*)F)N z_EW$jYmPE{91*VH<gFlWEj4hpzd<Q3*y~1`9+l<j|9IqZg!bvrfswPO<Mp(cPNdUM zFKcFBTpKlb@><uJKYbu<NSpbhz_cIVHPA(affiqCr%>c@F+=_}w1v0pU#*>CpyR$K zu1g(TS1?V{F~Wekcwj2zZ?cCmsSrvrpqU7##ZP?03N%GSL3*6uo;0w=kDPPeylnDS z!W`F@{d=!O0-0Ut8MWu-_?~0fcUVWNUzMR{rOh)aZ*wQzk2JL0EoO}TwKRU{m*G;A z&1dF6NOO|4;`g~U<GlmCEy0P5>d3Tet?idp@~2-9cI9E%4m&~uNqE{>=|(vYpQ2m0 z2&0|)zYlTM+O<1iU8gRc)9%YLJBG#62LBm;XL~)iZwmc9Y3hG{{``=<9A4<1PY5jL z2mA6>0X^r~EIX}bfwP<JiDmT8G!L&{D@bKP*Xv}R9k@&%3ry8h+|xTu2d;nGhuOY- zMkie`2yaf;r`=|1dxMYD7oE)I<HVW);|FnY#`{b!0!i)>A-u`}j*MtxnNmM|Ey<I$ zU|eh&S9Q;NwfFDkD+91%MW2IjHv2jL8t@OR@mgq)_-r{?s$c`E9(o+TIt*H}9OSj+ z+s}9HFw<nUICV;SPKM+DPTi>6DNHpy#qk;$NG29u8~a`8#_^V<ZC@DTD{Xu-Vwaa~ z<Zj;#cnkVWY2as-%*3_v?pEEMh2fc$DBG?mpk(P)ln=XhX8L{@17xZov#dbi<g5wd zAy~nc&xK4@`25HFN`Xib!!Rxaoq}hwr2-;IM*27kMAd#i&w0j}okEST>SK|<g`7|7 zcV2SjGUfM6gYUt?mhZGOfMrzgRy<e_2g!#h@|p(JC!oeUT|4S+wOG+KED@}?HK<(V zy};NpI*P9Ahu@toh?J*EElR}4rauWAO*(Nuk`nEGP4&(Efy-w&)Fm8y7tT=T$KX45 z=fo2CO4QCoVJ#sS6k&c`^|pbG-?_sm4|vdtjxxWFt+lH>=BblM8m#60l83i1S`sX6 zq`9rxvgCY*eSB`~M}6<%5w&T2=k~bGeIxea!Ad@k1OpX<@`O+6+)rms8De?7Gfr8L z@*i|PUBYVI1x9k88aAp!kd@Q7B}wZch%I9b=%r-5h7Rm;r>4aQmASy;Fo)PV<|l@( z^IM<aKj228^ow7Lju8wS%*W!{x$!PV7w>NkIya6Dq9hM#=bJI1N!lT7LA6qQ)WJMa zdZ<xz2XmpW1~Z|9U(c=$0?IQdB5w3&qA?Ir&iT%27s-aY%UQgX0l!7Pp1UzZNoN0* z*@ww(D@|MWLp{MopVBacU-QRt<lT#Jq;$WqPhp7iR(XD(e;Yu5;F`cXX+3o|VqNx^ zsqW~;a4w-G(S2ZCXuZC2J8W>Pg+aE}sx#ZqDXRFx#QNfMiwyG)*hg=>k#Fj=4ERpt zYd!J8<79K4U>SZ#6z3%f8@1E-6QX@EMoVv^C1ya^k<*8yk6O=NN9Wh9drZssR2<69 zbV;a5Rqcj7k_VsXs;&xUM|_B~nG(Mk|6s*Dopx^&tRe{^3jC;n&t_Ud$YG{;I|H%I zAbEme>+J2QKwdM~!NMaUgs$BdfXsqS?`e<b;mR&qtD2B>EDM!DHG9`9vfBdq+r69! z!;tA&#}sWUrCd&y)u|cM%V_L@psEgWQeALw*FHbz?g)oh1%gRwL1#<G%nhki{WCpz z8{x3QMw<-D05oj<^eytmfkfI<OpEEy7SCf-1=@Yl``;{&CyVoo+Tfe}iJRZim^_b@ zeV%U+Ao?{u+A<Uf%1TKOYzuFuJjLDFGm%V=f@B|W{GD?BEX8V09gAB3Fio|oC6Rq{ zDuIjVU#l{EG;FA;6q+*>X7&%9s3k4}U1EX03Cs8bE9Rc+^&1yvvkS4J6lG?S^m@WN z!*x=UBVRjczY*L~=~xYpaP6mLITwwI+&)cakyh{8(0c#j?1R5yxPPqw8gS-amBQ{H zY~GzAV~)&wDT)bq9_7zB*DxIKN3wn|<rss(YcVkRRposMO9+-^sPfbf<~zg+;Agg{ z(RRc{G_lbysllYeXw<}V-r>7_bcvZl2%p&yy1#RvGE_15yKIQ<PI7F3x%0rkOgw&! zA)wQ(RqWsxy{A$V6RPo`ze8&qNI}=K_1`=uZXqs=N8Jllky6A;k&P{u*7Zx~6^~E2 zzTAyWs|i&9cG;1k=X~1kX8v`P%C~CW(BxV{ZFA~)K{uWLtbE*>$d>s!(=mG57m4a< zMvE3I8uxsUL2@;<IGg3zo32l3;Rn40^LepE2G`N=JK7e5E6NW!fV$7$gUX=yGUK+m zjCIJ5&U)<Sann827CpuVUgr*pDW^X!IqkUsJjWJKZ4ryZ0xx8fwBIs|;jrZN&R+s2 zjhV1?(S`%{VxFS57Im1X%TW+TAwX>2$ohX;rVB!F(2R4O!zMIU=8oqTifYxs`oVYm zFyF)ji-@U1Y!f^jxf4FMA(-r?2fLG2x1PaBxh%EYe?c%IP=b4o{n^^@V%mf&Gh3^L zOoLY8n^<<QWkf$9oUf_<r+{;xR!k?~=>9t%Gr&eI=cTBJWrNx5DY;z|9``^t>Fwu^ zJ*)yA(~SL5w(<GjJbM+-j?IFQPvbLQ>;NtT^;MbDwhg60L6&HwOCzO~NIVv<wP(Q% z*@Sl8In5V#AWv4NC=d}8_amXm%0=wmbVb$TmsdC%1kAeH)65dCx51ZL&7jF+)=%l4 z&9%;LHby{)o_39y181Z{R!Xl`|IMteHf-BQ%*Rqq>OItrm{1<&neUo33%nfWq_+e> zci~s(oBW)WcEhhlsbNmUJsGPeM2P0NY_ElUZh35gSk?ZulUOIN@i$z>1Dpc;3DQdP zBSOv?Aj<sK5kPjnRFOrx_i4-<rsMez`}t92S{W1zfF|7bH_9XBmkm&DKJ_N~+ObQP zmn{VRkq>3#cav0X^>?ds%pxTdeyEJDcBbEul|Pd7dxe}e$5gK_>Al)aXyO>EZ*mu; z;Y?n%S|(;|HIy)nyZ#z^HnGjq-%f51plO^}%0{qkUR!QGzsER;5&`J@Vo%p+Hi8Cd zpd3VY|K-V>`82VFyl9@vF>zMk-5}%Ys)zg`?{`Ciuw9x!ZOX0hO1XU+k!|3ne8!`z zGZ2j1lp$M-3dapO8fA#Ndok~ea&`Jj!ujGfAc=erRB@c#O7kqg8k)d!omo@{EYTP{ zjwvy)02rbRMrG<nkF*MDAmlnM2fAcbYqJWQEyE~Qx*@RxoYeNed-2vF(nl^N@MWqc z=|>T)!}kPAtCadqoR)*$Qv>{kh{gA0czS%LzdpN2z<p2@0*U;-b{3%_EX)#}LyeU4 zVG`G%ovFX}tP|_Lt|dkweqA|^$MsH)vaRWDA4Afr7}6TD)<gMb|A#sfZ!PbSo~`wl zp8^oN+fkr~O0Q5sq@mv@OJDKYF8G`j3dYPsL6)J1@~aZvniwHF?~u?WON2L)%|QHx zYk;umdub3r7b3BO3`SZ+lw<gfkqdu%0+@K~-@~BU6f$%+5>8b&M_j@sj)3Zp5=qdA zO3#Db0;uq|FcBMSgieHoXMwuW+(&ziFM4D_XTZ!a6^C|=>j59}cN-k89Hmv;G+yXW zcNq04&2WC3=AF3j9S=N|=31j~yL(f1q8uWs@6{K6AYJeSx$TVK*{%h}N*#QKb&mE` zXn+v3!OUsf_C@!zeWak#R&na==*5(BjTG&JwV{sB1Oe<muI1W9-@0n)xPc9F1+?yQ zltNm`F-F)EzxtorV9A2%om|13B|rRr03CorG_i;XttQPAR&&agKN7iiIx@sAC|e?C zdWgZy_%OQ-wS~pYO3s-jKqE_hKZF^|8<GAV=)-1LaP~D)qs1VtLjx^EtMxPHC54zO z^ZmjE3s{YhyqU^ddGwDZ4<jsF)sR2z2LrS_>8JGPr!y~L39y2KqBiLCwfC;*E`vV3 zLbW`sSg?*J*sIVrq)R9npRlo7w|&<O*z(_izm95k-Rn&w!rTF5@A>VYPp+@7H@syE zW__?0p0959nY>TT?mm;<+YXN09^XG+&n{mM(D>#-$XM9!jR}x4>KY9sv~gU@YMKT@ z8#BE2exUeu@I12K&<tv`fMuGH48A@pS4E&IoTz@CY`xXuV}f$1uN%d&BUpY4#c3#s z{^6~;x|+WhJ1>{;M@M9ndgMhmrWU%YIoEE!a~HJd?4>^!;{uBUWo+wPPKBVZ*lKN% zr%DBdrpO0Zv-Im+jp$n>>LO$RB0`qP1X;rM#t`Ao_M<ezdQPtPKieN!#MSP0yOCGW z_i2CzfbsM7QunKmp+uC4$*!6nE>=!kn0_>V7TauDQPdIO$;M?fniB!7rX^O#zptOk z@+K_iMX0YVEQ|0P83sz(NI^t^&zwb}n9r$!T3;n>nFJPSPkxJy4bF?K=XMR-Qu6fk zB)?8L#xsKb8tG><zwg1-NA)xfCbM!Ok^4A<_FK4qT14w`KUf~TkG=!|mh8SKaHkjz z85Vxum72tGBUx77b$*XHZrUFu=6vm1WkO*|mOE{^a;^u<hV6PmrP39scblaRv(?Sx z|2;T>n@P7JA2s!eAxVf`ht(?_>tNB^90;T5qxR7|`|(9Qc*s9Yi3GY2Zt_+XlXkl{ z?Cyd;7KHSU4Ui~d50u^p)awFMbV6*hNje7&JE6i3+^J#1W%XmjdxwR8(-Y2vga_A^ z=u?2YmGi~o-6(-ax3ziDB0(FPA78qPWIc3IH)BQeR%DC29%{X7KkY4mVlr3+F4J>6 zXo3Z6l{$W)ntuU0pA-Rjf>i4^b^o4h%#Iv@n@1UArJLWJc4p|Q0{C-Vg?jAM%*qm# z5`!>J@sh=H&8J~=t2;kbjRnSN3w3V&NgSzzIWe{Q2|1t30=+GMPA5zk&7@355SALe z;lT~rYsBFuR+VKSsJ)vygd^^+&@F=FCA7Z)Ebc2W@9x)+n@?$y+yuk?v}p+D<+w~n zpLWo#@VE@<_s}uimmWh&Lg3g-@MFY&-MbUVeo}@6?nwMP(vWoQG#1$6Z@6tom-d0o z(fS>ex(_}XjL_QRvr@@6#d)Z13R5btCG-)E)ZQ`?HLH`vH&h)f%n4%O1T3mEYZV{? zdK!Y)<kI;aOJMmouIBI6*Z5d;%SdV5A*sC!7t#8bQoE0~FetT9e<hRq^Ldk?Ly}z; zJ{!(mb3D`S?0}i*q1mH2v_3#nHOglX0Hp<3Iul(^eWGAg0&)$eB<R|Yf2EG`wRrWQ zb!bi<kfRy!w#?A?xwBYYIWRA#^G>=J^w*(l^ma@15t1d3Tn#mF`203YrgvyWl`o)K zR>(EbYvDFodPco$gG2<g97z07w_aaobyJF@Q%|7UV<Z$}+FyM|KU%D4c{JdnK%_J= zOKJ18Sn)Yd(rVKpR5DgRIKzoFEZ{$O*B67E!dy&j<GUww)?b>8Uu-B!x*^8e?`+7x zz{T`yeWdv5g6~b^n*=FC<@A>8yhy!tA()AajX{+u;4WcM<!?p5gn5GVZEf*Ydt=82 z;TBfs5iDh??Y6U}5Hgo>?T26yMX*_fG#;AQk1r&oT5Z;+`M94Yn|?l$D~YM;Kr7S+ z&syV=d*qI?@qvDJly@8``#}V;Z8GoSUrOhg5E%hkR!TPTR8tJo4ISg&eltkmyzQpf zogzMmp}ssMg>SwSh8xsAtUd4XEPT=&StMMvzj#PQTV|W7tM5rEaqCP%US5J*uDCoN zg19tFGuz~}P4~x8VnB3?Z50MD>Q_lxVAH0GOAcTA2ykpLNts%FUoH6lOscg=u2?kn zKU|joj;}!esac`DC#>W9tF`Y$OEuCCNiwu!PYntZTF%IX;d<EteoP5Jz&4_F)&bI= z8pS#l#)*QnwaoF9?d*nS_q|iEN1^mb2%T3=0EW?lNvhM&WEja66Tm$9z(n`;^Un-F zd|ar1z42ajzc)Z5;{h(*trv;8Fr=$7*{}_QVYVI)6g>`v3;6F*n-9DM4f#1K{x6>b zUTyfeeh{TJ@{3X$!nAogJOPr@RtmD?3rCj>_$49Alq;Z2ff@NVP8j4W{~im<z<4Yw zj}Nv%KK=f3sx}A){2!4vq}=jyGy_5O`Cwoj<hQ6O;1}uM3c~gopbqmYObl|Be-9f7 zh0&&9cS{fQDWsEdcc32d_v^y{y@{6}mVxWuZnbMc?oF~6@=!KrQT43=h2i(Etp8pX z_<;-JOwKj=&9M~n>3>cIX|rli_TQU;{O}zD^LaYzGrlBuMv&z0>>!Al6X<!-YM}6c zN08Jv<p*SvS=4i@K|cM@snmeK_tQlXBze|(L-m;wpVbugi9CVQ*Noim@VI5Ob5taT zdhsyk>L<#m@`u9z@cDr+BS6?_q&$2t?$C&kB`JWrge;Oyx$R(KSJ!x7!g6EB*LLHe zq(u^_(MuNx6UXQ3(7vx(Z~&to>hk%>PBH$X@@SqRZ^;m}1Os)V`VEi96;$I6?)8aC zLCHui-PkwrmQuh*!Czm$4Q42RT7-3YXt3R^X5Zb2_6z(g8b23vRy{$yxi+83X5!7& zT=S)Q@lnuMEopnTXEQoCBM~xHZAHo+`5pI6-F#LDzI(}e+@~q#Xgq#5UfbIce14$l znCE)Cu=+X~<6kcT|E65VI7+G5E|sq{FXIUe@B%wzqOdCs`o2sQ5kMH~zqTcSAjW*Z zdh!~7zUi;lM~`k{X&+N@-#ovI&T}TI#j7TMoi%+-wk$(K?#63NO=_q5i=nC_c$uii zNSxB!)79b5rNfPRZC<01g-Hj?g85WN5qiSq>`T;jFr>n7asop^PVLa&o$IeF`#_q_ z!&P>kvNjbLBhaDka5jU<8vs=9Vu6zC+54!VbG%3&7x_D`LgLaeweh5NI~fVIgJ`m% zeW@-k&0cW4MUW-eEYm*|L6R9V<C5|>K0zifsTt()i^rED2azYC+NenU^ltdNWHu+1 z&ZMpIhDw&cbPj*HFe3G%gXFJkLI!mH+i>UuAe}Srs~nAKk;jC9?+Xc>O$STW3Xr7= zgW6q2o%X$AjzfR&e%oqOgAs%Q0Mzl%#AB~yxtJckg#gZ@kyxvVmw2BDEK=X?cuU{B zA$<SQVG2WTmhlJP&GqrNBpU{!<>zXtcya5^K!*}d`-7hBU}Lj+uFW%R1AU}J8V zQcuk#p<Z7{hPB)uoHwV<8v*5jGtpIpvETco8OH5Y6>{0rMXuZ?vv5*2onNT`eGPn* zAgk}zOt}f<6NIn8?YBFb7}V(kk>XxV(s5|+U*qR8;--b_C;nva#+oWu-%aE<qVZGM zVRP$1e%F79e_Y*j!)ALxI(zffpjhmT_5M;^n`Zg`6;uz4;iPjR<|M00AGP44-2{Ro z{N_+YMQ_u!pjlY<fXlrzp>$pbl~VORrQ_8-C~8}KdRNywW_@eUa=KV52Xx)9eQk8j zxORQtz}5|}?{fUJSo%KZKC>s)p>SI!7AQP42ua>w=3<RC*}!Vr9GnePs-L|b;#h*m zc(-bh_+Asu7raPj7~yibr_y)Y+7UB{5H7iMe+_>vHMY@5<xva|li<Fz+n<pL>o=7j z+rzLA;qIulyCtGuJPw0>Bny$mtTIR~>an`R>_sF!${q-hl!$v)SzU<JkWP#xQ`{Yl zo{4u^?3<1HqQSO((zr~$Og)k6ZMaa(Fh-!JgSs=GvO{?|%d;P=2#K30$wzgW0w=e( zoh-Mf_s0i(#w`1sGqpyRxPu9sY&4QU)CIafMvZ5CQSZOZ#yLo!!x;w(c?siDf#vVi zq`;s$8+bThOP;6_8MUG|-YYKXO_u+|76Aml5pO@}PCDZqr9l6hjuU2zkDfbpcSC_{ z|9TvxLSDJW?Xy)0^vD_0D58=}R&@5fidI6W_thy}R_w-ik#xfDl&TfxJm&4k%zvss z%RQXma{7>FX@9cACiYh~g;=Ny-~R>{(`w;IL=3ySLoTsezgN?x((d(?q6GbDEL+f3 z)L{lFO#Zt%yIX)~qd{hsIJAh2oqs?Vv1sbwaemm(wwW6%>1nJ^pG<oev7J`#c!=(L zv%Bk=CSbsIO}y0_Tmz}MXC7M3`kUl#rek*b6ThokImUI766?;V1zno!C*cgWs~yj2 zSb>(}?{D3SZqLTi59>$jb-1R2F$8E*oL;x7#kwr;hWyEcHb~)ZQq1;^E56HFRBKN= z_Cmqk2D<4<t}0^xaR)Ol<=&?KyIuWbYm@|fu{BylgbcM&mO*_AFx1U=r^ABN8zWr0 z;DB`NgWdF4T)h39FITQT-IuJ^dbk;?Mn2kg<P81%G%omsWdh@gHwANB`8$40w3_u6 z{4CABQ}u9V<D^J!XP_I)$s+Iyq+a`dwc6Xy7>~Ylg$^kidP*sySckkJ+%m5wYPFDJ zF4(HuFK@q=n^;U)GNsfaj8ZnHbA92T((Av~XTG7jmOQ<nMg!~usig-wB$#&>nCt#0 ze)ikF_oWBVd&m`at)3pYog5jO`mla_VGe=T#GSFA>&<AxV6@(cQ-ABFQkN<21Pe|I zBp<yrt_d{*p`Bf3gZhnu>yYKSI6|b($%Wg))O{Gx9ne+8I_WS{excKtE@|~<H8(xp zR635$@<?LeJ)Cb~_NAC@f)ze4j;DnA))L+l0nz@|_0f%%PO+8}vCoy*?VuV)aVmeG zMrvCX?j_vkL?-RHsgfo=28R1#PcC@{NY_+g2?CA~a$xioV0ytXF$*yyAVjBOpss(1 z-*({N<v1vtTxk8kioi32UGABdL(EdN1i%=oL&@|`5%IAl+St6x4EhXN)Wu7C<AHAs zox&JxYKEY7eaxZ8xT-#i)89!d{*^V1EK6CJtmtc1Vv36b*hjOCt3(9-nf$s^%e&GM zwF#43LhSQ*ZFEuwpsw@}>O)(%_t0ED%`yc`dhg-G8n%oxs*2s3{W@(Q?WzWdXTSDO zTT@dF{!D&~K{5egKuVPqr*A*ZTgVccGz!aN1<RGN8g4%w=Z9_BtZV#;K#2NUs%GA> z>xa{s)H2^PKZM#ZCNahx=%lkwECQ{HkLHM5>i?X+aCvLWS|;bwU@qp*2u;x_3d>e& z?TpW>8AB;$QKE3IrMA)L#HEn6pTZAHeonxY&6SK$T|CfG^2M^L>c9ax&iRASr!#CT zujQF}d!6%de=J7npWV-=@wmQ(a*!|z;}O{Z!~~V5Bq8zLDntQVJ1kFs$;9f1zZPkI zf8%v~dvmRNA^hqYH{!$A`QHy8kA4wcgEIo-HB4cVw2HI90DVzvo2B9%U@QD!qQxPt zehsb`q2|7wX~Gg-fR4nCQ-$uHec;Z%11F%XYOErhjsX@LttFqfl{a|gg^-Sd?L;O5 zR!*(yns%l?%Q75{P#aX;O4B7(u|GN?Qz1vpuhVotmK~1t#Q>p98W>j&pK3pm$&S8p z;dHFDtUY_UafPirSxJZzoq3rp__7>|EFcRYSt6(@eH_qkMf(+78g;M?3R)~BpOHaP zW<F1!PD>F;c;C%XZ8a#am@}6mJFaUzOB#Fcbb%qOLLU@JK|i6~>%@q(rl%LC0aQe@ z@Z5xyyT~EK^>VxhmSyBu!MQJMv@b9}6F$*!PUwq?Gmmqw48b9Tu5pK0Pg4a>Lh=KW z^c^Lxukqgz3yZ~$$!Y5MSQ}?w?&bbqv{Ude{6+0^8`3lh@EFnCP)@VNUCd1u7cbm9 z3+OH!)FX-bFH^5`k%)~gIoX9Jy&}hFCSKt<$1csAXbUgW2x{>RcM=m)RLx(-0UnO_ z8D}3=+fe*S1B7H`pLUXm;NU;j92D7~wZ+u^)C=Ns;_YI2vu4I`tOb?9V)zA!{HEEm zPre#@wk?+K*VYLk@;z<!>Uw_}&=wU@jouI4){W5vf#rc(<PysQ`*YP@5h$#*y6HdV zxT0V^IOIO5?gE+h)V8^UzU?qV<jS+xT<=DBP7#r%C)hR)IfQI+R+`iQ)rx<DYJ+GA z-q9pLEQ)*sAmOs`uR+>yDmuSp&M+bZ%><t%@3f1hZE?QOlnA1NSt1mKiG>aw&-4}Y z2o1u{1v+ygGh_7&8Bd+qKDW)HxD&xMn(z?~Tj@-kDbLj(Q($i1v@IO2<JotR);4M7 z=j@>U)(~H$2xfZtWFs9bTX9GOwNCNLTePhCojyI1A3gz$Qiw-sh`$rzn*clnt<qQF zRy>U^iHOWT+Oz@2`1BWkQvZ&~Z*POdet<_=BWy9;cRaJG0v$i?NEa?9#w?;;Yl<OZ z3j}Aga%vLfRsV49Jl^dL1I_|g^mU(be)$$}bN@HoT7gWRpIFi_l6hfBrcE|z^0xa8 zNbjmE$f^AFag$S?8)~Nks}o+$xy`@?sF}x5nzM3W&%fVQ&JIWYvg9#GPBVp7JP%MH z8~IIsH>w>u<&V{-HPA&G`mFtE@$|_GllZ#@)}c@gKb&fZGb5?nFr-<EbK&+BwBLxj zu+$h-vuv_(o;)~GFlr0P<owuDgAMpB?%bM)5gJ<TJXCdiojOPEFX(1TGdoVZ0aYFd zk#Mo?9JRvHaP(_otsJ~pc$o-&N2569gWK0V+n*1cJ)H|~*L{yfIML(qF4rM0+XBbO zAM0V*TpmM$Yivf}5$@9`-PqM8una7~xc#e4VS)I~4wM<cs5&r;fn_Lgf1uN$1z_^Y zzT;?Dh_#rq#yG83TuPYReQyvzrm6;Ej!6yc%RT|g0_bvpP=;4>v^?Hgpfi#9l5S?k z^e7EFEldmkWri$eAqO<t^J_}4iH06-LFbSCsj4azd#8mW!Kg#RoRf^?gRr%v-^u>< zSQNrkg{Ybt>HynHvc8zy((SU?&9(gO7wVn(V)3!+u6^ddv@$H4TCBdrAV9{GULC_u z`&Gu$0U)V=EY4*W{|s1ilGsldKV1Uq?c~ou^<gXS(#dXb!ybR`N`_Z%$$r8!{(9(~ zOXBG)o7E-BGz8$w(>QZ}KRJrL=k31+>GmDv%~|#f@3-#>knDUc09N0pK|(Gy*5nAh zC9|nE4a2M^h_5$B1@!xcK*DcCZ}X6|2;mppTyRzylV2;GQvQ?5izWlfCoP2$?91wY z42d9JlyKOHfRdNU{y3adJW+<15o{LCmw~?%=HuY&<)f-ByrCKr0Tl=S8(K7LznO_b zA!Ex-7%=|9RIS@0(uej&Ac$&IZ1Tmqyxo=4I!!!gsQ*2d+~EPPzC4~?d4T<q5Cymp zAvl-vxgCVaaY-NlB$d_dv7oZfpaIR1hsIK#(w2KpiK<%~)V4jA+2tbvnX7*Sd}nR5 zEd#QdcF>P3I>0CEwISs}T<AwL!Q?V8pjI^K*i-9A3at;<P6G4E2hq;}XoOqzJAict zc9NG&Rs+a{n<~<dUtUq=Et}<`L%`tc96{T&$}uD5k{Jv_zP2fFXUrM~+^jJ6fu@3{ zMd-)F&wu!CjL7O7Zp%F@t<sz10Eg}O=(wJYdR#w$d7v>S&v@Z9jIjP!<JI{Nh(uU= z5;$IDVzrP+q?ip0{D=g+`K8{qKijf)2TEE#@(%(zD%6Maf!Slnvx^mg#?cSLY${s$ z<A&jEQGK+vnv=i1kWs(?x8?UUhQqHb;#Do6yFB8ZZXer@AjlrTtLZZEU-uCiHMw>X zC^jlO^|bfoclP-H&}cSrZF_vP?jMzhxTX$(XnsG${x;~`A>S=K9lV2doL6P2zt5W& zdqa-5AQDM5cO2J|sy98*aMX9Su}clx93Yq*#XX`ex#+gklegR?l7D?WlgbN{2w&!1 zX?DptA%ZfS)-XQ2JLGqY#<us0KTiz)Xg_l~ri)V1A2BD%Qeg<9US!VnTr^QE|7@M& zHHFZ(b3CmnIBB3dQnXNOEl;RG%1Lan0n{B^ZE#kU|Htv%3_!W)=c3DnUlL0XNMh;z z!ywl`6T#$&SQ|h*AWW8S+Vkpqhr|vR(GP0h+G4hbzNk|!65%(7b@sa;2-(e_qm7U# ztXs4x1B~PrRh5|tAe`~3v>!bh`b8m}*b_1rKHz2-Bp;Z@(zGqg(BKyVh?cJm&whP! z%=I(5Mm*n0zjx>{;>xPBQA^RU2#U)emppr01E*c1fz+Nhb4Wn+4VL~T@OgPz*ny`l zlx-jc`H%m_1;Xb(0d3#JiY9NII6yH$HMX=Z53XO}hIFuL4AhiEcWn|mG@RnMKPN<5 zIO#h*;#K%|M0+(DPmD$Sm*b%n4h7F7XLx_4Ya&#Q{<LNtqEi0xU&!s#D_~5ZT326O z)FgjHV2F`WR3|k6fS`bL>G5T-fZ!Ftveu@JE$|nk#`>zadF;68Zw#5^)XA<I`^6uT z$v0>6zNWAZ@zfUq5$r$T%@}UkCj&j(Gb0n%{|TpkYat+FvFZ22mpOa@Ab>dc10CgJ z01_$(ZsX_OJTCl7b({_B*FyW{b2}AI770`rO=aa{Y*jZlTjVgFJEA#gx>&n{@Uszf zgP}>^|6%VfqpIwp?@>jN5DAfPkZur=Mnw=Qk?scRk}heKk}eSu5b5sj2I=nZ?z;Oq z>hJg75C8k=j{E73amMR=JjZjMv!A{9+H0;k=L%yWe;BOy7x?nbhm^@?>3+n}*LmVW zU_$Hf*1XT4;Yi-OuH}F6KwX&4qh3!m?AW9Z1;UuPfJJw(K`kO{|1;Ty9k6tu))QmW z5BXqiLf9b6cTjS$7PcH~pw?l_%zt6aW~k#yL0dc%ETWb}buVTP%(iRrXB$@i%6~7Q z5ww6Q%1<P)4}!8j-5~G`?<HX!{ZDP2x|P71|Dvk_0^r{0hH|RGPAgUr{Y%}DBINRU z17iry$DaR67JZn&1?cc`Zs9>UwiyrFG5omP6(k!N-Ft9dU^R-rCr1nyaFbg|OiN)% z_xua+v&y?gh)%t|RET8mB+$UJpFcSek{YyN>`lgChHC+A(B+_-UuDL6sa(agoWSaO z&^SVv)d{r63yhe3f}S=)j0dZ@PtSJRdYEGqZ<bTltF1H(rxGLpd_~k&gjv(0r;ayE z>|hifnek4{aydH~_9?K>vVf$a7GB~j7|^)Y1EL!T3mNz3Xo3c70=NAaGuK;<zi<b` zb0%*VuIz|1zoRdiJ-ncO9sMh3GcZpgWRzIKBHC&r{FCU-Zd|w;E1ACu(EyAiTtLe# zY_=~cDw}6F2B38@uA#U`PUey^r{lC7|9hLrpi7{*(>w@`85mH2FMo_6h$EL}1sM4I z>96O<xN$t$F|Jjn{!EGo$Gq=qr3Zy$+Dqy{{zFXIHD9AOX_GOS7W2TUZ)|bIoBH*P z*aJs((!pf`f<@ea$1s3&d4gfW&@P~bfy5Z#J0G=+h}z*j?gn7A+Dk2SplZP{*aG-k z_VM&5O8+N#4gXI#-}-JhZOu={$j5YYJUy)+oYKijz_tH<2nRrC8>FBX=vR{61yeNo z4mBIEA4WEiF)bjRFQ2h7IHpP_D%J{>E4g2;i#b(Wv5ChsRt!c#a-(_6+VBtf>~Bug z(cyz0*l)CvE?vI7luC5y5NIW4KhL2{be=-Y7v^<e`hKO`Y19{`#IE1FA!r^7iM@Ig zZxn!8A-PAw0Vwi@d*p&<y!_YIayZAX9#%<Lh6~Q*hWMQqwjCnlNiswcq;<E_7E*Qz z`W@~`sWp_|axrz@+83Ut9)dSDS9{ZC;m5W7HlqULZi=T6Boc!q)ZRfF^5%z{MOgRG zK8L9&aSOJ#+VGsiy}RN`^A;Upr{v@N<NmTX9b4}XvCkJ5cHgDHBo&XO4U^oOW_|%I zIaD`4zcN1s`zc+SQT60{W+AY5`V{G$lG_uyejvTb|7STs|Mh$gEpFuZ)jrUKU4WFr z`>t>)S(gnchJ4%}6ml01;V}%ukDFV)vPcHLl#MErBeVUYJ~7@cpPyGpq5P;|F{B8j z|Ma!I>uY5$25;&@q=}t>C?@P2ba#C-5K=3W1cZ|jRUTuW@-xm&fp<GKx6gaRUqy(u zw=v*!njWil4y@~$-)I0eY|^^K*8XU#DoU;|#`~w(&KW^?$Ks{#<r9>C_#!Mp*w-J@ zl}w&~8BmxlUI^oxbt94$0;kN@5ztPCsdezjE*=!S<%TO}r8~02+(3J`QXZ#gjz@7n z-sS0OiAVGMMtDF+$H5ZctBXNo8kPi1oFv9~Q~89en82v3Shx2{$JcE^x(gHR^y<8O zNa~=?1P9!dMq!t700L#~wKja!-5yhvu@Q^Wu9S(SpuN#59c-t@fNY?)D_qsBtQJE& z^Znmt{{qs#i)b@SL-{Q-BteE}{nXFq^A>@l7`ZDOWS3z>t;4y({W0d;q7LTF{wJ<f zGBs}-Kl5TFg1Uvdo>B=BYcKK%S{6GJJx1>5s62YWKamUpD7>IXQFFBUl-{ozlBk`j zqXJ3%-{UB6TIl5?s)}YzC&uWk=DqPo?WWW_VwWpR;!qUI(Vnl9y8if5(AvoePG-{X z{Lac3Tp6F`sP|ksJB~3Xt`r4TdHXPbPd&eFv$v#kad2g3QOczY6}tRU@cL3*vG<!k zQ*Zn+J~r!`+jn>Ew$A4{8)Mdujr1<xT;cATZ#Syg2WhnR7A!oz-~r$rr1s)JnF@SY zkEAuWZ*w?-t=Q}xaBPy(N^5v2lxnAe_2o8T`DE#kr3197F=*<1Ra!^8h;~ueF|>aY zd$&D0EO1n!bJA;4fJ3<f)lGDET2a50?Qk`ZK`HfB#cDH?KG+V-6aJaof=vwrAKKww z29HA9gYXp;p56PjijM{58&Gw{VhV}`bl;la>gROhO~|3(FfiE#DTGy`HeOo;A&N~N zrXO!;oWUi(!v~M?YKyeUyh+J7iW%EMi}j;?`tqyP;j3XZsoP(~xk#Bk^P4xGn)unV z4K<PyB%7`uv5nZ|7;f%xsrhDtVo*5mwhNzIeFq>Rw4ca~;BOWwLfS;tU_DM>RP1gD z{>8LIQM+q8=4B_Xg5;*<QQKesYUM+-&}!-?*;}C*5{|cAIU39}(I1lVtWD=r=SQS4 zUkrjel3w;QOmES3XrO63r8@kgt49tYUkn8&n!3g>&YGsM=6riKVxTX;2ZMUr3tyxP zM3G3Boi6i34GnZhJdg_mYEGd|uYQK<)H62RnGA5yQDpsE?wm!e1mcB7((HfY#bYp< zFOpjdVex_$ghC0tL%d~KACOWkd5tYggH9IFP$3xSy7hG~;$fAH3NC+uCZN4XKLJj> zc^6UH%Y)_EgaW!r^FJ6oG)qStr!lJNU#MF@c&}RUSvCK|%R5d_urvRadvHpUZc8dU zisQOhd9b^!-n~i$bvb<5+^>2qE7r+LEDc9`%|Aj>0bu*Ej*z><!mhIe27$E_^zms_ z*-PC#Z5&;ezy;lT3r8v~jtU=^j`4}U_vTcYOpc)HYYKHy&s?4IiWBq1sD)>Ut>M$F z&Q{|WBd{Z-2o<<&e8}`6PpzRcB!ul+th+Ln+2bW*EX-2^`t_h8JG4XoEfjmgL+1;) zxb|OYl=2OrS<1*rPXF|d_M+h$@otwJNt*x-Q~upewYZOa7BT$0HWzLnhoWXyvUO@` z9(S%0cj>VgRDdU}y5b=0lklZiE^L64!1DPiY%WbWGz{N%+g`xhWQSOL^Nu>s(Trdh zm8}&b9`Y*{R!=j%+dMg5&m+aSrxtx{c@Iu&l)@6gj@ubf_*bw^@8s69I1Z5dIci)% zZobh0bfx-hSEp1`7ko#>*HdRR?7%)I-{1G&AT&5*DEcT!ZdKxT)y?<Aak0IBcVUda zSrf~=+kf}3r5Y?iV1U!Y15CU?*hv=-l78F;E!0uTft&%T9@@*Q>|X=fBLUlzeW-=T zGna`Kfg5!;@hINlrwo@t{@M$&W5)<p>S(4)vrXFrb`u3~Fyx(UMD@leGfU|<9727o zT+rgHCNz8nM9EwNo?ATN`1-Nl*A;%cjfQdNi|Run7IglTNP&~%HcO$KN{SvWw_eQy z(LpqBbrfl1i#oZ_c({@Qu1F#1UDb?+r*D<J)2cX9QlD1Y^%BiYHZ__7GEQmeJ;R)Q zuhc4C<?DtE7A~f4w0JyQU>*Ia`D@lxen}OdJ?^0Nu!M{9K+6ZGP_(^&cCBDP@qyaW z1k@{kg*Nv`cR+|l)_#<thty9V%ap{~e?Qk!;Jw$9Td{Dx#Y5B4`khoBpy$xTJWeY2 z11?Po8gvoVGR!JFJqE8RmHl+e+Hx0H9EU#LIor;xe!_FO9D_#rbKLgEhyE+k*pFx; zzGAI86y$zsEXHrViVOxVtbmf&HHY<MvhtN}P|N6JLS<4ytTPGEi{xlW6YSNz5TGdT z^3gPZ;*m@-kh;p6KK7UTOyZSN`BZoGt(n!FG{$IYjt7e(0>{?Qc!nr`#o-R)0KmL9 z{j%I%TyZs9yA(`qzDu*_OCTOxE-`oi-&{cWPvb#OO_)k!A8NF4Cg9OOa=C`lZSQsy zC;5~)$jaA@!|_M3lrP$gv3NlLRIhMVQAH=P)Qiq2KLIr^^REokWWv*k1zm<9+ul06 z;LM?>2_dQ^e|^)fQ=>oGw);RgNa5G$RP@6q^5owI;WqW;yebV}-jO1(BAHcSy#!eK zNHAN(c-UdEA@NtT3o93&%S9~TKymL}@8Zvdg7=Q~K&jZ?zwbwD&(Xt}1UfD;Wmfk9 z{jNl%OwL_Kpv%<!iMD3*Qq6QlrpzMV3Ul}&@&_8(-5xyceZ(%ICuP3)_L*0%*90Aq zKtAcVgYaUPr71~bg%sB#Cd=d6fZOx;nUzTiqC_RA9#7KG&rGCfA}~n5xZ{hFN%ynM zl<(@FSvWXRy@eJ`6kZo>!CXNJ`%RxtaS$0VJcqJ>jez0D>1*~HYvOLpX4b8}a<?W` zhA)vBidvl`X74W_D7A$V6Hdk&Ax+$ZBy?%%R7}4wMs4fdVsZ5i;5bY!LWfB|f-n?a ziCUU((URjk5#Cj|Mma5H<z&CzP_Ukn(0zG}FppMccLUBi3`0v?4f`GtWi{-Dt5;PT z=s9?EGkJf`pYI8L8e$ToK0;k+bWhRso<WBJ-`;ML@U`&o8=Y-JWlW3(iVY!4b+$5h z$7zM)G~~0SKKL6kh}T+YX4<?HGiSY)kY{GbsSg<0${kXqLsIr;>Xy^eItKRrW)vTH zdAZ88?+0=ye<Kf8FPtfN`dQkBc@6kz#;Bai7Yz&O$clP31oEZTWI;3L&22rKipsq} zpGgMEZu-?hJbE9b1?6FX9%S%PJo?!31KN?l2t)CR3=4l)2W;kUKqD);Gb-zfa4kl9 zQ|qh%)vRbw4?pR(7kWv>Wm>+$)H6tp)^D$~+6fvbw)WbZxzRFXMS7i|>lGEn*HE~5 z?O-_ZOt;-1$tPAbpV1Z(xF{GzW^2_7S9{->YJeKMy!N!;4f*r;O@?q{=&qDWxhfk- zhR+;EkOYB_HY%^o{c|2ygIAW#Vk!4lea7lB#}9X5B300JW^1}&80B?2nsp~A1$TZn zP(^RO9{EBdUGGu;Y~(M)a$0e?5p@~!(5$1F?O=H1p>$#Zc{Gy|2}4K)uxaxRV7rf* zHMoW!f{<9cxodk`D#f1cpa#`w>9o=_<g!bw6MVmC6f7OVb1}qn{wNzHU21B~Gu*Tw zv4}5F(43s7`Ij((eGH0Sw8hG>JogdQX&G6MB!&O8Ht<#=@Op*ISVCrj@Fri9&*MT# zJdQWVdQ=kU*DvfPvf{KV%9r&vn2A0T3uzBlpOlQtvdjJuxf`g5x@vYi07HKm2{*)v zU2G_)AV%Nraz4&WRL%fzi~jymgnTaWiQM8*)HzSYH<D<Ydqh5Ai76-LaZNd`_I>4I zn|pH+ym>MjCA7R$KsoYa7vNPCU#mbM+Tr`>^#$d4P@@p9<Yso*9LR^}fbt>!dVjNz zHqLtQUZdkHVF&gcVAcSlrEB#DAK!mKf(9srqf$k~93gxlr>MsGzSp6CUMi{m(?_@z zClgy?ZfUb>5>(#fA|GKX|DT!nhyWTTb5R}=D9a6q1>0W${`6xiz}0y?NJQ!FQ?*?1 z@-l5MFikhrUGWWcT=?^_hWP60eRn1y{K0b!+67MH{0#yD{$Qgz21BR*CSjheP)tRo z_SArdTV?3z8tv(CY6~NcjQyk3z+U`6!%jLpOk0ot_f~%q#)E$H{fK&I*nyZ4La4gx zn@V_44Ci?{ws8OZR(0sbQI7f(SSo!2wy!|96`gSkfa`q4v4_>}|E)t*FzgS!#kFyv zzaWc--fA#6KS2c0kwp9xm=(^yx8_1GZsKcSLIZ+v34t_%hxuk<UMkAc5BOgF?@&F# zxDTcB4Nvl6ZSyd?ptrK`S(%3bB&Qek^8fbMQRu~E)O<JC{cr<QLO01tx%71dBV)R6 z;(t4JAl(BmHflc_1C8gwtuuh$itkv?B?eF@BfsnazBRxUytv?ee@qY>mS#b4&kOqa z8P!k$^ZYPW;-CwV#Wp^aNmTN&`}8OS1%NOpMKA-^3b%HWbX+ZH2iI<6$t)iE%(}ji zAKwX1$o)3)$(u;p#`6Hz2mKxf`XeVox9Sm#v&50bWwUg~1GAV+i5HulNriyoS+VQs z4-w9Or1e_mEdgtWrBoG;vNS0c0Ds`^Ob36sW!9)BtGhl*(Qp3{3hE&A1IK$X{P%z6 zRnaThPyG8U8dI;qiheokN(Go_kueOTk)MxMeO~S6X;e{5b@4NS7R6EE2m-$~_^VfH zxqeiu6DYx`md!KnMK5=>W3Bzg6xf}|{o0itbt#D&np=X3H3Sen-Q`v#aXyH9zrH`= zTLk(teL8sE&^S<h8t^`YV~T<2x=UsN=-K{Kq`q72TBKe&%x~eZ%jS8(^1&+ezSpBo z5B_8tx=CO;jT!agz~<fG`GOR2LKpkWoB{_Fq%Z4kUZ|cBQW`h4A26Gwe7+iP0IfYk ziNn^tt2!1%g=yt-+*1jILY!Y<cQb2M(hAIE)1FSnh^aor!(Fd+(JD8c6pb6JN{q}I zXMH1Cz4>r;mpf3~kLS#4eX{Vwh4MN`<bZq0+h;NlE@7bP6Ugwb8^PNjnzL3BU<uhr zNPZA$0`_Z|$@Qz`R>Es*9Oea6b}}dRzLs7^;Bwl~c49o4OvM{?)6y(1XTqqlh>t;< zlmqgMnMAf3RgngEdfq{Rq}67q`&m*qYymZGDrgU3<8eci)-(61zrsxU&uSD#g-Rjb zRw|s(EcBrRvrvbZtzbH5?NK)d<raXasPtB{ZDc~G?rL2C`ZXnlJprM|Yel`Z2wn~6 zq$a`3NxJ8f*%Eb0G2P=kMPBlt$LQC>FuRUOHIb05Ny-5HC_war9D$@0ar=@Y<Y+=A zc(G(U_!K9#lyciYQ6iex4H~r5))5I&gA9*L?|kAjje~ymWGk)BsQ>4Gc$Iy&rwC^N zVw==8o)R@y1k+oUZB|{%WYU1!NOn8b^P4$MLWXr~$|bVWgr6{LpGCz$ccZ@JkF)2` z-GoR47>%T_866BeDdLJ&BbVqbhE4bd^RAp!&~|3%lx{02PFFgNn6|tyyIIp`J*t!p zL$?^tzd5RE=8U(Wa7u)za}Rmw-*O+UlHyYNf7*<iXs)nH*WP%ZT>!BdX*{0Mf3m1} z#h!p?{}(!o2h-zz=`X6bZdKC5opchj-=wT}18ohrNoLBr%5Q&$0diErK2=c<`OdhE z5K+M<gK6)jK(7Dtgv`&R9bnh}%;7=_IClC^>FPESb54;}Pr3Egv^Fx8p^-1iIMm!; z9flIIo1RS!XOVm@AgL27mkFnqu^0`iyF3Dnx3Nq7#CGEqIshtOpCE4T{pRLd9+A&j z7SuDa+iyIlt<bL=6<YZ2*%znf!FAokfZtt&gltedS*Dg7>+Z#~5a5&^FoNtO+0FCL zo}+H(*t<gTaFc}^O^p{i#?L6YxRMU$_=1)KT==zn-U-ywiHLV#37n@Rl(_JJeQFN~ z(!lr+diMA(b!1lxeJn9gep_W2xoGIm$1A?)B^OL-R_mYR4TQ)=rM3oVP$YM-UUgPe zM-i4qfrNg-3`s6vx%g)Kd`H(tDcWpcpuGP1p77@&ZN)lU4=0yy;i|hI710=yx%rOE zC>||*C7)6GTUJ(WcK?e|seG<}*YS<iRMWjiuF1kXC}G|%D<uW7HjM6?Z#>4B39&!< z)ht^$lY?eYVdFF(mVXDtlH(`hzmSy882%|%A8QH#N!>}uig!}yx=p<4Ipmgk1#r<U zE3rwStkzjXrFs0`vI{XwzxS6riPPDD$(*CpTtPbjff6_o?gAlKm{N&~*mI)X0NvP! z#&R+tTeFMSOe>{*M^s$shrE_s^@NV~!8s~PVIjXUWCbwT4e@2s;C%T!O>08ja`rx5 z0VPC0d7^1#7=I`~xw*fjjS@r6hEg~L@tXATbph&EqAi}9sh(`{U&Et_(30QQ%^$?Z z#v!Me*_4uMz*~yURqcyb`^)5$x9nG03BJse{wl(seLoTL3|lDC&z0zX4UjF9DZ9ky zb790hJiK@M+~I{fXS32Ee3<b~IXPf7IB<5k3u^uRq0Ka9BSAq!dU_P;PK=;8<g>42 z@_VAyD-gI6Y1~q2<xy(6UnE0#XO5<(<-iO|aKxqkUiqwqd@dFC2B$+OLXq?hm+nHo z{c#SNQk$}fpgALDSiW(?M?+5d+y>x4Q2<vhR}gwHSnkv?-TmG7_sFDFv427`h6NP2 zCxh~ZU?Khx3eEhBj{Ps>L8c@9&_rdjHVDTfr*g@4Pxic{`dKbGQ<Ht?+z=EYaXCTg zx*N@{S6P~`i?3Ok1UyyLpE)Q3<jW$v6ivL~Kp{9_ur>M`C~H($b$-<^tfiq0XZt?T z3~~VE%HV5nJmJ?sI$=dT!SeRVK(E67Iaju-nU;NEoIO3anfd0Hd`;x`?Jp`)LW$T( zt4~s(L|@{zrY#Zz)N`Hv_goBms*O*mDgj@ym4Yc{eb!s?Hwa$h_(Pc?Iq5@$b~y?t z@K38*U(&Ae+J~g}S;gciX<T9Old#<6kKMgj?(AGQbvuc@`V!%uemzPCa9GHu!fEw? zS_wMU6|XXM5x&4=41_@3ON4#4*+m3%p6=&$KZoJH#4C8E5JqhBc`vm0)m>(KbT|F8 za_sP0?^<B^oa~%UVM@Oh`~LklzulV(W1;T3BmIr)Lk5%;z#KG}XdyCNd0Q^;#D}OA zUvX>Fb)WCn%E;SJSL9fdaI6JI9wQp%F`Y$D92YP&n)D+>&_SVl+#u<68IOzI&3aQ< zLV&NsUF53U%p8#4{xWmV@PW9c?zCH}VH|eoRHGRg@RHHBD6rQ#E&(i2m)YU>!wP_- zN0R>XO(!0%4%#6B_w(XVJO5f;F%{O%uv4A!+hN=gM*-`eReM}{bwpXhP5c)thqs>> z%Smhl%RdCP{6rq{QRcRnmQcz+JO!>hL~I;@fHC28mSGf(q~04~DAXBloMJwR*Y!W1 zJ%-y^&jO@g{KicM;=FGTP=e@sQCrfz72;e~6BFI{l?)@BA-*D*cAIm7$;Uom2Ln6q z&v~;EI!o*&QJ%o;F8?-!B#_w*1U4%G)ip(cPex{$j-RtYHbpH*hR`O%&0H@G-*^Is zWRcbyRB?yUrU$$tIMs8F;hT{Kj~gU%?+^G($fpyenV<Jd2!(+n)T=kFF7-$vOcx>c z**4m2LzC6;DQ+d5s+<m=A*Nsf?`0nfJcS}4IA?b$)Zup##V`W98SR$)Z25ocA-5_v zZn{1pz7c*8M2c-n5|r;Z@jWg6p$OQKIT-$u#bv|Fu2gImQGHR2MqTF@DR5{aS<HGy zYRsw@mo%mZt-rr<Nh>GX*L)RxY32DFd9<Nd&v1MJg9lj4ygcB)DxY$>^;Y$*sP#9y z)lV$hlC#dSYxxn4>+=%3!m0t3StPX&o$S%>Asf<evYg;GhwRvOO_mJ0HS)J<;EYH{ zTztr<(49teO9ko7+u&K}Im5I4&kj$le>5Ld7iddHJ}Y2Ck7=8EnB^EG0IY{z;AXew z50b+(I=llCYXK9pBB7p269Y%`3WPUE?Mz)lyAAYTwEgyVr{cATIW4n|SI=WopT~H{ z6Cb4m0$gKo?yIuz0dK_YO>{@a{;)WiQJ<HaZrdm_+U4;D{JxHtFNqFu7GkjFN3WD} zoGifbVb~1_3C`hE1FWHd%{713j9ji4CmUtyDh|rZ?(2v5PdRsXf$SRe42iUZL!)1L zxyPcGl01Gj&)N0;ve}7*`8Vw*!}^7v<8*o3Tdh~IFlEUT3`1xm8~4fQgKcCFpp6V} z>Fl?NJbV$vP#aNMRI&i{&<H0~C9P`znxbA+{`CjEQ)Ai+OVW<kekFJd{1wyXZ|}1d zW8gAtT}{^(H!bIYbTWrf4M^EtD{wP^r+E3*&Q;}j$j5rMBNRwP2sKEyL|_IAva{@q z7nfgIOuefbrlY~w`QYF3kJPzIZW<gt=@G=1{=);kSPLjgqYvy7oW6>>a+a5rvhq2u zIJ4HZ@dStlR6TVcHWbsh!WDKcgE_wt3?;*1PL1-OTZ@Zi=Rmk-fbb6QbQ`dcP!5yg z%5||Q6RAP-BJm+Cxi3;%=X^wP-K^^A==)4Z!zAi>O^+7lNEJNCt^nEvFhj*nMN1*O zUIp5Li1^(e)!ih>*GAWntfAXH=acii^B52dFd*6Cu}LMMT*&Q;X&gF}H>(F!n)^$a zE@Z4U5nx28(1*JNR(qVjI<XF>O6S$8bl}gR;un?CzJ&DI<(=`$b_SU>>&*0j(K=lO zQKaBxYK>cqy99Q>E{h!!Dr>Fs=DzJqdyXJ?z9=U)nljXpMZSx1`ejnJrwVZ60@6oH z-xa*>OK@<zf)(AKVEhS5dZn<ed>&#vl=r+QI;JV~ic?ju(37d~-wSct_nTDSe2(=5 z&^1IBr5+~U)><1>HfbFB=_yCJf!<_JaW`Yg#oNq}u^UivEFJspXo9DG%T2A&MF`BM z=ydGTasvQCkDD!A@XHOy>@m{THgK5RBzGXJ4WX&4n>K8cviT5W$aNDH?cE}k9Mgru zcmPL*jf(MLBkl98%K1D2@yqX<F4C&naPzChuWGNhUNU@eYy&ONu)BxSwtSVc#_(ik z?3yl6=?ZZSO{PbGO==USbSa+fl437mh-kg~A?!82yV|E}y$Lu{&dsF*yip#sUU4X5 z=J#dTtB9RZ>ga=vYbJnwM|piCHW%p475y<EX~-v58<6Xbrpbq2sq`TnW_ZOp)Ic0p zl4!rK83LnSaqI@|8U%)1VdR1-j6TW5>GOCwv&i3N`-HMM-PNOYGq`!fmIgcP8`9hk zmXlAgoXbiU3IYEm7b{0(acrIsUO2#}fg1b#T7cU-$A9_HHvNC1tB3#|UH&F01sq#d zeL#Pe^K*|q6i*Ct1yRJeGCC@eNW4@Hrt`s53Gj?&v*jXOd~<I=>=mmm*me>E^!&y| z1L)v!Ia7monTno|%ts?jaFq8gJFm$a3z=nV8VUh<cz&$u&4%p>zgMP9`U4{w3E&3v z*v$SrGRpv>!>!DTYr*(cS&Qn<Jkf$MIZ0$)YU9T;?T-f=qW&=#L+ci&zu?@2Nw{c* z8L92CUy%pcjoYqrR9XDwvQ8fvQ`Vu6%_#SIcn{Fhc4X5(-?hpL&erxjGYLRmwyzoz z9m+)EGcDcn5Fo1&Y5;C+Ca(}31du|<+}P7TiF4aJ?!H!w&3HSMefrv|eD0WM<mMc! z70^G88YI0(7iSbtm6z{uZw?u@wSR5R#H@N}qM~J+xziwP*KwqI>#Qa6B-R&r&$$M? zdRBgH#~`QtS(mBOp=ZdZR&f5}q3rlJ+IAKZ%uOBeSfCazdvOm=VAXzI$a86of_P;c z41ZJpkcrvS!ZG*O(l6cmSO=jfkeQY2`)G$PY^(Y?&%u34#4aLyKv3RYlB3>@LSVHj zX`<_B{4Ox#7ZPYHqy7|!Fgsei#>Z!xYO+Ri>a&5bCcWM?yu#Cgn>jfIj^ORo)#5z8 z7>|{$)?ce$=kgNF0Wy?5EB6SzJ1$kR?^h$-iyZvEWwN~LmUDRt95S;xlgeDiKaWQ& z0eqGh$8h7_eay_Z=JOr;X5z;JpekJOeZy{Rbtq-=1Y3kSf$U=$-}h<|R*tu3YE`I@ zOl+Gh(kDCu3IJojXgd?>%*Ft#yhd}bew#jb#C&$L)z6mK0__S}@8;l`<Wne)Vg3Z+ zXi5}B$b^WQgiR0IcyVi29_O}Wj4r-m9jvXG&SSfywF!ToPtuYbYxS^=(@IJ~f)|*u z0Xh&3*5&pv;fR6A=srITSreXuh}jNX<Lod*3DQ_vzY^@82>z`7(?S&?FbW3=DV{4X z5eYKj*m;3?_rADR{yf@vQ~BMW)l4NlLO0XT*JraUr+FUjgkxl4AU^PFtsl=aQ|EZx zEbR@HdL^SGuUM-<{>QcQr4*#YBn(%{x^{KN!LdRn>-Phxn;8iznvuLSa!)6#Btg2B zT@t;b7gGzs{+J)FcKgB9+nkd-R^{4P7!YZ~t>5o*z5H8_xs|i>T<V*Mk!Xc7X@*$* z=b*sJhkfmW@tSq=uWZ7!$19O4f^h%g`hxEvyPFj^#QU_4Ber}cj&0{G8P5V0y1M3< zNfNlpq$AWAhYIsqz0>d?wX)41$<I9w@*V$_VyM(hJ?b#F#o_iT=bi`D43Kz+R;TXe z7jIU7cyeq)Y~f$;M~ot;F)`0?HuiA|CG@@k6No`ab&GzqYvFgy*1#Q#r+!K|C-qI@ zigWtam7O|Ejk-YM6_Pr;&a07R6J)+otIcN7JFoNE)!%1x>hSm_>gnkGwBEIv5lDvw z6%K~4G(b6g#Pf>MMs1>)yO~o&ApR=HqbIbzkulQFi;I()EL%1S81H=GSisG0J&H@c zJ~+*=KAX!#Ne=!45R+X&g`Qg;U33_JRSDsp7C-ijYr%qWlqDK3C@7KajAqDN;^Jfc z#($DX8UKNuWg7ChXkY7k){IgPccTNRy?f!B;x?ZRB#w`nHTt6!OlgMB_vHFIfpl(L zI#@NV_G{(?vd#?Fbc1Rk^3vxYUQWIa#i2LJc@3$|u6Me0UoR3Te=I|YsA<2Fy?dXc zf6-Sf98{@Pk}=@=fIW#>5PiAUbkeOjkL46u86eYWZlIN-;Wn6W&K6Z|W7Dd>fZRvA z68Y^#Tve9ePoBx^)W`VZb4h-*s&JZqv&ow<yVdQlL0M+0Z0bM=y1noA;;{fb)Pm*w zLkeB}WSLmKCuQ1xV;1?}UwUQfwKA&YkbTC}qa_8+er!MmThR?2+p%?A^0~vQh(IX6 z#@No)r;3kOt4vSNvJkB^ChBeqw-+HdzWMvn`pWlS1AnS~9G9UiT8px`YRTtVqh8}c zcnO*GlMudnKU*zytQiDy+o5#6S>12i=21H#s9wPjK$g7IwcXu%xAK#}>Wc9=IIB$% z%gBb7Y5&w^>kU>b=)mmRpXNJPyNxFe)5huK<=Hz%qnzL_fcqe}Xo8m2_m_58uEQ<8 z>%QR-u!b=>ty}bM@lW|g*b#!}z8yAd!U&O)(6fjSi?Y)I00byp;fUx=6<gyQc-;q@ zk^2jQ@Sbp0BjmMVe9z}pIV>xu@zHfL9x>Kpq<0Y`A~$vcGe%<Tdo1H1|Bl%W;=~gn zdRQGwycr+q&il(znBZm3rH@nV8~sQ*Zg_#_N8VJi4g=53+uNig&;?c6s{+(7uDuw! zs^BcH-v<k<mOk|cJx;MgO?N|b&<K?!=cxpfKDnhL^S#quZ2^YWcJTB_ye$kPUlyV= z`F?muJ9up5dH^??$ji$O1;BV;)=u>GsqB(>llvT*v(>EGsEqmhW^njc#pU_e&Tz`I zdWv4Pw>~aQp!#g)$}#kbY{f1pioDr-HrVd12y<Nd{Sg5ML#e{+12nIjinvwC!$bB~ zudjwSFaFNv%!vKXhUYQah4+}zYh}}tiIXp%Y9`(Qp(t^Tq=)IG=F%d%4_kKse4yab zn2x+d4eCgvq_2)$hdhJiDOz%6$H%h#wCA_egk;(~IFfiKqcMEHYeLD#Dq3@6W{xwz zi~z#6DMH3*=J(9!x16NRQ3<0tHY>>IEuBgwryo8?W)#O>`;b+&1e8^xk^9AP?H~IU z=dSAL+;4cellctYv{ATY_Cl#4+a`spR7ll-M`UA$iX+y}%cXcVAy1`$#GDFImY@&E zEOI`?^VhbMhc2S>k<;F!X?FdZ+qb`bnMR0#yO)4<0@IV~#)D_E)I>5H=!&d?@1HqZ z`viPxg%3Z)dFrb1AX(fZj8L6u+Caw`4!5V6TG*Y{t}-up4Ug^f&CO<}p7lyrbt{{9 z{nMrGQ@E46jgLiWnoSR-wQXneJtb1h0_HSu;GVQPtHpnqN&j(0yyI*&+2nXd^;PI{ zlBE@!+3jt?sP;U<sI(ao#yFTmu@O(ionK=AB2A!$Z)(yKh!yL8Xm9k8>)GW)Vrv88 zDIWu!L-!QQsg@x-l_1j0F$CE&h}!MwvU8sm^w<jxesa|^Ie36E5&@JKr<!jI9uOKn z=KNt{mbRuzGjlys{vP#gSIQ4I5M<TRZn%4{vkRT{$f!Y~C=fYwKff2C*9-H%ukG;{ zMghQC8m%%Zq++D^<-Ab%Q0=bFllO2KuFgYQWatFk1NSxV`}HWdeJ~{#JK2twl@LOE zABD$7d@_hRO*ia9`B123QL41n9+f+pbLNh|7;M`Ofxd`x&+LyW57cV?nGdP7lpJf7 zHP_RbN4;b?a=`H1yRfvffO_SLJX}(?`jZDV4`ZaMqoc|;t<{6^pP!M^?_~#b<0`oY zgxDJav>8|iY?XJ5iX~cWATsWe6*g8fQg9EJ?=Zlsd3PV}0cu}&1Y#Yu8PxX(gMQ;Z ztX<E8!o0?Vzkr_DEQodEboi3p4Db#z;ZMhdePeCqus&iha8q*u8a@Y)@s#`Vojc5= z;v&!G`J7rgG0dW^`rMo#z4k&SgRBWDAxC{M4@gIT$jdy_K+S4yy0jNZql``COrT`_ zBco_0g(_Bk>ILbrj8y|<tgmo895vAqiW~(!w_tTFsPG4o;ew!}h~9e@HdtBVFR{8{ zk_{BOx)O5!eBPh;{{Nr)|M>{~rrFD5@aO~~Zn+kbBzkfn^+AmnS>7`sO{)O?Re%Ow z!lGh1nk-M3?NNXbyzYi(<;m%+&B7YFc*KsqxQm$I**zbDKP-DBJ}@a7E#8cg!^(ly z;F&Iw_0@YacCT?@ePD2U5*$u0r$~3BdQ5oVdF!|PIRFuPY~~1|BOGQlNjPGXtLTBC z>YQI;RjGL-pqAH?YjJeSyKyw)&OlTxrcFh_E34{}IMC^crAnAeph@Ca%B)T;=S)*v zKD7l{rj*v{>kRG<@e4i<=SV)ggL2pMBlCpx)lufJR9!|=kq}++XN}mHEmXVG+1{8u z`Q}`ZDklait8Ydw58AIo1?hlR%<6!Ww?6iQGnc;XXQdQx@HZX!B9UGysAg3XXozp2 zwsd-EL<Ooe%W0p7sQ|#5$nJ^u!&$(w2dIWgSho$@BU`w9Zw$J_75@0PIWxXJe4ZF+ zLYm_2W)_4==E&00U31hYblbjn@^BfLiKLx^7Q#=A3%`#LKlr;OK5~Oo`;5pLOwRrY zDheZegm@F^&;lpw=b*NyDX3ZSd*q>P=3%S~$@Z_`A0-16fTcQ&B+qV;IUEDX%6s|I zssQSjOA>o^>%RV{bZD=YZbsr0CVtJ^4y4iJyBO8IR~LZ-y`aEUyd|UvnbhwdKVvhJ z_W%tC-u$gmev!kuCK>qo4MOsrN)+`#ck^!oh+&qxyY}x&4PQ3IegOn88AG&P^H8_z zyIFt|Dlsv+dt`UvSN+eh#~*X>iBauUuak0b(At<X+V1tlN}6=c%>1PDZ}_AiskQ|? z0UO7^B%CTX?MQ{x78EijszCq=A4()XvX;NZ8{X{ik0qhpO^dCIA@f2v){vpE9e<&j zE6=fZ+~x)CL*Ow7=nLIa-fhtTTiHSdRNHP_pA@K)4(u89LE8t&S!9#KaiN{uv}GBm z`=<P?su=WXFfUR`l7)mf+0X2cCTrM}l?WNOj$)t9YUUBYOyuLrjnUc7ju2sHpchD@ zmr@ltI<Ej0mrl2daqrgB*bkfs(G*jw4pt4~AV~!%SJSDal>#1b_i9Oe#zKG^Yt>a7 z6~3_#opKj;)>|o1B*<ymrV)t)R1`iZ`q(Jfnvlp}>Rl@-A(jr?7|N-h33<+6QPdIF zJh~40x-?qf<$^L%E(ZlEQvwU<j2V69`}y@TPU3QD@w!17<{q#m(au@?YfExa-g-@y zIhw+}Jp5x8Z!`Kczw$x7l$~!ubyIo55s~3#fNk>6UngwfEp==>qB9Z&4lKvUN0l|F zt&Edo9p3p|5AAXa|0a@+5Orj}k4+4wsG0#Ay%Sc2cpV}`Y!C6Y7FV8{aZV;mOeQUH zR#f$KR&T-;U`xFWH6Q^!I7mcBS+vLk(}TiMj^p{f9-*L`jtWcP+Ps5|o@51MS*)sF z2x%j8*`FJcFo))&Y-*NyVXu-6AiSXu`K5i|;kELY?dkmr>;;pO?fLmfPUi!!&e#1@ z!h4mA(k{6LfhxZ?Tjk`&G69HUUuf^&f6nlu7})5r*NT~<XK|_DBhLi*^U1;gI-1<` zv|7B}eN!<Z6=Z4*{7l?1hBr%?Ej8mqWw2QT0mD6tu*5nEW#d_<ThPYwL`N(Um~OXI zfqEC3s#Whx8IyP#Dy5rz<Nq;qUO+-Ad0j>sXzuMpC4e&_|9*BaU>jHMxZ=JhyGi2s z$5Azn5Wa{Mjr0p(@W1(6swcVj#>Q)oysv@S0l0)bt7Ik(IT*SM`UB@2=~UL?Pr~W# z2tvF5y58(y8g-|9+zeDBsyMxIWj7Vk+SL7oyCpj9KX#$fayvT{*S7ig+WIQN%vhFA zB>Ii-2`x4SJyT%>71gEm38#|MiJMlDZ($lm#uHiEN1sG6GZ-;5%=Y<`Z21loE~lJ! zI1c&ymIZBvRzvv?zw)eh^Xzgg9><67Qb>C#+IWpyV0YzsOyY9#l`H$v+qR-cMg^`) zRCz8gNAvBJ#1q!$|8z-qW62_+!Pr|M82xS5FqR=QF&kvOYT%&LV%O}V#_E)6U&Ex8 z9=5)4fwxXXxot@uA8xu-rB5sIxlY1L!;e<$pri9$y4~pQY^>40)f^7QN-^KZVA?KM zATN&#<>ClWE648MIz^(6O_rKf>@($VC6JmBFJPB3DL{rBTymc-sapCtE<q)?B9T2G zFX&W<<Am}{ct5F{L+NC9GR3=t^5ZoncG5!zw2|alciGcTYAXa^(>3B#+_Ih354yNw zT<)bW1Y?EdP-v|&_WQoV9an+g=cS({-5w}Rm6>;TZncEeaAKwNjBPq|X8+_tO)OI{ zRJX6NL(a{z(y=R~;B(uEEY4JJ><=D9#W42Z2uX~=?3Vs^Cgwg-Ak~MC<6(`p`lgrv zTncxz?e!`ict?MvdP)Uih-*x^svPg9zWG{-d~H*;W6TO_Ba3y8V<tZ5Z-3Qk9~nSY zRoBs_ghK2dgzMB!^zPwenKz&|LaLMHK<utZQYhnCgdaj=9@o$A2>Q1o1$h*pJhp_J zz;!y4HU3Q}D4U%x;yvMRHe0MYgU9#UDA-5a`%FuLu%sWyu+%UlU6iOo!{ERN&u>7! zWv)`K1=*Iq1KIFB`l!U?W+sj_i%AkM?K2A$U=zrw^ZRQv>TJScd^Lxj>)ZJpy&Rhf z0lTdQ&X8`3ZOau4$9e7%6t;UWXi&nLpKeVksSHkMc(`0+!DhfpLb&PE(|R_ZhM?Vd zOS_No9CFk&Y2fhdjvid;|8wDj`Vl)dwzR@__XptDyA}k8u~1BSjW4J~=<jbLTlnX1 zw0~hsp&}AEazt>Fu6PxveU9jHt>QeA_k*c#;)sxmdn*1*^1)cxtikIxYE>UkLhPxG z7ufpE-V9`ghA)1-@atWWkq<A7`le7&n=4UTr9s5@aGrWX?`SjpxDL<YCrR5|FMJ=N z$iZ|O<Z#OG+STzrd{j1K!?<uaUuE4?6=~=3Tpr@TiRnp5o)+1qyLl}v)6X`EQN^VE z@$mI!0^x*HTnWXzH)apn56+#L*+*znLfn6EecPW~560Iu=8?z;E0k4@7ebHqjtTHq zS?B3*|1zP455Oo;_4zA7IY)gccyey7)TAu;6rrf3Yc|5zd$IWKK=r701(y<uK#EW( zNmz!r^E%l@JB3PyrmY70gCdPt^DwhFzdOJ2CFiKX!ROyK-<0To7b?-j>?T=~wHWpi zQGZ)Z;zRdS<@jSFAsUrY>8fY!Z!3bB<GG{-Z|8`&qsMb^pGnMdaLY)uK0v()ZHAw9 zyH1mQ+kufk^KDT+`u7z=_gnFh{1ERnJoOiRsPVCDyy47b+Wb=*UGkZ6YZDc=Mrq@6 z=<juyf3x>I4Jxj~Z`)%Re<Ufu*dwS>C_ID5t=yM#+f60DP#0I)5E9nQHgIF8Nmn6i zi%#SX4bQ@p0f!DNl99BEdN{lKO$j0dtHBU4Xd!@y6~vtSgt7@`fAui;Inbmmk2&6J zyqzScgm%Qbed!6a6H;$zP;`DV3ia9;O|WDXQxaXS##Lj-_fG|W_7>E$mNR72qCFVp zKKB_!iP+d$=TJuLqy)?}6mu^Qd5h9(@^BlvkJQ**K3DVEddXW(DRwI6nrB83$t3Y+ z=!QjoC}y23Fi0t_e(4dU$0CMK8s2O7ZD4<PqMEnUy@NLt{N(mMl*irOc1<;#?d##Y zD=&Ud46_aJh)&E(vQv$|-|^IAh1SBmhHfj{OXj!}#~92$*4X7V<2FiIGWEB8q=>H_ zY&k%tcvm%bxk|WF4UDx;=G$IlE(6sf1)Z#Max^lpH{YOdIt4qO+L}ATvze|Z3@+V` zQourn+ghGuaY-}s&g9h=f;;7l(z3t(o)d?H>m>Z`*Zzxe>P5|U+Juqp$252QX2!&c zJk^YLsrxnz^P{T{J$))tD2N)&W}4><uX4si?p|)&Ht;+me})_=<{?CznB=00$QqjP zlsjyAYU32QW0UgY4Sp1DvgqK6=gW=RuoG@?-hK8_ZQ0W|gYl|nKE6bZ%hi#8cEQm< zOQ8XtqgJ8;h*ZIbz~P5aoBt;{^nv8V<x69*CZf7AAl8meSN-bcj|UwXeraFt&*?2A z6toRw-MOK~ki`fh9^Ivr6^;LOrbydLk5bU9odX|+w7fN5#BQrxoh@q-WgnQ`V=6QZ z_mYh&KTRS=8Zp6AsV3Xr&eiFPFq~mFxKikb%pOk!v6-Ea{WL{i<DT~?%2p;cy$_@U zgEnl(6tN8oHoNGvo6p|FD$(wvnM+1@V_N1&M#?@OZEY=y0gUnt$M?i(M;yblQF9wA zb%khWmSsODFPrSg&$p?WMB~j8G%q$Ah;QH*eI*kK72Q&&ZTiiJCbR5N`Mx$WDfNC* zkLJu${~*gNkHDn4f|q6bG!UISHtneDar?uaVt3b=pcd*P{dk&xOCS&0D5+XcpqcT9 z?|MTbYJkxHxErI{<9+&1UP@xPI$Y+1?~gK>qlstDh6*~#_)9i8*u$z;w)(FXrm}oh z8DCm?=yyz2>g4oB#$EcsdEx2CXoe|Bdfj2Hl$z~-xAyJw_(CZ&R>f8n{r4)yMqyUD zF=C|ML_9j~CU3f<)`i!$4C`Gash{X>uR?t*gonCf>1;vSa7`{%UxjF?JC37JqlM+= z7JDRLVARmvOBU%Q&0DmSwyD96WlS9T8Ulq;y_xH`#vO4qb*`mb=1kF19=TzWd4uf) zBl<nAmG<|Xep+B@?55RmzODZ>?|WUHv*vxVF-yO8Tbehb!jv^+NfoM;m*p{;^I)4I ze)!p3a~F}GU9e$C1AI}WYmAj8jRhFFxzETklFIpw%klpNlhpuHU5Ub*go!5p=;>YX zqOg5{?$Yf_#yr>y^^=CEq{P$E#kQL-6`--$N=X)DGSb62E)@xtS8*bDequ4^M|a0l zv?IM|ahTFl-zW(k)WzQ*udE&@jwVuyek0J<6K2M9lLWYac3B4VrYYHGU*Ds9K3E8o z;}Tad!-XFZ%Xg&lMA`4YD9z5|5RT4$`_hDKlWaCui!#NzGz_JWJ1X5|coe|1jv3Ns z+N8Lv`SJJCUz|S1I`TJA|5dnOgw38Tvnn!TcOqDRTTE`P`h?>+zcOpGw8~)E|009_ zZWC`*K=OE(_fjsRMBGp38;c7yVmN8kypJZ^^zpYNlf}<GuQ^(jFO!*LU2=>~LyUET zl{bAIW2|Pi`}}5S(qv}hvZhN3?cEeU3tr(x_rCZ`qW%UjutT4~U|4X%g@ThVA*Ghc zGq1SGDiWS@ts8}0&2gJ0)OA0^n0zgec@NUa&=+hPc188Mvy_|A-}XLHVUQvWF!4vf z`iX_0|87L4K=np;M3qH>Aq5M&zGWbzei+qyL^|;q`KWq~T5h*}jvR-<vORBDfFVJ} zw&jGF%~QDV)4@YMjuxwUU8WzJ?wbF&X4dJ}p0es;*&X;8vthb9l6)8)Xf4rtq5P*7 z{JI{(qAbJ<h@q|Z43s80Znq3X7{oyb<sYBY_Is)QhPyiBnU!u5bmLnuZPUAcWavGy zSWw@7K`<DI%zj>+-%2JSQ!-q)J<VhtY#Jc<b5fqqSQ>U%eN8qDV6W(nQXeds+rVQe zjcUPcYkf!?o7U^bo3Zfi_^s#FAvWz3rbYksUh&GW*t&)0=La14PmSWbND4+7nX=hN z>=8OB?agN?_T_NitT4v3P8{)Q!2+JAJS3)|A4igg84LDOf}gUAX>U@CjZidH#b;G9 zRc9GGl?%6--uH#=So&#mcd4an9!DBZ^<^yvlisO~s<IyAHriBKnAC}Clqc>!s><I{ zAyWrIrID)59k!2DfxFqVGxN%B1?pTM;r(B@?koTd8L2sS3~U}S1N_w%xpTJ%+<rIz z3c<f|8H&!HRuDEDLv#uKYvC=ZdQs(qI~)g`ALymg{xF~artg%{0k1et9R<24<q*zE zMO(+6qP=oTymImP3PFSirDQ1eQn%THF=z^)*s!&uA#Wtzz?k`r2j<%FCv1HA7z)kK zenT5NSjha7QC!UgdBx2-#BTnNKOMMEBq*p@brg_7-`&g%Mzo;HK{k;GG@Dph(uV%^ zzlR`n*bqzTNQ1uPqXf8Sb*}>#4irQ8S)E*1fck%HFhNOqwKp;9W!+B1%S(@_ucNCG zS-CydGnm8zuCGIg&oy#8;?$TcjtURmJ-E5~9d^FD)}1qSINNbSTWA9Fw~5rFvv1T9 zp9y!bb*l8+V><>(h_0`{UpXD2ceuU0Te0>dkU~>Kbvg_1G-U_UBvwDUHRtyo-Z8qp zRyx@RglKiA68#?gZKqm3%Rg)-D?u=ze58?1=+G^G4Pn1&x!2{|Mf()7*N3Cl_~R)h zTNf6A-z<Gs6FTKeHj+*&Slsk7s|DP?7Ox3N?v&S^In~uA5Z>h5MWC9@zGl}7e|5j^ z+%vWBOTQHNsQEi**HJ9-mWO@c1Q>df+=UNtXlcQ*%!+f6!$?}fbx>6`V4MGnlPWgN zc9^9?)YvYsa)gQHj9c%lQ6{_zYjBjz4{hhSGTB0#lq<)^u9IZb{1R+yqJg$1eaZ?{ z=)(5^ZXSoI>~66EJZ_h!oeo;Zp4-UHS1t9^mr&j2lyxYK>Yw@;O}9xbd!KWpy{o~| z3DhVcizOD^OE@w;^iq#G6?MwERqlFcJmaHtF#VGy&F%_G35Tx15rJ@a&@lDw;*I%+ zI5pK{-RZE%*KNpRtZFwSW^vrs<sXRH#f4VGx@V81dRz=vVjr~?lf`;Y=5DL$3r}9W zOMA}XRINH4Dcr#ZUG_(KW->4raWMofrJyN@3zl8c19-ub>5)KfE?Y;^bF*>20gL)) z70bO8I!p%VKX~B!*Un>{+h2w%A(;j#V|3@UmYb$koh^u%5)qDS0?>x4hNdTis56aI zUOXgb@GQBNk90Hvq{`~<%L?r8dkX_BaMzz{A{6~SMM{KBIi;O0on!V0<lCA|XVp82 z9y+LEcGqV+L>MiP3RLO)6#n#9?*0=H=Ah!hRZ)yOGHelph1$(TmsDm;8T<V8nZYSK zyT<|2c*U}*@qpgPQpPIYOFJ82MXcsShH_PGn{Z{l_PwyTLQbRW<kqxPz@jh2{9u`e z6iMVejcF~dgTP+Cdh{_pb|7QSFdB^qi2#L=*)$_Qhe0$y_sowLT$g)Q+KPKSm->CZ zyG!WU4)nWHW$m1!8kQ=y<CkN79yiwhWH@Ty5-dtG_b=0T%hp4uHxFjChm&~R-uxyh z7}$ct-SNkh@*owWu873O&(-)USTigWTDHiy9sUW=(Bt+vo#@Odv{ch8P&38i`ia`f zSDT#p-83crl8ZR30D+Eu6e7~=ZUOCM#N4X7S$Z+hjo!7BYJWoA>FD?S!^Kj|rJ+wA zM;F!}>ETnFyd`4OVcwS|rmDB4EE{%Z+uWku2u6H_LDz@5#5Qv=HJ?@tTkZIye+lf- z$C|6$Rs`*gE5d06_qgt<*CSY_Pt+9|^|n9s-TBgmZS+_<>^k+^E@~Wm1}FBLqi(;6 ztS*HshUSI~^IeMyRROGX3+BhArdm&v?q-|b_6c>&kk*avCSRYoak@0rVe-|E(>iJ$ z%)V1rlM$teod@f9p-&@x`CUgYyk!1(`ZxS$CLaR^l!-Z&Rt_|YUiEVeNs6~A8oyTc zXLA#6E{4QAGEQ)rhkky&TD4^IYDm1eAhhHf4rx1Gm5zBO*Y^6boZQB!AUwoe8P-ze z2}S}GTvlqWN1%-r1F~lg`+X>@#qBkIB1NdL9AhZ1c&(nW^KRkO=SuhE%Rv2F8w0*% zL3Sz6O~b&MmsDK_F<s(Rzl$@}JJkzA(in!m#P{&7&3X{?(d)T5ywx42-sF{jBcP#> z$!$)vK4|p&Sd=wuW{Jw_w#OLAH^j0}&Xj$>8{xjs4!lil=FZCztaa!NC{uj%vpZ%@ zH9*%*b*P`T`{gQAnP7F9`PB>252kd>nL;d4n`QE(BkB76#RYJq0xq5k?>3dh#7l`r z*G-x&oHJDM%|9UqR_AEtM1)PNjaRK5woYGm)KGO&XP~z*Sh=F+|2e7|IW3Gk7rHO| z+pLwjjg<G%w_>=q)(x=rlf>DnJRBDPRa0D6l2pf`S6$7$%IOj%NXoh?5Op<2;C8bf z?ptWp*)C%HkyACWJUJTUkqJ?m%bWsh0*B%EY;}IE2_RZ8o3hR;34so)ZI*R(oCyv? z%pEIm%~8?&O?HxX2oYlOCT_(KXgapBb?;cHUS_zH^xPRsA8`})i+W3`+E>4`J-cDF z-kyBVP*oM)?6?!D!^QS_78;H8r9###nrjO`!ltdUu%>~G6D%8-e*Y(O(Ts)jgQ3|j zICLtCnOtT}DUs#KS<G_U#}fxpI;!u*5+_&nJugB(qT_alBa$+(uoSzDqtbv>;!BhS z&g&N3cw^7m$~i0Dg`p_d!onfr>EUFAtUr-dHwtWe%F{f(39#S~ou*vvYd6+KS^3<z z(5X;dvxGkG>nN9*qniud4`kN&<A~<L5CrM0m%5V_+PTTG&2^Z|rpBu@g(S>yX#|;D z`zF=%i$m;*oI^~iNNkW5>G#>yl=gc0b_Q7F6f58F-@Fm)T$l94uq3mH%8{2K`;rWp zi>s%Rw3Me&&XOL_C|e1dIgr#ayu7dvJwx%-Vo5vwA86uII)A_y9DZbpQzs~n(-;3F zKiZ3v5FQ(eq5b>W`{jv=A4n7kf@_wvj#VE>qRH?OK0nEipR94qVwBZuICwxhtYp}g zdoe{FqhW0jp3&$`&7<kLsCS1ldXA_A<8w^bFOecG{hQSJUjw>x=BENrT1cE;pfQ-T zHmAINF6;{5DrQ|2BU(O0Z04bj@Zqh&TjD_moun_3^FKb|iqE@zHoj}KxDjg5n59v? z_fE0EdwIH-iEuf^x9E7PGB`d1=;_{@{#a_Yu0jNxzvjcNjvF@@crQ8;O*ufPnT>UL ze%$>LYvpb~Dg~QQ9wEvMIfMaG{S474R5bd8TB3vzEdfyUHU&fJpiew==XtYqGm;I* zYy^Gv^Y`j9&s>g=s^BGlyLRfL!VxqZS1(SB_U1QN1Lj2Z`b({XlgEwwipSsbR4<~x zah&;J9nB&c;=$`}EK$(w(PfvDE$FB#e-ltF2`Q!3cxwI?C&2WnF^vKJ@X)FK<=6Vk z?EkB~EB}V_{on603}c(f2-yu{tRqW`WT_ZhG_p6gL4!faD9cRtHOpAX(x)ti8p_y1 zG-#BH>>~4)vXwoA&qJN_{pI@?eCCHaXU>`P%-qlYx}VqUx~}`aUik8vwH8V5t>nye z8)MIu?7=novdZaG-1*KN-$F1k`f|IwW+PUiMs}@hIl0uJ7;MF#Q<$*^K*DnN?bCMK zFAQS$MSS){Tg8ncZ9Nr%{(E9?{J*obfhFx9qKI<>dd_!}^DCod`rEm^ohN(_r-CxC zNBvML$pa^JM@_C<g4aI6;BU!j+lBy+VGhZNwTc@y34FhP(qpQR^gte~**%`zELkWy zspv+jE+Scjs${K)9~k9Iisp4814S5pH6pXv_6ChC7SpoUQ}Zx*R{%$<FF*^(7}8N8 zXTaoXn2}s;lsvTdl>SBr>{zslwPX%iBq#j>iS{Ah>8r-eBWA!J8Bv|_>aKf};4i$d zP}>Il2uCyQC7^?}1+=laD66?6gknk^a>07V#zO_oiGjQocY)X{dTnhf?sfdMrO$e6 zIc9)n>ZugLMt4M#Y#uLv0%cm3a4im5H_99K5JgH)80u=J|Ior;w&9<-c}2Cp`PH5N zTk{CTMFB(!lvmOI!3ki_0rKajVtcFhJ;l#Ciuh!aipt8L$wd)Dp!Q@#S)7(vd#kl~ zR3lF#X5jXg?g(1I1rIRNHjSEtwL<C}()Fv$imRI&z&TPw-IL}r8V#Wqs;!RGyMSuk zur+a#y^AHoPA$@`wu((Hn((qyW#_Q|Ukq~>z(zTjG}9QsY)TyB1+okwVtS?|6@sNw zX04x7XdUc5@xN;Ls1hV}NL3W=G)pIjlyd}5;e$vh^L3<f2_Y45X#trSt8zE@g{PkD z;HA6~v|>54d@Zkj=$f34b5o{(=>_8Gk4&C>vhh_sC-&^6hnHWwzsX15dza=xYHgq5 zRt_SvE>*n$8tLB>d4a?`|LX*2K$EImDrlz{FxoAbkoK4?pjXf3G?-_c9r_2|aCc`@ zPF?S1dX2v0#&U~O+Hp#xGJByYj{MyI+Co0&OWzaciwo)_xL0`t@KV41j3rYGb_5|; zP-nd&f8tcW4!_`RspDB<m%3WSDVJtDsiP&8*7qC}S*b2iNES999mSXG5C^-A3vCyp z;)G>Z+7m?<kr5A)ibX+>S7~*3JIRcIjSvCWbAcjok~rG$HxOVim<3`0&^R?nQ1JTF z%Z`nW8F1^+z`gEF{{7VcWFgA`(#qYOdK=MQ-HVGlYTix;;M=&4e<FQ(1jj!1Uen4l z6mu$$4vzmaZ`ai}d{E0?m?QzfIH>LY$^oPA+XjG%M$9+M!6a!wWg%%c;SyX0(f@uq zUwmcoWrMkDn^&vZS|Fl~_&iJ26gH_rtx~-wAXL#k&k9pve_u$!M6W~RC60Yz25k?{ zxuNWJ7cAm{p4|;3>j()Z^c4mTnv*&TY@}BWFY(veu?{~zXF6KgFOiML^;?NmULDOP zAGreLs)r6kJyH09%7HmeQpsSm1`{NfAF0gG$@8=>wiITac)>+bghDsa2a>F@Dw33N z<jTSd63_%h*tJWn*E9tk?d`PPGQv#8*s;VrQM>dfAH-aE*L>oK=UV;DL(AM-V>=!! zvMMU@XQ>PoOgR|Eb-<DDZTs0hiEH8F*PRJ#)tT8b+dzoFhp{ezX}BVX5~rkLr=K9D z_j2~!JA#>bYJ|woEk4GUMt}<KAg=i?EbK$Cb<wHc`#Dl#3;1MQ=!r@`_83E38GYvF z8$%CSpmtHS!h0!Ji?&_#-~_UcayCY@`aDi_TYk4D>eQ`Ap_JAY4`5=G*anex@?kYM z>m^|P&ez+{fo#N5wg0hEl`I$@uLksG1(9oaI{X_-e@QE_**S{-xJjo*23uAkvI*mi z)=;$;;C8v5F-Dc0tu^Zd54b}s=}p`9WNMq|@fWv&bOJ)UOlMM}Y;2)DiN&JY;FrB+ zz}fb8|0vVR2&3a)34SA%?Rr<Udd@oaBw)U-148R%%mc_5Sg+(~ytVGB*McaI{8fCo zzy|jc$9eg|CDfZh${+HsRU8=jri*wz<N(&~uYEyVJs*+B80?skQ|br5u1EJDC1S8C zc~KU!b$XEs>{r%Fm}Os_;ZMH~#e`@~(9gj_wATxTk-#NOG6%2Fy5sziU1J!nyo8S< zL%Gxe!Mj^MSZ#&0LO>6(HCLfJHIOSB;tU^FB>%*lRo);jx}uI&i*IE@)1n(y$M75E zsHw#h?d#qduaBg0N|<Cc|6ZF>^Sr^lBI@#XmrEy!>BgJr!3?;xz4Ml!N_<x>W5xo* zVw?)x8fyxjT=j(oJ>*jTfcpI6Gm}s0r@`yH-T7dCI_Xp1%&R)r+zFr0Xe6(UN`$Xt zMNc{@J3feE-6vZw?UD^YA2WCm2|)nek}%%P?AeEpfBBr5=V8F}ul2}86YZVrFm<l{ z>gtzH#5wAeZF4|s#@#3JG{mbYvMD7O84{6U^^|bZ`zhfs_4r}wUPyADi`$9co1LN` zON~Xnmkw!}xq2i<W1{O<gR813r<ZDJ@}BQ?91~p23iTe<zSE8a=8%wEhob6z*WGat zwd7t{Aa)sYVCl2Pej8!___*qC1zsF<MK}nF>R>ps<<)V3^nn)RX=oei9A`QYmu!mm zZK(YH;iEzp=6=GR46$H=o<uv3B1PAm&>rXwivxNbSM5Ur+BMjJI69w-7RWJM=+&DI z2SnDEMgm@0zQ!)dfGhyup<7O#(T4}=9~3MVs@>QirNKVE6unA*7YSJ>Dj(_?rxhNg zE}cyvSg3+K?2bR<i#`ezu)+!(e%4JsmW#v0y%N^EXH}Pt_`uk1E|a*m1Gw`4gO>?l zEo}$=$S_zCp7kQs3WfsaBz5TZ#W7{Of$*pL*-Y8D>56*{`TEd4HuAKxWc%lFUUKGt zip8EAX-cEc-X~?Rrue6hGzF2xYAwo@ZG&01JUFl`vOHPScsMBdJl1GT&~2ww{0*RU zuMvRK-~W6S-`4SP6b5gt)aJmIdGkhvHP(R_o~@K~#-^TN(Qm(~v{?pGa5T4buwemF zKuq#gQchMgSN4;&KG&4UVBbdPYggRv8`<mB)$v@bnw2nuDKAGrmct*;UmoWE!p2#^ zUcgIyoq=K2jk7!><y5IcdZs)J7@X5-o+*l&vkRXxFXOa_Q$9d@FrM0FhSEwTY3yj_ zZ2_{&8v*`->+<XWf_)7ljfEkfcjoI?*gNp-RzouYC(xk~36}&c%*dEm*3_-DZCp&? z@i$5GRA0*d<w8X)3>Q5q2>;Q&!q3rSaZB?!CZEj=4clMh5{LGHjXhrroBuW45BI+` z?q*ZIVB35R@l1_2h!YYkxxU;!U`|5ZnkYl0mq;|aeu}Pycdl)s$a*i=Mm%gZRoaw# zeO@0Xx^!1`D-`YeZwB@%kHa<qeZgfx){Tu2z8GMm5t#TQQT(a?NAokuR8xQcRdH|0 zZ%F_I2m9B$PwgX*??3069#ZXs74HXBw_>(fBqwW#D!6j>egkL(x<3UFT>}{teFO09 z=YIIeayj+zToUg#fj|09Zc64i1+7f1MH7XtWutP>Mi7l(PRppg#z4KcZQ<C$D+dpo z^>v1g)#6Hf3QwQ2`&RVX$CGB}N16oo#Vd@nO2H#xs!dZ*yw;swV3-=oqi3{lFIH&W znI+0>=Y5l<iJN#m`CM0-QTcoC5r^!Yn`&FdXqV98az)FYu$-bj$90HHcF`Uk&Ue}= zU+`EjW2`qUaR3>?mCgA{mloJA>`jcj)4IlYkp)tY$XO>f+iTvAo~$?MPSt(ZKL?ew zWLwQ(zEOWCwmM<WJ>v3dxKn*bmcwu|HwgpFt{<q25#~fDGS_u-##&-3Zfp{1xp#hj zSHMEHCXlXgBR|;y+(|`9Y5JvWw0^B?K$}mPr-chOc^}+ekO%}A;>=SUQO^&k^Nf8X zOZa#C`yn8SXfd`ndW5`_{0Re!@X+W<<?F{dGm#$m>RgGeuhQ&g(xyK$SX7GRC$hJ1 zbSB?qqz4Gzm?v0F2AIH&yD~QU21C=C(;-k^?&kU8KB}*!VfTGC%QVI@5ua%Kvg$<^ zJ7cP!)Huti((2=?ky|1w2x_?Lj^;Tr($B20k`U8zy5HEXnRnf4a6akSP=Jv?MN=bF z<#hw6BKh>5X`<tjP3B2%o_z%@{0?T_;vd2+C|_?PX6|dtFx(!mOdv`L&;O2b-<JA; zP!;Qe_K*Y+=`9N$*>iC;4D74MP!i}W>mgTThkd2FV(Q!Un{dk~+38hf0eSLLXw--k zpRDEVE$@bladHYeFJj*+$iU-p_R<%*UJluFyC0B*GSGdu1&JfOvIpx0aM<~}cP6Oj z90dDRwd#tnWQ142nc3@^cUd8xSrh$KuMVrx<!N`4_MIwwC<`}E&2QJ4ptN3}l7v%f zq~Wj3*&h1{>HVBC^M?YjG5{{_HP7#UK;(SbpP*a~mtX|olH5d~gv3M7{%+dYUN0NE z$r~M@XpzbR;)+L9_$DEOH$o3xcLrL5O0}^ZW^NZtEJ``oIh0Sqr061l9Pc9b_UZbG z0)OL!r|f<+x66?qy8<CtPuGX_s6Xrf&wKBFVQ)g`Ir!qiQOn;j6Zb=aP|=?H@Gtuh z@biGS18k8;?k$|Seqho30Qdn;WBDZeQcR|aW-jCZtkVEe7zI=h?ymC(M8Dc4V3a+0 rrvMl*zEj6_4m63s*ZrUWo4a6r^;b$HpV3eM0I!S27Di9cVMzZ4o7S8A literal 0 HcmV?d00001 diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000000..391e7d67c5 --- /dev/null +++ b/pom.xml @@ -0,0 +1,911 @@ +<?xml version="1.0" encoding="UTF-8"?> + +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + + +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <groupId>org.apache.comet</groupId> + <artifactId>comet-parent-spark${spark.version.short}_${scala.binary.version}</artifactId> + <version>0.1.0-SNAPSHOT</version> + <packaging>pom</packaging> + <name>Comet Project Parent POM</name> + + <modules> + <module>common</module> + <module>spark</module> + <module>spark-integration</module> + </modules> + + <properties> + <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding> + <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding> + <java.version>11</java.version> + <maven.compiler.source>11</maven.compiler.source> + <maven.compiler.target>11</maven.compiler.target> + <scala.version>2.12.17</scala.version> + <scala.binary.version>2.12</scala.binary.version> + <scala.plugin.version>4.7.2</scala.plugin.version> + <scalatest.version>3.2.9</scalatest.version> + <scalatest-maven-plugin.version>2.0.2</scalatest-maven-plugin.version> + <spark.version>3.4.2</spark.version> + <spark.version.short>3.4</spark.version.short> + <spark.maven.scope>provided</spark.maven.scope> + <protobuf.version>3.17.3</protobuf.version> + <parquet.version>1.13.1</parquet.version> + <parquet.maven.scope>provided</parquet.maven.scope> + <arrow.version>14.0.2</arrow.version> + <codehaus.jackson.version>1.9.13</codehaus.jackson.version> + <spotless.version>2.29.0</spotless.version> + <jni.dir>${project.basedir}/../core/target/debug</jni.dir> + <platform>darwin</platform> + <arch>x86_64</arch> + <comet.shade.packageName>org.apache.comet.shaded</comet.shade.packageName> + <!-- Used by some tests inherited from Spark to get project root directory --> + <spark.test.home>${session.executionRootDirectory}</spark.test.home> + + <!-- Reverse default (skip installation), and then enable only for child modules --> + <maven.deploy.skip>true</maven.deploy.skip> + </properties> + + <dependencyManagement> + <dependencies> + <!-- Spark dependencies --> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-sql_${scala.binary.version}</artifactId> + <version>${spark.version}</version> + <scope>${spark.maven.scope}</scope> + <exclusions> + <exclusion> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-hadoop</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-column</artifactId> + </exclusion> + <exclusion> + <groupId>com.google.guava</groupId> + <artifactId>guava</artifactId> + </exclusion> + + <!-- We're using "org.slf4j:jcl-over-slf4j" --> + <exclusion> + <groupId>commons-logging</groupId> + <artifactId>commons-logging</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-catalyst_${scala.binary.version}</artifactId> + <version>${spark.version}</version> + <scope>${spark.maven.scope}</scope> + <exclusions> + + <!-- We're using "org.slf4j:jcl-over-slf4j" --> + <exclusion> + <groupId>commons-logging</groupId> + <artifactId>commons-logging</artifactId> + </exclusion> + + <!-- Comet uses arrow-memory-unsafe --> + <exclusion> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-memory-netty</artifactId> + </exclusion> + </exclusions> + </dependency> + + <!-- Arrow dependencies --> + <dependency> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-vector</artifactId> + <version>${arrow.version}</version> + <!-- Exclude the following in favor of those from Spark --> + <exclusions> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-annotations</artifactId> + </exclusion> + <exclusion> + <groupId>com.fasterxml.jackson.core</groupId> + <artifactId>jackson-core</artifactId> + </exclusion> + <exclusion> + <groupId>io.netty</groupId> + <artifactId>netty-common</artifactId> + </exclusion> + <exclusion> + <groupId>com.google.code.findbugs</groupId> + <artifactId>jsr305</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-memory-unsafe</artifactId> + <version>${arrow.version}</version> + </dependency> + <dependency> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-c-data</artifactId> + <version>${arrow.version}</version> + </dependency> + + <!-- Parquet dependencies --> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-column</artifactId> + <version>${parquet.version}</version> + <scope>${parquet.maven.scope}</scope> + </dependency> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-hadoop</artifactId> + <version>${parquet.version}</version> + <scope>${parquet.maven.scope}</scope> + <exclusions> + <!-- Exclude the following in favor of jakarta.annotation:jakarta.annotation-api --> + <exclusion> + <groupId>javax.annotation</groupId> + <artifactId>javax.annotation-api</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-avro</artifactId> + <version>${parquet.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-hadoop</artifactId> + <version>${parquet.version}</version> + <classifier>tests</classifier> + <scope>test</scope> + <exclusions> + <!-- Exclude the following in favor of jakarta.annotation:jakarta.annotation-api --> + <exclusion> + <groupId>javax.annotation</groupId> + <artifactId>javax.annotation-api</artifactId> + </exclusion> + </exclusions> + </dependency> + + <!-- Others --> + <dependency> + <groupId>org.scala-lang</groupId> + <artifactId>scala-library</artifactId> + <version>${scala.version}</version> + </dependency> + <dependency> + <groupId>com.google.protobuf</groupId> + <artifactId>protobuf-java</artifactId> + <version>${protobuf.version}</version> + </dependency> + + <!-- Shaded deps marked as provided. These are promoted to compile scope + in the modules where we want the shaded classes to appear in the + associated jar. --> + <dependency> + <groupId>com.google.guava</groupId> + <artifactId>guava</artifactId> + <version>14.0.1</version> + </dependency> + <!-- End of shaded deps --> + + <!-- Test dependencies --> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-core_${scala.binary.version}</artifactId> + <version>${spark.version}</version> + <classifier>tests</classifier> + <scope>test</scope> + <exclusions> + <exclusion> + <groupId>org.apache.comet</groupId> + <artifactId>*</artifactId> + </exclusion> + + <!-- We are using arrow-memory-unsafe --> + <exclusion> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-memory-netty</artifactId> + </exclusion> + + <!-- We're using "org.slf4j:jcl-over-slf4j" --> + <exclusion> + <groupId>commons-logging</groupId> + <artifactId>commons-logging</artifactId> + </exclusion> + + <exclusion> + <groupId>com.google.code.findbugs</groupId> + <artifactId>jsr305</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-catalyst_${scala.binary.version}</artifactId> + <version>${spark.version}</version> + <classifier>tests</classifier> + <scope>test</scope> + <exclusions> + + <!-- We are using arrow-memory-unsafe --> + <exclusion> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-memory-netty</artifactId> + </exclusion> + + <!-- We're using "org.slf4j:jcl-over-slf4j" --> + <exclusion> + <groupId>commons-logging</groupId> + <artifactId>commons-logging</artifactId> + </exclusion> + + <exclusion> + <groupId>com.google.code.findbugs</groupId> + <artifactId>jsr305</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-sql_${scala.binary.version}</artifactId> + <version>${spark.version}</version> + <classifier>tests</classifier> + <scope>test</scope> + <exclusions> + <exclusion> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-hadoop</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-column</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.comet</groupId> + <artifactId>*</artifactId> + </exclusion> + + <!-- We're using "org.slf4j:jcl-over-slf4j" --> + <exclusion> + <groupId>commons-logging</groupId> + <artifactId>commons-logging</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-hive_${scala.binary.version}</artifactId> + <version>${spark.version}</version> + <scope>test</scope> + <exclusions> + <exclusion> + <groupId>org.apache.comet</groupId> + <artifactId>*</artifactId> + </exclusion> + <exclusion> + <groupId>commons-logging</groupId> + <artifactId>commons-logging</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>junit</groupId> + <artifactId>junit</artifactId> + <version>4.13.2</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + <version>3.23.1</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scalatest</groupId> + <artifactId>scalatest_${scala.binary.version}</artifactId> + <version>${scalatest.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scala-lang</groupId> + <artifactId>scala-reflect</artifactId> + <version>${scala.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scalatestplus</groupId> + <artifactId>junit-4-13_${scala.binary.version}</artifactId> + <version>3.2.14.0</version> + <scope>test</scope> + </dependency> + + <!-- For benchmarks to access S3 --> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-hadoop-cloud_${scala.binary.version}</artifactId> + <version>${spark.version}</version> + <classifier>tests</classifier> + <scope>test</scope> + <exclusions> + <!-- We're using hadoop-client --> + <exclusion> + <groupId>org.apache.hadoop.thirdparty</groupId> + <artifactId>hadoop-shaded-guava</artifactId> + </exclusion> + <exclusion> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-annotations</artifactId> + </exclusion> + <exclusion> + <groupId>javax.xml.bind</groupId> + <artifactId>jaxb-api</artifactId> + </exclusion> + + <!-- We're using "org.slf4j:jcl-over-slf4j" --> + <exclusion> + <groupId>commons-logging</groupId> + <artifactId>commons-logging</artifactId> + </exclusion> + </exclusions> + </dependency> + + <dependency> + <groupId>org.codehaus.jackson</groupId> + <artifactId>jackson-mapper-asl</artifactId> + <version>${codehaus.jackson.version}</version> + <scope>test</scope> + </dependency> + + </dependencies> + + </dependencyManagement> + + <profiles> + <profile> + <id>release</id> + <properties> + <jni.dir>${project.basedir}/../core/target/release</jni.dir> + </properties> + </profile> + + <profile> + <id>Win-x86</id> + <activation> + <os> + <family>Windows</family> + <arch>x86</arch> + </os> + </activation> + <properties> + <platform>win32</platform> + <arch>x86_64</arch> + </properties> + </profile> + + <profile> + <id>Darwin-x86</id> + <activation> + <os> + <family>mac</family> + <arch>x86</arch> + </os> + </activation> + <properties> + <platform>darwin</platform> + <arch>x86_64</arch> + </properties> + </profile> + + <profile> + <id>Darwin-aarch64</id> + <activation> + <os> + <family>mac</family> + <arch>aarch64</arch> + </os> + </activation> + <properties> + <platform>darwin</platform> + <arch>aarch64</arch> + </properties> + </profile> + + <profile> + <id>Linux-amd64</id> + <activation> + <os> + <family>Linux</family> + <arch>amd64</arch> + </os> + </activation> + <properties> + <platform>linux</platform> + <arch>amd64</arch> + </properties> + </profile> + + <profile> + <id>Linux-aarch64</id> + <activation> + <os> + <family>Linux</family> + <arch>aarch64</arch> + </os> + </activation> + <properties> + <platform>linux</platform> + <arch>aarch64</arch> + </properties> + </profile> + + <profile> + <id>spark-3.2</id> + <properties> + <scala.version>2.12.15</scala.version> + <spark.version>3.2.2</spark.version> + <spark.version.short>3.2</spark.version.short> + <parquet.version>1.12.0</parquet.version> + <java.version>1.8</java.version> + <maven.compiler.source>${java.version}</maven.compiler.source> + <maven.compiler.target>${java.version}</maven.compiler.target> + </properties> + </profile> + + <profile> + <id>spark-3.3</id> + <properties> + <scala.version>2.12.15</scala.version> + <spark.version>3.3.2</spark.version> + <spark.version.short>3.3</spark.version.short> + <parquet.version>1.12.0</parquet.version> + <java.version>11</java.version> + <maven.compiler.source>${java.version}</maven.compiler.source> + <maven.compiler.target>${java.version}</maven.compiler.target> + </properties> + </profile> + + <profile> + <id>spark-3.4</id> + <properties> + <scala.version>2.12.17</scala.version> + <spark.version.short>3.4</spark.version.short> + <parquet.version>1.13.1</parquet.version> + <java.version>11</java.version> + <maven.compiler.source>${java.version}</maven.compiler.source> + <maven.compiler.target>${java.version}</maven.compiler.target> + </properties> + </profile> + + <profile> + <id>scala-2.13</id> + <properties> + <scala.version>2.13.8</scala.version> + <scala.binary.version>2.13</scala.binary.version> + </properties> + </profile> + + <profile> + <id>semanticdb</id> + <properties> + <scalastyle.skip>true</scalastyle.skip> + <spotless.check.skip>true</spotless.check.skip> + <enforcer.skip>true</enforcer.skip> + </properties> + <build> + <pluginManagement> + <plugins> + <plugin> + <groupId>net.alchim31.maven</groupId> + <artifactId>scala-maven-plugin</artifactId> + <version>${scala.plugin.version}</version> + <executions> + <execution> + <goals> + <goal>compile</goal> + <goal>testCompile</goal> + </goals> + </execution> + </executions> + <configuration> + <args> + <arg>-Ywarn-unused</arg> <!-- if you need exactly RemoveUnused --> + </args> + <compilerPlugins> + <compilerPlugin> + <groupId>org.scalameta</groupId> + <artifactId>semanticdb-scalac_${scala.version}</artifactId> + <version>4.7.5</version> + </compilerPlugin> + </compilerPlugins> + </configuration> + </plugin> + <plugin> + <groupId>io.github.evis</groupId> + <artifactId>scalafix-maven-plugin_${scala.binary.version}</artifactId> + <version>0.1.7_0.10.4</version> + </plugin> + </plugins> + </pluginManagement> + </build> + </profile> + </profiles> + + <build> + <pluginManagement> + <plugins> + <plugin> + <groupId>net.alchim31.maven</groupId> + <artifactId>scala-maven-plugin</artifactId> + <version>${scala.plugin.version}</version> + <executions> + <execution> + <id>eclipse-add-source</id> + <goals> + <goal>add-source</goal> + </goals> + </execution> + <execution> + <id>scala-compile-first</id> + <phase>process-resources</phase> + <goals> + <goal>compile</goal> + <goal>add-source</goal> + </goals> + </execution> + <execution> + <id>scala-test-compile-first</id> + <phase>process-test-resources</phase> + <goals> + <goal>testCompile</goal> + </goals> + </execution> + </executions> + <configuration> + --> + <scalaVersion>${scala.version}</scalaVersion> + <checkMultipleScalaVersions>true</checkMultipleScalaVersions> + <failOnMultipleScalaVersions>true</failOnMultipleScalaVersions> + <recompileMode>incremental</recompileMode> + <args> + <arg>-unchecked</arg> + <arg>-deprecation</arg> + <arg>-feature</arg> + <arg>-explaintypes</arg> + <arg>-Xlint:adapted-args</arg> + </args> + <jvmArgs> + <jvmArg>-Xms1024m</jvmArg> + <jvmArg>-Xmx1024m</jvmArg> + </jvmArgs> + <javacArgs> + <javacArg>-source</javacArg> + <javacArg>${maven.compiler.source}</javacArg> + <javacArg>-target</javacArg> + <javacArg>${maven.compiler.target}</javacArg> + <javacArg>-Xlint:all,-serial,-path,-try</javacArg> + </javacArgs> + </configuration> + </plugin> + <plugin> + <groupId>org.scalatest</groupId> + <artifactId>scalatest-maven-plugin</artifactId> + <version>${scalatest-maven-plugin.version}</version> + <configuration> + <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory> + <junitxml>.</junitxml> + <filereports>SparkTestSuite.txt</filereports> + <stderr/> + <tagsToExclude>org.apache.comet.IntegrationTestSuite</tagsToExclude> + <systemProperties> + <!-- emit test logs to target/unit-tests.log --> + <log4j.configurationFile>file:src/test/resources/log4j2.properties</log4j.configurationFile> + <java.awt.headless>true</java.awt.headless> + <java.io.tmpdir>${project.build.directory}/tmp</java.io.tmpdir> + </systemProperties> + </configuration> + <executions> + <execution> + <id>test</id> + <goals> + <goal>test</goal> + </goals> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <version>3.2.4</version> + <dependencies> + <dependency> + <groupId>org.ow2.asm</groupId> + <artifactId>asm</artifactId> + <version>9.1</version> + </dependency> + <dependency> + <groupId>org.ow2.asm</groupId> + <artifactId>asm-commons</artifactId> + <version>9.1</version> + </dependency> + </dependencies> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-surefire-plugin</artifactId> + <version>3.1.0</version> + <configuration> + <systemProperties> + <log4j.configurationFile>file:src/test/resources/log4j2.properties</log4j.configurationFile> + </systemProperties> + </configuration> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-source-plugin</artifactId> + <version>3.1.0</version> + <configuration> + <attach>true</attach> + </configuration> + <executions> + <execution> + <id>create-source-jar</id> + <goals> + <goal>jar-no-fork</goal> + <goal>test-jar-no-fork</goal> + </goals> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-failsafe-plugin</artifactId> + <version>3.1.0</version> + </plugin> + <plugin> + <groupId>com.diffplug.spotless</groupId> + <artifactId>spotless-maven-plugin</artifactId> + <version>${spotless.version}</version> + <configuration> + <java> + <toggleOffOn /> + <googleJavaFormat /> + <removeUnusedImports /> + <importOrder> + <order>java|javax,scala,org,org.apache,com,org.apache.comet,\#,\#org.apache.comet</order> + </importOrder> + </java> + <scala> + <toggleOffOn /> + <scalafmt> + <version>3.6.1</version> + <file>${maven.multiModuleProjectDirectory}/scalafmt.conf</file> + </scalafmt> + </scala> + </configuration> + </plugin> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>flatten-maven-plugin</artifactId> + <version>1.3.0</version> + </plugin> + </plugins> + </pluginManagement> + <plugins> + <!-- + <plugin> + <groupId>org.scalastyle</groupId> + <artifactId>scalastyle-maven-plugin</artifactId> + <version>1.0.0</version> + <configuration> + <verbose>false</verbose> + <failOnViolation>true</failOnViolation> + <includeTestSourceDirectory>false</includeTestSourceDirectory> + <failOnWarning>false</failOnWarning> + <sourceDirectory>${basedir}/src/main/scala</sourceDirectory> + <testSourceDirectory>${basedir}/src/test/scala</testSourceDirectory> + <configLocation>${maven.multiModuleProjectDirectory}/dev/scalastyle-config.xml</configLocation> + <outputFile>${basedir}/target/scalastyle-output.xml</outputFile> + <inputEncoding>${project.build.sourceEncoding}</inputEncoding> + <outputEncoding>${project.reporting.outputEncoding}</outputEncoding> + </configuration> + <executions> + <execution> + <goals> + <goal>check</goal> + </goals> + <phase>compile</phase> + </execution> + </executions> + </plugin> + <plugin> + <groupId>com.diffplug.spotless</groupId> + <artifactId>spotless-maven-plugin</artifactId> + <executions> + <execution> + <goals> + <goal>check</goal> + </goals> + <phase>compile</phase> + </execution> + </executions> + </plugin> + --> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-source-plugin</artifactId> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-failsafe-plugin</artifactId> + <executions> + <execution> + <goals> + <goal>integration-test</goal> + <goal>verify</goal> + </goals> + <configuration> + <trimStackTrace>false</trimStackTrace> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.rat</groupId> + <artifactId>apache-rat-plugin</artifactId> + <version>0.16</version> + <executions> + <execution> + <phase>verify</phase> + <goals> + <goal>check</goal> + </goals> + </execution> + </executions> + <configuration> + <consoleOutput>true</consoleOutput> + <excludes> + <exclude>**/*.iml</exclude> + <exclude>**/*.log</exclude> + <exclude>**/*.md.vm</exclude> + <exclude>**/.classpath</exclude> + <exclude>**/.project</exclude> + <exclude>**/.settings/**</exclude> + <exclude>**/build/**</exclude> + <exclude>**/target/**</exclude> + <exclude>.git/**</exclude> + <exclude>.gitignore</exclude> + <exclude>.gitmodules</exclude> + <exclude>**/.idea/**</exclude> + <exclude>**/dependency-reduced-pom.xml</exclude> + <exclude>**/testdata/**</exclude> + <exclude>**/.lldbinit</exclude> + <exclude>rust-toolchain</exclude> + <exclude>Makefile</exclude> + <exclude>dev/Dockerfile*</exclude> + <exclude>dev/diff/**</exclude> + <exclude>dev/deploy-file</exclude> + <exclude>**/test/resources/**</exclude> + <exclude>**/benchmarks/*.txt</exclude> + <exclude>**/inspections/*.txt</exclude> + </excludes> + </configuration> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-enforcer-plugin</artifactId> + <version>3.1.0</version> + <executions> + <execution> + <id>no-duplicate-declared-dependencies</id> + <goals> + <goal>enforce</goal> + </goals> + <configuration> + <rules> + <banCircularDependencies/> + <banDuplicatePomDependencyVersions/> + <banDuplicateClasses> + <scopes> + <scope>compile</scope> + <scope>provided</scope> + </scopes> + <ignoreClasses> + <ignoreClass>org.apache.spark.unused.UnusedStubClass</ignoreClass> + </ignoreClasses> + <dependencies> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-sql_${scala.binary.version}</artifactId> + <ignoreClasses> + <!-- the followings conflict between spark-sql and spark-sql:test --> + <ignoreClass>javax.annotation.meta.TypeQualifier</ignoreClass> + <ignoreClass>javax.annotation.Nonnull</ignoreClass> + <ignoreClass>javax.annotation.meta.When</ignoreClass> + <ignoreClass>javax.annotation.Nonnull$Checker</ignoreClass> + <ignoreClass>javax.annotation.meta.TypeQualifierValidator</ignoreClass> + <!-- this class is not properly excluded from comet-spark right now --> + <ignoreClass>org.apache.parquet.filter2.predicate.SparkFilterApi</ignoreClass> + </ignoreClasses> + </dependency> + <dependency> + <groupId>com.google.code.findbugs</groupId> + <artifactId>jsr305</artifactId> + <ignoreClasses> + <!-- the followings conflict between spark-sql and findbugs --> + <ignoreClass>javax.annotation.meta.TypeQualifier</ignoreClass> + <ignoreClass>javax.annotation.Nonnull</ignoreClass> + <ignoreClass>javax.annotation.meta.When</ignoreClass> + <ignoreClass>javax.annotation.Nonnull$Checker</ignoreClass> + <ignoreClass>javax.annotation.meta.TypeQualifierValidator</ignoreClass> + <ignoreClass>javax.annotation.Nullable</ignoreClass> + <ignoreClass>javax.annotation.meta.TypeQualifierNickname</ignoreClass> + </ignoreClasses> + </dependency> + </dependencies> + <findAllDuplicates>true</findAllDuplicates> + <ignoreWhenIdentical>true</ignoreWhenIdentical> + </banDuplicateClasses> + </rules> + </configuration> + </execution> + </executions> + <dependencies> + <dependency> + <groupId>org.codehaus.mojo</groupId> + <artifactId>extra-enforcer-rules</artifactId> + <version>1.6.1</version> + </dependency> + </dependencies> + </plugin> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>flatten-maven-plugin</artifactId> + <executions> + <!-- enable flattening --> + <execution> + <id>flatten</id> + <phase>process-resources</phase> + <goals> + <goal>flatten</goal> + </goals> + </execution> + <!-- ensure proper cleanup --> + <execution> + <id>flatten.clean</id> + <phase>clean</phase> + <goals> + <goal>clean</goal> + </goals> + </execution> + </executions> + </plugin> + </plugins> + </build> +</project> diff --git a/rust-toolchain b/rust-toolchain new file mode 100644 index 0000000000..55d7013e42 --- /dev/null +++ b/rust-toolchain @@ -0,0 +1 @@ +nightly-2023-09-05 diff --git a/scalafmt.conf b/scalafmt.conf new file mode 100644 index 0000000000..7c8c2b3818 --- /dev/null +++ b/scalafmt.conf @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +align = none +align.openParenDefnSite = false +align.openParenCallSite = false +align.tokens = [] +importSelectors = "singleLine" +optIn = { + configStyleArguments = false +} +danglingParentheses.preset = false +docstrings.style = Asterisk +maxColumn = 98 +runner.dialect = scala212 +version = 3.6.1 + +rewrite.rules = [Imports] +rewrite.imports.sort = scalastyle +rewrite.imports.groups = [ + ["java\\..*", "javax\\..*"], + ["scala\\..*"], + ["org\\..*"], + ["org\\.apache\\..*"], + ["org\\.apache\\.comet\\..*"], + ["com\\..*"], +] diff --git a/spark-integration/pom.xml b/spark-integration/pom.xml new file mode 100644 index 0000000000..6af59ac694 --- /dev/null +++ b/spark-integration/pom.xml @@ -0,0 +1,106 @@ +<?xml version="1.0" encoding="UTF-8"?> + +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + + +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.apache.comet</groupId> + <artifactId>comet-parent-spark${spark.version.short}_${scala.binary.version}</artifactId> + <version>0.1.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <artifactId>comet-spark-integration-spark${spark.version.short}_${scala.binary.version}</artifactId> + <name>comet-spark-integration</name> + <packaging>pom</packaging> + + <properties> + <!-- Reverse default (skip installation), and then enable only for child modules --> + <maven.deploy.skip>false</maven.deploy.skip> + </properties> + + <dependencies> + <dependency> + <groupId>org.apache.comet</groupId> + <artifactId>comet-spark-spark${spark.version.short}_${scala.binary.version}</artifactId> + <version>${project.version}</version> + <exclusions> + <!-- This is shaded into the jar --> + <exclusion> + <groupId>org.apache.comet</groupId> + <artifactId>comet-common-spark${spark.version.short}_${scala.binary.version}</artifactId> + </exclusion> + </exclusions> + </dependency> + </dependencies> + + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-dependency-plugin</artifactId> + <executions> + <!-- create a maven pom property that has all of our dependencies. + below in the integration-test phase we'll pass this list + of paths to our jar checker script. + --> + <execution> + <id>put-client-artifacts-in-a-property</id> + <phase>pre-integration-test</phase> + <goals> + <goal>build-classpath</goal> + </goals> + <configuration> + <excludeTransitive>true</excludeTransitive> + <pathSeparator>;</pathSeparator> + <outputProperty>comet-artifacts</outputProperty> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>exec-maven-plugin</artifactId> + <executions> + <execution> + <id>check-jar-contents</id> + <phase>integration-test</phase> + <goals> + <goal>exec</goal> + </goals> + <configuration> + <executable>bash</executable> + <workingDirectory>${project.build.testOutputDirectory}</workingDirectory> + <requiresOnline>false</requiresOnline> + <arguments> + <argument>${project.basedir}/../dev/ensure-jars-have-correct-contents.sh</argument> + <argument>${comet-artifacts}</argument> + </arguments> + </configuration> + </execution> + </executions> + </plugin> + </plugins> + </build> + +</project> diff --git a/spark/README.md b/spark/README.md new file mode 100644 index 0000000000..6f45d9212a --- /dev/null +++ b/spark/README.md @@ -0,0 +1,22 @@ +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +# Apache Spark Plugin for Comet + +This folder implements a plugin for Apache Spark to use Comet native library, using Spark's extension framework. diff --git a/spark/pom.xml b/spark/pom.xml new file mode 100644 index 0000000000..d3afa48eca --- /dev/null +++ b/spark/pom.xml @@ -0,0 +1,238 @@ +<?xml version="1.0" encoding="UTF-8"?> + +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + + +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.apache.comet</groupId> + <artifactId>comet-parent-spark${spark.version.short}_${scala.binary.version}</artifactId> + <version>0.1.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <artifactId>comet-spark-spark${spark.version.short}_${scala.binary.version}</artifactId> + <name>comet-spark</name> + + <properties> + <!-- Reverse default (skip installation), and then enable only for child modules --> + <maven.deploy.skip>false</maven.deploy.skip> + </properties> + + <dependencies> + <dependency> + <groupId>org.apache.comet</groupId> + <artifactId>comet-common-spark${spark.version.short}_${scala.binary.version}</artifactId> + <version>${project.version}</version> + <exclusions> + <exclusion> + <groupId>org.apache.arrow</groupId> + <artifactId>*</artifactId> + </exclusion> + </exclusions> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-sql_${scala.binary.version}</artifactId> + </dependency> + <dependency> + <groupId>org.scala-lang</groupId> + <artifactId>scala-library</artifactId> + </dependency> + <dependency> + <groupId>com.google.protobuf</groupId> + <artifactId>protobuf-java</artifactId> + </dependency> + <dependency> + <groupId>org.scalatest</groupId> + <artifactId>scalatest_${scala.binary.version}</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scalatestplus</groupId> + <artifactId>junit-4-13_${scala.binary.version}</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-core_${scala.binary.version}</artifactId> + <classifier>tests</classifier> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-catalyst_${scala.binary.version}</artifactId> + <classifier>tests</classifier> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-sql_${scala.binary.version}</artifactId> + <classifier>tests</classifier> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-hadoop-cloud_${scala.binary.version}</artifactId> + <classifier>tests</classifier> + </dependency> + <dependency> + <groupId>junit</groupId> + <artifactId>junit</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>com.google.guava</groupId> + <artifactId>guava</artifactId> + </dependency> + <dependency> + <groupId>org.codehaus.jackson</groupId> + <artifactId>jackson-mapper-asl</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-hadoop</artifactId> + </dependency> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-hadoop</artifactId> + <classifier>tests</classifier> + <!-- Note we don't use test scope for this artifact. This is because it's only needed + to provide InMemoryKMS class that is shaded below, to make Spark test happy. --> + </dependency> + <!-- We shade & relocate Arrow dependencies in comet-common, so comet-spark module no longer + depends on Arrow. However, when running `mvn test` we still need Arrow classes in the + classpath, since the Maven shading happens in 'package' phase which is after 'test' --> + <dependency> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-memory-unsafe</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.apache.arrow</groupId> + <artifactId>arrow-c-data</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + + <build> + <plugins> + <plugin> + <groupId>com.github.os72</groupId> + <artifactId>protoc-jar-maven-plugin</artifactId> + <version>3.11.4</version> + <executions> + <execution> + <phase>generate-sources</phase> + <goals> + <goal>run</goal> + </goals> + <configuration> + <protocArtifact>com.google.protobuf:protoc:${protobuf.version}</protocArtifact> + <inputDirectories> + <include>../core/src/execution/proto</include> + </inputDirectories> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.scalatest</groupId> + <artifactId>scalatest-maven-plugin</artifactId> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <executions> + <execution> + <phase>package</phase> + <goals> + <goal>shade</goal> + </goals> + <configuration> + <createSourcesJar>true</createSourcesJar> + <shadeSourcesContent>true</shadeSourcesContent> + <shadedArtifactAttached>false</shadedArtifactAttached> + <createDependencyReducedPom>true</createDependencyReducedPom> + <artifactSet> + <includes> + <include>org.apache.comet:comet-common-spark${spark.version.short}_${scala.binary.version}</include> + <!-- Relocate Protobuf since Spark uses 2.5.0 while Comet uses 3.x --> + <include>com.google.protobuf:protobuf-java</include> + <include>com.google.guava:guava</include> + </includes> + </artifactSet> + <filters> + <filter> + <artifact>*:*</artifact> + <excludes> + <exclude>**/*.proto</exclude> + <exclude>**/*.thrift</exclude> + <exclude>git.properties</exclude> + <exclude>log4j.properties</exclude> + <exclude>log4j2.properties</exclude> + <exclude>**/SparkFilterApi.*</exclude> + </excludes> + </filter> + <filter> + <artifact>org.apache.parquet:parquet-hadoop:tests</artifact> + <includes> + <!-- Used by Spark test `ParquetEncryptionSuite` --> + <include>org/apache/parquet/crypto/keytools/mocks/InMemoryKMS*</include> + </includes> + </filter> + </filters> + <relocations> + <relocation> + <pattern>com.google.protobuf</pattern> + <shadedPattern>${comet.shade.packageName}.protobuf</shadedPattern> + </relocation> + <relocation> + <pattern>com.google.common</pattern> + <shadedPattern>${comet.shade.packageName}.guava</shadedPattern> + </relocation> + </relocations> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-failsafe-plugin</artifactId> + <executions> + <execution> + <goals> + <goal>integration-test</goal> + <goal>verify</goal> + </goals> + <configuration> + <trimStackTrace>false</trimStackTrace> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>net.alchim31.maven</groupId> + <artifactId>scala-maven-plugin</artifactId> + </plugin> + </plugins> + </build> + +</project> diff --git a/spark/src/main/java/org/apache/parquet/filter2/predicate/SparkFilterApi.java b/spark/src/main/java/org/apache/parquet/filter2/predicate/SparkFilterApi.java new file mode 100644 index 0000000000..55e9cf5c6b --- /dev/null +++ b/spark/src/main/java/org/apache/parquet/filter2/predicate/SparkFilterApi.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.parquet.filter2.predicate; + +import org.apache.parquet.filter2.predicate.Operators.*; +import org.apache.parquet.hadoop.metadata.ColumnPath; + +/** + * Copied from Spark 3.2, in order to fix Parquet shading issue. + * + * <p>TODO: find a way to remove this duplication + */ +public final class SparkFilterApi { + public static IntColumn intColumn(String[] path) { + return new IntColumn(ColumnPath.get(path)); + } + + public static LongColumn longColumn(String[] path) { + return new LongColumn(ColumnPath.get(path)); + } + + public static FloatColumn floatColumn(String[] path) { + return new FloatColumn(ColumnPath.get(path)); + } + + public static DoubleColumn doubleColumn(String[] path) { + return new DoubleColumn(ColumnPath.get(path)); + } + + public static BooleanColumn booleanColumn(String[] path) { + return new BooleanColumn(ColumnPath.get(path)); + } + + public static BinaryColumn binaryColumn(String[] path) { + return new BinaryColumn(ColumnPath.get(path)); + } +} diff --git a/spark/src/main/java/org/apache/spark/sql/comet/CometScalarSubquery.java b/spark/src/main/java/org/apache/spark/sql/comet/CometScalarSubquery.java new file mode 100644 index 0000000000..98dc025422 --- /dev/null +++ b/spark/src/main/java/org/apache/spark/sql/comet/CometScalarSubquery.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet; + +import java.util.HashMap; + +import org.apache.spark.sql.execution.ScalarSubquery; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; + +import org.apache.comet.CometRuntimeException; + +/** A helper class to execute scalar subqueries and retrieve subquery results from native code. */ +public class CometScalarSubquery { + /** + * A map from (planId, subqueryId) to the corresponding ScalarSubquery. We cannot simply use + * `subqueryId` because same query plan may be executed multiple times in same executor (i.e., JVM + * instance). For such cases, if we delete the ScalarSubquery from the map after the first + * execution, the second execution will fail to find the ScalarSubquery if the native code is + * still running. + */ + private static final HashMap<Long, HashMap<Long, ScalarSubquery>> subqueryMap = new HashMap<>(); + + public static synchronized void setSubquery(long planId, ScalarSubquery subquery) { + if (!subqueryMap.containsKey(planId)) { + subqueryMap.put(planId, new HashMap<>()); + } + + subqueryMap.get(planId).put(subquery.exprId().id(), subquery); + } + + public static synchronized void removeSubquery(long planId, ScalarSubquery subquery) { + subqueryMap.get(planId).remove(subquery.exprId().id()); + + if (subqueryMap.get(planId).isEmpty()) { + subqueryMap.remove(planId); + } + } + + /** Retrieve the result of subquery. */ + private static Object getSubquery(Long planId, Long id) { + if (!subqueryMap.containsKey(planId)) { + throw new CometRuntimeException("Subquery " + id + " not found for plan " + planId + "."); + } + + return subqueryMap.get(planId).get(id).eval(null); + } + + /** Check if the result of a subquery is null. Called from native code. */ + public static boolean isNull(long planId, long id) { + return getSubquery(planId, id) == null; + } + + /** Get the result of a subquery as a boolean. Called from native code. */ + public static boolean getBoolean(long planId, long id) { + return (boolean) getSubquery(planId, id); + } + + /** Get the result of a subquery as a byte. Called from native code. */ + public static byte getByte(long planId, long id) { + return (byte) getSubquery(planId, id); + } + + /** Get the result of a subquery as a short. Called from native code. */ + public static short getShort(long planId, long id) { + return (short) getSubquery(planId, id); + } + + /** Get the result of a subquery as an integer. Called from native code. */ + public static int getInt(long planId, long id) { + return (int) getSubquery(planId, id); + } + + /** Get the result of a subquery as a long. Called from native code. */ + public static long getLong(long planId, long id) { + return (long) getSubquery(planId, id); + } + + /** Get the result of a subquery as a float. Called from native code. */ + public static float getFloat(long planId, long id) { + return (float) getSubquery(planId, id); + } + + /** Get the result of a subquery as a double. Called from native code. */ + public static double getDouble(long planId, long id) { + return (double) getSubquery(planId, id); + } + + /** Get the result of a subquery as a decimal represented as bytes. Called from native code. */ + public static byte[] getDecimal(long planId, long id) { + return ((Decimal) getSubquery(planId, id)).toJavaBigDecimal().unscaledValue().toByteArray(); + } + + /** Get the result of a subquery as a string. Called from native code. */ + public static String getString(long planId, long id) { + return ((UTF8String) getSubquery(planId, id)).toString(); + } + + /** Get the result of a subquery as a binary. Called from native code. */ + public static byte[] getBinary(long planId, long id) { + return (byte[]) getSubquery(planId, id); + } +} diff --git a/spark/src/main/resources/log4j2.properties b/spark/src/main/resources/log4j2.properties new file mode 100644 index 0000000000..04cdf85330 --- /dev/null +++ b/spark/src/main/resources/log4j2.properties @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn + diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala new file mode 100644 index 0000000000..029be29d87 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.util.HashMap + +import org.apache.spark._ +import org.apache.spark.sql.comet.CometMetricNode +import org.apache.spark.sql.vectorized._ + +import org.apache.comet.vector.NativeUtil + +/** + * An iterator class used to execute Comet native query. It takes an input iterator which comes + * from Comet Scan and is expected to produce batches of Arrow Arrays. During consuming this + * iterator, it will consume input iterator and pass Arrow Arrays to Comet native engine by + * addresses. Even after the end of input iterator, this iterator still possibly continues + * executing native query as there might be blocking operators such as Sort, Aggregate. The API + * `hasNext` can be used to check if it is the end of this iterator (i.e. the native query is + * done). + * + * @param inputs + * The input iterators producing sequence of batches of Arrow Arrays. + * @param protobufQueryPlan + * The serialized bytes of Spark execution plan. + */ +class CometExecIterator( + val id: Long, + inputs: Seq[Iterator[ColumnarBatch]], + protobufQueryPlan: Array[Byte], + configs: HashMap[String, String], + nativeMetrics: CometMetricNode) + extends Iterator[ColumnarBatch] { + + private val nativeLib = new Native() + private val plan = nativeLib.createPlan(id, configs, protobufQueryPlan, nativeMetrics) + private val nativeUtil = new NativeUtil + private var nextBatch: Option[ColumnarBatch] = None + private var currentBatch: ColumnarBatch = null + private var closed: Boolean = false + + private def peekNext(): ExecutionState = { + val result = nativeLib.peekNext(plan) + val flag = result(0) + + if (flag == 0) Pending + else if (flag == 1) { + val numRows = result(1) + val addresses = result.slice(2, result.length) + Batch(numRows = numRows.toInt, addresses = addresses) + } else { + throw new IllegalStateException(s"Invalid native flag: $flag") + } + } + + private def executeNative( + input: Array[Array[Long]], + finishes: Array[Boolean], + numRows: Int): ExecutionState = { + val result = nativeLib.executePlan(plan, input, finishes, numRows) + val flag = result(0) + if (flag == -1) EOF + else if (flag == 0) Pending + else if (flag == 1) { + val numRows = result(1) + val addresses = result.slice(2, result.length) + Batch(numRows = numRows.toInt, addresses = addresses) + } else { + throw new IllegalStateException(s"Invalid native flag: $flag") + } + } + + /** Execution result from Comet native */ + trait ExecutionState + + /** A new batch is available */ + case class Batch(numRows: Int, addresses: Array[Long]) extends ExecutionState + + /** The execution is finished - no more batch */ + case object EOF extends ExecutionState + + /** The execution is pending (e.g., blocking operator is still consuming batches) */ + case object Pending extends ExecutionState + + private def peek(): Option[ColumnarBatch] = { + peekNext() match { + case Batch(numRows, addresses) => + val cometVectors = nativeUtil.importVector(addresses) + Some(new ColumnarBatch(cometVectors.toArray, numRows)) + case _ => + None + } + } + + def getNextBatch( + inputArrays: Array[Array[Long]], + finishes: Array[Boolean], + numRows: Int): Option[ColumnarBatch] = { + executeNative(inputArrays, finishes, numRows) match { + case EOF => None + case Batch(numRows, addresses) => + val cometVectors = nativeUtil.importVector(addresses) + Some(new ColumnarBatch(cometVectors.toArray, numRows)) + case Pending => + if (finishes.forall(_ == true)) { + // Once no input, we should not get a pending flag. + throw new SparkException( + "Native execution should not be pending after reaching end of input batches") + } + // For pending, we keep reading next input. + None + } + } + + override def hasNext: Boolean = { + if (closed) return false + + if (nextBatch.isDefined) { + return true + } + // Before we pull next input batch, check if there is next output batch available + // from native side. Some operators might still have output batches ready produced + // from last input batch. For example, `expand` operator will produce output batches + // based on the input batch. + nextBatch = peek() + + // Next input batches are available, execute native query plan with the inputs until + // we get next output batch ready + while (nextBatch.isEmpty && inputs.exists(_.hasNext)) { + val batches = inputs.map { + case input if input.hasNext => Some(input.next()) + case _ => None + } + + var numRows = -1 + val (batchAddresses, finishes) = batches + .map { + case Some(batch) => + numRows = batch.numRows() + (nativeUtil.exportBatch(batch), false) + case None => (Array.empty[Long], true) + } + .toArray + .unzip + + // At least one input batch should be consumed + assert(numRows != -1, "No input batch has been consumed") + + nextBatch = getNextBatch(batchAddresses, finishes, numRows) + } + + // After we consume to the end of the iterators, the native side still can output batches + // back because there might be blocking operators e.g. Sort. We continue ask for batches + // until it returns empty columns. + if (nextBatch.isEmpty) { + val finishes = inputs.map(_ => true).toArray + nextBatch = getNextBatch(inputs.map(_ => Array.empty[Long]).toArray, finishes, 0) + val hasNext = nextBatch.isDefined + if (!hasNext) { + close() + } + hasNext + } else { + true + } + } + + override def next(): ColumnarBatch = { + if (currentBatch != null) { + // Eagerly release Arrow Arrays in the previous batch + currentBatch.close() + currentBatch = null + } + + if (nextBatch.isEmpty && !hasNext) { + throw new NoSuchElementException("No more element") + } + + currentBatch = nextBatch.get + nextBatch = None + currentBatch + } + + def close(): Unit = synchronized { + if (!closed) { + if (currentBatch != null) { + currentBatch.close() + currentBatch = null + } + nativeLib.releasePlan(plan) + // The allocator thoughts the exported ArrowArray and ArrowSchema structs are not released, + // so it will report: + // Caused by: java.lang.IllegalStateException: Memory was leaked by query. + // Memory leaked: (516) Allocator(ROOT) 0/516/808/9223372036854775807 (res/actual/peak/limit) + // Suspect this seems a false positive leak, because there is no reported memory leak at JVM + // when profiling. `allocator` reports a leak because it calculates the accumulated number + // of memory allocated for ArrowArray and ArrowSchema. But these exported ones will be + // released in native side later. + // More to clarify it. For ArrowArray and ArrowSchema, Arrow will put a release field into the + // memory region which is a callback function pointer (C function) that could be called to + // release these structs in native code too. Once we wrap their memory addresses at native + // side using FFI ArrowArray and ArrowSchema, and drop them later, the callback function will + // be called to release the memory. + // But at JVM, the allocator doesn't know about this fact so it still keeps the accumulated + // number. + // Tried to manually do `release` and `close` that can make the allocator happy, but it will + // cause JVM runtime failure. + + // allocator.close() + closed = true + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala new file mode 100644 index 0000000000..1c0f7bbf70 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -0,0 +1,472 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.nio.ByteOrder + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.SparkSessionExtensions +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.comet._ +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import org.apache.comet.CometConf._ +import org.apache.comet.CometSparkSessionExtensions.{isANSIEnabled, isCometEnabled, isCometExecEnabled, isCometOperatorEnabled, isCometScan, isCometScanEnabled, isSchemaSupported} +import org.apache.comet.parquet.{CometParquetScan, SupportsComet} +import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.QueryPlanSerde +import org.apache.comet.shims.ShimCometSparkSessionExtensions + +class CometSparkSessionExtensions + extends (SparkSessionExtensions => Unit) + with Logging + with ShimCometSparkSessionExtensions { + override def apply(extensions: SparkSessionExtensions): Unit = { + extensions.injectColumnar { session => CometScanColumnar(session) } + extensions.injectColumnar { session => CometExecColumnar(session) } + extensions.injectQueryStagePrepRule { session => CometScanRule(session) } + extensions.injectQueryStagePrepRule { session => CometExecRule(session) } + } + + case class CometScanColumnar(session: SparkSession) extends ColumnarRule { + override def preColumnarTransitions: Rule[SparkPlan] = CometScanRule(session) + } + + case class CometExecColumnar(session: SparkSession) extends ColumnarRule { + override def preColumnarTransitions: Rule[SparkPlan] = CometExecRule(session) + } + + case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + if (!isCometEnabled(conf) || !isCometScanEnabled(conf)) plan + else { + plan.transform { + // data source V2 + case scanExec: BatchScanExec + if scanExec.scan.isInstanceOf[ParquetScan] && + isSchemaSupported(scanExec.scan.asInstanceOf[ParquetScan].readDataSchema) && + isSchemaSupported(scanExec.scan.asInstanceOf[ParquetScan].readPartitionSchema) && + // Comet does not support pushedAggregate + getPushedAggregate(scanExec.scan.asInstanceOf[ParquetScan]).isEmpty => + val cometScan = CometParquetScan(scanExec.scan.asInstanceOf[ParquetScan]) + logInfo("Comet extension enabled for Scan") + CometBatchScanExec( + scanExec.copy(scan = cometScan), + runtimeFilters = scanExec.runtimeFilters) + + // iceberg scan + case scanExec: BatchScanExec => + if (isSchemaSupported(scanExec.scan.readSchema())) { + scanExec.scan match { + case s: SupportsComet if s.isCometEnabled => + logInfo(s"Comet extension enabled for ${scanExec.scan.getClass.getSimpleName}") + // When reading from Iceberg, we automatically enable type promotion + SQLConf.get.setConfString(COMET_SCHEMA_EVOLUTION_ENABLED.key, "true") + CometBatchScanExec( + scanExec.clone().asInstanceOf[BatchScanExec], + runtimeFilters = scanExec.runtimeFilters) + case _ => + logInfo( + "Comet extension is not enabled for " + + s"${scanExec.scan.getClass.getSimpleName}: not enabled on data source side") + scanExec + } + } else { + logInfo( + "Comet extension is not enabled for " + + s"${scanExec.scan.getClass.getSimpleName}: Schema not supported") + scanExec + } + + // data source V1 + case scanExec @ FileSourceScanExec( + HadoopFsRelation(_, partitionSchema, _, _, _: ParquetFileFormat, _), + _: Seq[AttributeReference], + requiredSchema, + _, + _, + _, + _, + _, + _) if isSchemaSupported(requiredSchema) && isSchemaSupported(partitionSchema) => + logInfo("Comet extension enabled for v1 Scan") + CometScanExec(scanExec, session) + } + } + } + } + + case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { + private def isCometNative(op: SparkPlan): Boolean = op.isInstanceOf[CometNativeExec] + + // spotless:off + /** + * Tries to transform a Spark physical plan into a Comet plan. + * + * This rule traverses bottom-up from the original Spark plan and for each plan node, there + * are a few cases to consider: + * + * 1. The child(ren) of the current node `p` cannot be converted to native + * In this case, we'll simply return the original Spark plan, since Comet native + * execution cannot start from an arbitrary Spark operator (unless it is special node + * such as scan or sink such as union etc., which are wrapped by + * `CometScanWrapper` and `CometSinkPlaceHolder` respectively). + * + * 2. The child(ren) of the current node `p` can be converted to native + * There are two sub-cases for this scenario: 1) This node `p` can also be converted to + * native. In this case, we'll create a new native Comet operator for `p` and connect it with + * its previously converted child(ren); 2) This node `p` cannot be converted to native. In + * this case, similar to 1) above, we simply return `p` as it is. Its child(ren) would still + * be native Comet operators. + * + * After this rule finishes, we'll do another pass on the final plan to convert all adjacent + * Comet native operators into a single native execution block. Please see where + * `convertBlock` is called below. + * + * Here are a few examples: + * + * Scan ======> CometScan + * | | + * Filter CometFilter + * | | + * HashAggregate CometHashAggregate + * | | + * Exchange CometExchange + * | | + * HashAggregate CometHashAggregate + * | | + * UnsupportedOperator UnsupportedOperator + * + * Native execution doesn't necessarily have to start from `CometScan`: + * + * Scan =======> CometScan + * | | + * UnsupportedOperator UnsupportedOperator + * | | + * HashAggregate HashAggregate + * | | + * Exchange CometExchange + * | | + * HashAggregate CometHashAggregate + * | | + * UnsupportedOperator UnsupportedOperator + * + * A sink can also be Comet operators other than `CometExchange`, for instance `CometUnion`: + * + * Scan Scan =======> CometScan CometScan + * | | | | + * Filter Filter CometFilter CometFilter + * | | | | + * Union CometUnion + * | | + * Project CometProject + */ + // spotless:on + private def transform(plan: SparkPlan): SparkPlan = { + def transform1(op: UnaryExecNode): Option[Operator] = { + op.child match { + case childNativeOp: CometNativeExec => + QueryPlanSerde.operator2Proto(op, childNativeOp.nativeOp) + case _ => + None + } + } + + plan.transformUp { + case op if isCometScan(op) => + val nativeOp = QueryPlanSerde.operator2Proto(op).get + CometScanWrapper(nativeOp, op) + + case op: ProjectExec => + val newOp = transform1(op) + newOp match { + case Some(nativeOp) => + CometProjectExec(nativeOp, op, op.projectList, op.output, op.child) + case None => + op + } + + case op: FilterExec => + val newOp = transform1(op) + newOp match { + case Some(nativeOp) => + CometFilterExec(nativeOp, op, op.condition, op.child) + case None => + op + } + + case op: SortExec => + val newOp = transform1(op) + newOp match { + case Some(nativeOp) => + CometSortExec(nativeOp, op, op.sortOrder, op.child) + case None => + op + } + + case op: LocalLimitExec => + val newOp = transform1(op) + newOp match { + case Some(nativeOp) => + CometLocalLimitExec(nativeOp, op, op.limit, op.child) + case None => + op + } + + case op: GlobalLimitExec => + val newOp = transform1(op) + newOp match { + case Some(nativeOp) => + CometGlobalLimitExec(nativeOp, op, op.limit, op.child) + case None => + op + } + + case op: ExpandExec => + val newOp = transform1(op) + newOp match { + case Some(nativeOp) => + CometExpandExec(nativeOp, op, op.projections, op.child) + case None => + op + } + + case op @ HashAggregateExec(_, _, _, groupingExprs, aggExprs, _, _, _, child) => + val newOp = transform1(op) + newOp match { + case Some(nativeOp) => + val modes = aggExprs.map(_.mode).distinct + assert(modes.length == 1) + CometHashAggregateExec( + nativeOp, + op, + groupingExprs, + aggExprs, + child.output, + modes.head, + child) + case None => + op + } + + case c @ CoalesceExec(numPartitions, child) + if isCometOperatorEnabled(conf, "coalesce") + && isCometNative(child) => + QueryPlanSerde.operator2Proto(c) match { + case Some(nativeOp) => + val cometOp = CometCoalesceExec(c, numPartitions, child) + CometSinkPlaceHolder(nativeOp, c, cometOp) + case None => + c + } + + case u: UnionExec + if isCometOperatorEnabled(conf, "union") && + u.children.forall(isCometNative) => + QueryPlanSerde.operator2Proto(u) match { + case Some(nativeOp) => + val cometOp = CometUnionExec(u, u.children) + CometSinkPlaceHolder(nativeOp, u, cometOp) + } + + case op => + // An operator that is not supported by Comet + op + } + } + + override def apply(plan: SparkPlan): SparkPlan = { + // DataFusion doesn't have ANSI mode. For now we just disable CometExec if ANSI mode is + // enabled. + if (isANSIEnabled(conf)) { + logInfo("Comet extension disabled for ANSI mode") + return plan + } + + // We shouldn't transform Spark query plan if Comet is disabled. + if (!isCometEnabled(conf)) return plan + + if (!isCometExecEnabled(conf)) { + // Comet exec is disabled + plan + } else { + var newPlan = transform(plan) + + // Remove placeholders + newPlan = newPlan.transform { + case CometSinkPlaceHolder(_, _, s) => s + case CometScanWrapper(_, s) => s + } + + // Set up logical links + newPlan = newPlan.transform { case op: CometExec => + op.originalPlan.logicalLink.foreach(op.setLogicalLink) + op + } + + // Convert native execution block by linking consecutive native operators. + var firstNativeOp = true + newPlan.transformDown { + case op: CometNativeExec => + if (firstNativeOp) { + op.convertBlock() + firstNativeOp = false + } + op + case op => + firstNativeOp = true + op + } + } + } + } +} + +object CometSparkSessionExtensions extends Logging { + lazy val isBigEndian: Boolean = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN) + + private[comet] def isANSIEnabled(conf: SQLConf): Boolean = { + conf.getConf(SQLConf.ANSI_ENABLED) + } + + /** + * Checks whether Comet extension should be enabled for Spark. + */ + private[comet] def isCometEnabled(conf: SQLConf): Boolean = { + if (isBigEndian) { + logInfo("Comet extension is disabled because platform is big-endian") + return false + } + if (!COMET_ENABLED.get(conf)) { + logInfo(s"Comet extension is disabled, please turn on s${COMET_ENABLED.key} to enable it") + return false + } + + // We don't support INT96 timestamps written by Apache Impala in a different timezone yet + if (conf.getConf(SQLConf.PARQUET_INT96_TIMESTAMP_CONVERSION)) { + logWarning( + "Comet extension is disabled, because it currently doesn't support" + + s" ${SQLConf.PARQUET_INT96_TIMESTAMP_CONVERSION} setting to true.") + return false + } + + try { + // This will load the Comet native lib on demand, and if success, should set + // `NativeBase.loaded` to true + NativeBase.isLoaded + } catch { + case e: Throwable => + if (COMET_NATIVE_LOAD_REQUIRED.get(conf)) { + throw new CometRuntimeException( + "Error when loading native library. Please fix the error and try again, or fallback " + + s"to Spark by setting ${COMET_ENABLED.key} to false", + e) + } else { + logWarning( + "Comet extension is disabled because of error when loading native lib. " + + "Falling back to Spark", + e) + } + false + } + } + + private[comet] def isCometOperatorEnabled(conf: SQLConf, operator: String): Boolean = { + val operatorFlag = s"$COMET_EXEC_CONFIG_PREFIX.$operator.enabled" + conf.getConfString(operatorFlag, "false").toBoolean || isCometAllOperatorEnabled(conf) + } + + private[comet] def isCometScanEnabled(conf: SQLConf): Boolean = { + COMET_SCAN_ENABLED.get(conf) + } + + private[comet] def isCometExecEnabled(conf: SQLConf): Boolean = { + COMET_EXEC_ENABLED.get(conf) + } + + private[comet] def isCometAllOperatorEnabled(conf: SQLConf): Boolean = { + COMET_EXEC_ALL_OPERATOR_ENABLED.get(conf) + } + + private[comet] def isCometAllExprEnabled(conf: SQLConf): Boolean = { + COMET_EXEC_ALL_EXPR_ENABLED.get(conf) + } + + private[comet] def isSchemaSupported(schema: StructType): Boolean = + schema.map(_.dataType).forall(isTypeSupported) + + private[comet] def isTypeSupported(dt: DataType): Boolean = dt match { + case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | + BinaryType | StringType | _: DecimalType | DateType | TimestampType => + true + // `TimestampNTZType` is private in Spark 3.2. + case t: DataType if t.typeName == "timestamp_ntz" && !isSpark32 => true + case dt => + logInfo(s"Comet extension is disabled because data type $dt is not supported") + false + } + + def isCometScan(op: SparkPlan): Boolean = { + op.isInstanceOf[CometBatchScanExec] || op.isInstanceOf[CometScanExec] + } + + /** Used for operations that weren't available in Spark 3.2 */ + def isSpark32: Boolean = { + org.apache.spark.SPARK_VERSION.matches("3\\.2\\..*") + } + + /** Used for operations that are available in Spark 3.4+ */ + def isSpark34Plus: Boolean = { + org.apache.spark.SPARK_VERSION >= "3.4" + } + + /** Calculates required memory overhead in MB per executor process for Comet. */ + def getCometMemoryOverheadInMiB(sparkConf: SparkConf): Long = { + // `spark.executor.memory` default value is 1g + val executorMemoryMiB = ConfigHelpers + .byteFromString(sparkConf.get("spark.executor.memory", "1024MB"), ByteUnit.MiB) + + val minimum = ConfigHelpers + .byteFromString(sparkConf.get(COMET_MEMORY_OVERHEAD_MIN_MIB.key, "384"), ByteUnit.MiB) + val overheadFactor = sparkConf.getDouble(COMET_MEMORY_OVERHEAD_FACTOR.key, 0.2) + + val overHeadMemFromConf = sparkConf + .getOption(COMET_MEMORY_OVERHEAD.key) + .map(ConfigHelpers.byteFromString(_, ByteUnit.MiB)) + + overHeadMemFromConf.getOrElse(math.max((overheadFactor * executorMemoryMiB).toLong, minimum)) + } + + /** Calculates required memory overhead in bytes per executor process for Comet. */ + def getCometMemoryOverhead(sparkConf: SparkConf): Long = { + ByteUnit.MiB.toBytes(getCometMemoryOverheadInMiB(sparkConf)) + } +} diff --git a/spark/src/main/scala/org/apache/comet/MetricsSupport.scala b/spark/src/main/scala/org/apache/comet/MetricsSupport.scala new file mode 100644 index 0000000000..6d08423b1a --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/MetricsSupport.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import org.apache.spark.SparkContext +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * A trait for Comet operators that support SQL metrics + */ +trait MetricsSupport { + protected var metrics: Map[String, SQLMetric] = Map.empty + + def initMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = { + metrics = Map( + "ParquetRowGroups" -> SQLMetrics.createMetric( + sparkContext, + "num of Parquet row groups read"), + "ParquetNativeDecodeTime" -> SQLMetrics.createNanoTimingMetric( + sparkContext, + "time spent in Parquet native decoding"), + "ParquetNativeLoadTime" -> SQLMetrics.createNanoTimingMetric( + sparkContext, + "time spent in loading Parquet native vectors"), + "ParquetLoadRowGroupTime" -> SQLMetrics.createNanoTimingMetric( + sparkContext, + "time spent in loading Parquet row groups"), + "ParquetInputFileReadTime" -> SQLMetrics.createNanoTimingMetric( + sparkContext, + "time spent in reading Parquet file from storage"), + "ParquetInputFileReadSize" -> SQLMetrics.createSizeMetric( + sparkContext, + "read size when reading Parquet file from storage (MB)"), + "ParquetInputFileReadThroughput" -> SQLMetrics.createAverageMetric( + sparkContext, + "read throughput when reading Parquet file from storage (MB/sec)")) + metrics + } +} diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala new file mode 100644 index 0000000000..c930c7d763 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.util.Map + +import org.apache.spark.sql.comet.CometMetricNode + +class Native extends NativeBase { + + /** + * Create a native query plan from execution SparkPlan serialized in bytes. + * @param id + * The id of the query plan. + * @param configMap + * The Java Map object for the configs of native engine. + * @param plan + * the bytes of serialized SparkPlan. + * @param metrics + * the native metrics of SparkPlan. + * @return + * the address to native query plan. + */ + @native def createPlan( + id: Long, + configMap: Map[String, String], + plan: Array[Byte], + metrics: CometMetricNode): Long + + /** + * Return the native query plan string for the given address of native query plan. For debugging + * purpose. + * + * @param plan + * the address to native query plan. + * @return + * the string of native query plan. + */ + @native def getPlanString(plan: Long): String + + /** + * Execute a native query plan based on given input Arrow arrays. + * + * @param plan + * the address to native query plan. + * @param addresses + * the array of addresses of input Arrow arrays. The addresses are exported from Arrow Arrays + * so the number of addresses is always even number in the sequence like [array_address1, + * schema_address1, array_address2, schema_address2, ...]. Note that we can pass empty + * addresses to this API. In this case, it indicates there are no more input arrays to the + * native query plan, but the query plan possibly can still execute to produce output batch + * because it might contain blocking operators such as Sort, Aggregate. When this API returns + * an empty array back, it means the native query plan is finished. + * @param finishes + * whether the end of input arrays is reached for each input. If this is set to true, the + * native library will know there is no more inputs. But it doesn't mean the execution is + * finished immediately. For some blocking operators native execution will continue to output. + * @param numRows + * the number of rows in the batch. + * @return + * an array containing: 1) the status flag (0 for pending, 1 for normal returned arrays, + * -1 for end of output), 2) (optional) the number of rows if returned flag is 1 3) the + * addresses of output Arrow arrays + */ + @native def executePlan( + plan: Long, + addresses: Array[Array[Long]], + finishes: Array[Boolean], + numRows: Int): Array[Long] + + /** + * Peeks the next batch of output Arrow arrays from the native query plan without pulling any + * input batches. + * + * @param plan + * the address to native query plan. + * @return + * an array containing: 1) the status flag (0 for pending, 1 for normal returned arrays, 2) + * (optional) the number of rows if returned flag is 1 3) the addresses of output Arrow arrays + */ + @native def peekNext(plan: Long): Array[Long] + + /** + * Release and drop the native query plan object and context object. + * + * @param plan + * the address to native query plan. + */ + @native def releasePlan(plan: Long): Unit +} diff --git a/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala b/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala new file mode 100644 index 0000000000..ac871cf60b --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import scala.collection.JavaConverters + +import org.apache.hadoop.conf.Configuration +import org.apache.parquet.filter2.predicate.FilterApi +import org.apache.parquet.hadoop.ParquetInputFormat +import org.apache.parquet.hadoop.metadata.FileMetaData +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec +import org.apache.spark.sql.execution.datasources.DataSourceUtils +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.RecordReaderIterator +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions +import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.{DateType, StructType, TimestampType} +import org.apache.spark.util.SerializableConfiguration + +import org.apache.comet.CometConf +import org.apache.comet.MetricsSupport +import org.apache.comet.shims.ShimSQLConf +import org.apache.comet.vector.CometVector + +/** + * A Comet specific Parquet format. This mostly reuse the functionalities from Spark's + * [[ParquetFileFormat]], but overrides: + * + * - `vectorTypes`, so Spark allocates [[CometVector]] instead of it's own on-heap or off-heap + * column vector in the whole-stage codegen path. + * - `supportBatch`, which simply returns true since data types should have already been checked + * in [[org.apache.comet.CometSparkSessionExtensions]] + * - `buildReaderWithPartitionValues`, so Spark calls Comet's Parquet reader to read values. + */ +class CometParquetFileFormat extends ParquetFileFormat with MetricsSupport with ShimSQLConf { + override def shortName(): String = "parquet" + override def toString: String = "CometParquet" + override def hashCode(): Int = getClass.hashCode() + override def equals(other: Any): Boolean = other.isInstanceOf[CometParquetFileFormat] + + override def vectorTypes( + requiredSchema: StructType, + partitionSchema: StructType, + sqlConf: SQLConf): Option[Seq[String]] = { + val length = requiredSchema.fields.length + partitionSchema.fields.length + Option(Seq.fill(length)(classOf[CometVector].getName)) + } + + override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = true + + override def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + val sqlConf = sparkSession.sessionState.conf + CometParquetFileFormat.populateConf(sqlConf, hadoopConf) + val broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + + val isCaseSensitive = sqlConf.caseSensitiveAnalysis + val useFieldId = CometParquetUtils.readFieldId(sqlConf) + val ignoreMissingIds = CometParquetUtils.ignoreMissingIds(sqlConf) + val pushDownDate = sqlConf.parquetFilterPushDownDate + val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp + val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal + val pushDownStringPredicate = getPushDownStringPredicate(sqlConf) + val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold + val optionsMap = CaseInsensitiveMap[String](options) + val parquetOptions = new ParquetOptions(optionsMap, sqlConf) + val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead + val parquetFilterPushDown = sqlConf.parquetFilterPushDown + + // Comet specific configurations + val capacity = CometConf.COMET_BATCH_SIZE.get(sqlConf) + + (file: PartitionedFile) => { + val sharedConf = broadcastedHadoopConf.value.value + val footer = FooterReader.readFooter(sharedConf, file) + val footerFileMetaData = footer.getFileMetaData + val datetimeRebaseSpec = CometParquetFileFormat.getDatetimeRebaseSpec( + file, + requiredSchema, + sharedConf, + footerFileMetaData, + datetimeRebaseModeInRead) + + val pushed = if (parquetFilterPushDown) { + val parquetSchema = footerFileMetaData.getSchema + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringPredicate, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + filters + // Collects all converted Parquet filter predicates. Notice that not all predicates can + // be converted (`ParquetFilters.createFilter` returns an `Option`). That's why a + // `flatMap` is used here. + .flatMap(parquetFilters.createFilter) + .reduceOption(FilterApi.and) + } else { + None + } + pushed.foreach(p => ParquetInputFormat.setFilterPredicate(sharedConf, p)) + + val batchReader = new BatchReader( + sharedConf, + file, + footer, + capacity, + requiredSchema, + isCaseSensitive, + useFieldId, + ignoreMissingIds, + datetimeRebaseSpec.mode == LegacyBehaviorPolicy.CORRECTED, + partitionSchema, + file.partitionValues, + JavaConverters.mapAsJavaMap(metrics)) + val iter = new RecordReaderIterator(batchReader) + try { + batchReader.init() + iter.asInstanceOf[Iterator[InternalRow]] + } catch { + case e: Throwable => + iter.close() + throw e + } + } + } +} + +object CometParquetFileFormat extends Logging { + + /** + * Populates Parquet related configurations from the input `sqlConf` to the `hadoopConf` + */ + def populateConf(sqlConf: SQLConf, hadoopConf: Configuration): Unit = { + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) + hadoopConf.setBoolean( + SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, + sqlConf.nestedSchemaPruningEnabled) + hadoopConf.setBoolean(SQLConf.CASE_SENSITIVE.key, sqlConf.caseSensitiveAnalysis) + + // Sets flags for `ParquetToSparkSchemaConverter` + hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, sqlConf.isParquetBinaryAsString) + hadoopConf.setBoolean( + SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, + sqlConf.isParquetINT96AsTimestamp) + + // Comet specific configs + hadoopConf.setBoolean( + CometConf.COMET_PARQUET_ENABLE_DIRECT_BUFFER.key, + CometConf.COMET_PARQUET_ENABLE_DIRECT_BUFFER.get()) + hadoopConf.setBoolean( + CometConf.COMET_USE_DECIMAL_128.key, + CometConf.COMET_USE_DECIMAL_128.get()) + hadoopConf.setBoolean( + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key, + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.get()) + } + + def getDatetimeRebaseSpec( + file: PartitionedFile, + sparkSchema: StructType, + sharedConf: Configuration, + footerFileMetaData: FileMetaData, + datetimeRebaseModeInRead: String): RebaseSpec = { + val exceptionOnRebase = sharedConf.getBoolean( + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key, + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.defaultValue.get) + var datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + datetimeRebaseModeInRead) + val hasDateOrTimestamp = sparkSchema.exists(f => + f.dataType match { + case DateType | TimestampType => true + case _ => false + }) + + if (hasDateOrTimestamp && datetimeRebaseSpec.mode == LegacyBehaviorPolicy.LEGACY) { + if (exceptionOnRebase) { + logWarning( + s"""Found Parquet file $file that could potentially contain dates/timestamps that were + written in legacy hybrid Julian/Gregorian calendar. Unlike Spark 3+, which will rebase + and return these according to the new Proleptic Gregorian calendar, Comet will throw + exception when reading them. If you want to read them as it is according to the hybrid + Julian/Gregorian calendar, please set `spark.comet.exceptionOnDatetimeRebase` to + false. Otherwise, if you want to read them according to the new Proleptic Gregorian + calendar, please disable Comet for this query.""") + } else { + // do not throw exception on rebase - read as it is + datetimeRebaseSpec = datetimeRebaseSpec.copy(LegacyBehaviorPolicy.CORRECTED) + } + } + + datetimeRebaseSpec + } +} diff --git a/spark/src/main/scala/org/apache/comet/parquet/CometParquetPartitionReaderFactory.scala b/spark/src/main/scala/org/apache/comet/parquet/CometParquetPartitionReaderFactory.scala new file mode 100644 index 0000000000..693af125b9 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/parquet/CometParquetPartitionReaderFactory.scala @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import scala.collection.JavaConverters +import scala.collection.mutable + +import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate} +import org.apache.parquet.hadoop.ParquetInputFormat +import org.apache.parquet.hadoop.metadata.ParquetMetadata +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.PartitionReader +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions +import org.apache.spark.sql.execution.datasources.v2.FilePartitionReaderFactory +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +import org.apache.comet.{CometConf, CometRuntimeException} +import org.apache.comet.shims.ShimSQLConf + +case class CometParquetPartitionReaderFactory( + @transient sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + readDataSchema: StructType, + partitionSchema: StructType, + filters: Array[Filter], + options: ParquetOptions, + metrics: Map[String, SQLMetric]) + extends FilePartitionReaderFactory + with ShimSQLConf + with Logging { + + private val isCaseSensitive = sqlConf.caseSensitiveAnalysis + private val useFieldId = CometParquetUtils.readFieldId(sqlConf) + private val ignoreMissingIds = CometParquetUtils.ignoreMissingIds(sqlConf) + private val pushDownDate = sqlConf.parquetFilterPushDownDate + private val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp + private val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal + private val pushDownStringPredicate = getPushDownStringPredicate(sqlConf) + private val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold + private val datetimeRebaseModeInRead = options.datetimeRebaseModeInRead + private val parquetFilterPushDown = sqlConf.parquetFilterPushDown + + // Comet specific configurations + private val batchSize = CometConf.COMET_BATCH_SIZE.get(sqlConf) + + // This is only called at executor on a Broadcast variable, so we don't want it to be + // materialized at driver. + @transient private lazy val preFetchEnabled = { + val conf = broadcastedConf.value.value + + conf.getBoolean( + CometConf.COMET_SCAN_PREFETCH_ENABLED.key, + CometConf.COMET_SCAN_PREFETCH_ENABLED.defaultValue.get) + } + + private var cometReaders: Iterator[BatchReader] = _ + private val cometReaderExceptionMap = new mutable.HashMap[PartitionedFile, Throwable]() + + // TODO: we may want to revisit this as we're going to only support flat types at the beginning + override def supportColumnarReads(partition: InputPartition): Boolean = true + + override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { + if (preFetchEnabled) { + val filePartition = partition.asInstanceOf[FilePartition] + val conf = broadcastedConf.value.value + + val threadNum = conf.getInt( + CometConf.COMET_SCAN_PREFETCH_THREAD_NUM.key, + CometConf.COMET_SCAN_PREFETCH_THREAD_NUM.defaultValue.get) + val prefetchThreadPool = CometPrefetchThreadPool.getOrCreateThreadPool(threadNum) + + this.cometReaders = filePartition.files + .map { file => + // `init()` call is deferred to when the prefetch task begins. + // Otherwise we will hold too many resources for readers which are not ready + // to prefetch. + val cometReader = buildCometReader(file) + if (cometReader != null) { + cometReader.submitPrefetchTask(prefetchThreadPool) + } + + cometReader + } + .toSeq + .toIterator + } + + super.createColumnarReader(partition) + } + + override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = + throw new UnsupportedOperationException("Comet doesn't support 'buildReader'") + + private def buildCometReader(file: PartitionedFile): BatchReader = { + val conf = broadcastedConf.value.value + + try { + val (datetimeRebaseSpec, footer, filters) = getFilter(file) + filters.foreach(pushed => ParquetInputFormat.setFilterPredicate(conf, pushed)) + val cometReader = new BatchReader( + conf, + file, + footer, + batchSize, + readDataSchema, + isCaseSensitive, + useFieldId, + ignoreMissingIds, + datetimeRebaseSpec.mode == LegacyBehaviorPolicy.CORRECTED, + partitionSchema, + file.partitionValues, + JavaConverters.mapAsJavaMap(metrics)) + val taskContext = Option(TaskContext.get) + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => cometReader.close())) + return cometReader + } catch { + case e: Throwable if preFetchEnabled => + // Keep original exception + cometReaderExceptionMap.put(file, e) + } + null + } + + override def buildColumnarReader(file: PartitionedFile): PartitionReader[ColumnarBatch] = { + val cometReader = if (!preFetchEnabled) { + // Prefetch is not enabled, create comet reader and initiate it. + val cometReader = buildCometReader(file) + cometReader.init() + + cometReader + } else { + // If prefetch is enabled, we already tried to access the file when in `buildCometReader`. + // It is possibly we got an exception like `FileNotFoundException` and we need to throw it + // now to let Spark handle it. + val reader = cometReaders.next() + val exception = cometReaderExceptionMap.get(file) + exception.foreach(e => throw e) + + if (reader == null) { + throw new CometRuntimeException(s"Cannot find comet file reader for $file") + } + reader + } + CometPartitionReader(cometReader) + } + + def getFilter(file: PartitionedFile): (RebaseSpec, ParquetMetadata, Option[FilterPredicate]) = { + val sharedConf = broadcastedConf.value.value + val footer = FooterReader.readFooter(sharedConf, file) + val footerFileMetaData = footer.getFileMetaData + val datetimeRebaseSpec = CometParquetFileFormat.getDatetimeRebaseSpec( + file, + readDataSchema, + sharedConf, + footerFileMetaData, + datetimeRebaseModeInRead) + + val pushed = if (parquetFilterPushDown) { + val parquetSchema = footerFileMetaData.getSchema + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringPredicate, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + filters + // Collects all converted Parquet filter predicates. Notice that not all predicates can be + // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` + // is used here. + .flatMap(parquetFilters.createFilter) + .reduceOption(FilterApi.and) + } else { + None + } + (datetimeRebaseSpec, footer, pushed) + } + + override def createReader(inputPartition: InputPartition): PartitionReader[InternalRow] = + throw new UnsupportedOperationException("Only 'createColumnarReader' is supported.") + + /** + * A simple adapter on Comet's [[BatchReader]]. + */ + protected case class CometPartitionReader(reader: BatchReader) + extends PartitionReader[ColumnarBatch] { + + override def next(): Boolean = { + reader.nextBatch() + } + + override def get(): ColumnarBatch = { + reader.currentBatch() + } + + override def close(): Unit = { + reader.close() + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/parquet/CometParquetScan.scala b/spark/src/main/scala/org/apache/comet/parquet/CometParquetScan.scala new file mode 100644 index 0000000000..e3cd33b41f --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/parquet/CometParquetScan.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import scala.collection.JavaConverters.mapAsScalaMapConverter + +import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.SerializableConfiguration + +import org.apache.comet.MetricsSupport + +trait CometParquetScan extends FileScan with MetricsSupport { + def sparkSession: SparkSession + def hadoopConf: Configuration + def readDataSchema: StructType + def readPartitionSchema: StructType + def pushedFilters: Array[Filter] + def options: CaseInsensitiveStringMap + + override def equals(obj: Any): Boolean = obj match { + case other: CometParquetScan => + super.equals(other) && readDataSchema == other.readDataSchema && + readPartitionSchema == other.readPartitionSchema && + equivalentFilters(pushedFilters, other.pushedFilters) + case _ => false + } + + override def hashCode(): Int = getClass.hashCode() + + override def createReaderFactory(): PartitionReaderFactory = { + val sqlConf = sparkSession.sessionState.conf + CometParquetFileFormat.populateConf(sqlConf, hadoopConf) + val broadcastedConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + CometParquetPartitionReaderFactory( + sqlConf, + broadcastedConf, + readDataSchema, + readPartitionSchema, + pushedFilters, + new ParquetOptions(options.asScala.toMap, sqlConf), + metrics) + } +} + +object CometParquetScan { + def apply(scan: ParquetScan): CometParquetScan = + new ParquetScan( + scan.sparkSession, + scan.hadoopConf, + scan.fileIndex, + scan.dataSchema, + scan.readDataSchema, + scan.readPartitionSchema, + scan.pushedFilters, + scan.options, + partitionFilters = scan.partitionFilters, + dataFilters = scan.dataFilters) with CometParquetScan +} diff --git a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala new file mode 100644 index 0000000000..5994dfb41e --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala @@ -0,0 +1,882 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Long => JLong, Short => JShort} +import java.math.{BigDecimal => JBigDecimal} +import java.sql.{Date, Timestamp} +import java.time.{Duration, Instant, LocalDate, Period} +import java.util.Locale + +import scala.collection.JavaConverters.asScalaBufferConverter + +import org.apache.parquet.column.statistics.{Statistics => ParquetStatistics} +import org.apache.parquet.filter2.predicate._ +import org.apache.parquet.filter2.predicate.SparkFilterApi._ +import org.apache.parquet.io.api.Binary +import org.apache.parquet.schema.{GroupType, LogicalTypeAnnotation, MessageType, PrimitiveComparator, PrimitiveType, Type} +import org.apache.parquet.schema.LogicalTypeAnnotation.{DecimalLogicalTypeAnnotation, TimeUnit} +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ +import org.apache.parquet.schema.Type.Repetition +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, CaseInsensitiveMap, DateTimeUtils, IntervalUtils} +import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseGregorianToJulianMicros, RebaseSpec} +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy +import org.apache.spark.sql.sources +import org.apache.spark.unsafe.types.UTF8String + +import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus + +/** + * Copied from Spark 3.2 & 3.4, in order to fix Parquet shading issue. TODO: find a way to remove + * this duplication + * + * Some utility function to convert Spark data source filters to Parquet filters. + */ +class ParquetFilters( + schema: MessageType, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownStringPredicate: Boolean, + pushDownInFilterThreshold: Int, + caseSensitive: Boolean, + datetimeRebaseSpec: RebaseSpec) { + // A map which contains parquet field name and data type, if predicate push down applies. + // + // Each key in `nameToParquetField` represents a column; `dots` are used as separators for + // nested columns. If any part of the names contains `dots`, it is quoted to avoid confusion. + // See `org.apache.spark.sql.connector.catalog.quote` for implementation details. + private val nameToParquetField: Map[String, ParquetPrimitiveField] = { + // Recursively traverse the parquet schema to get primitive fields that can be pushed-down. + // `parentFieldNames` is used to keep track of the current nested level when traversing. + def getPrimitiveFields( + fields: Seq[Type], + parentFieldNames: Array[String] = Array.empty): Seq[ParquetPrimitiveField] = { + fields.flatMap { + // Parquet only supports predicate push-down for non-repeated primitive types. + // TODO(SPARK-39393): Remove extra condition when parquet added filter predicate support for + // repeated columns (https://issues.apache.org/jira/browse/PARQUET-34) + case p: PrimitiveType if p.getRepetition != Repetition.REPEATED => + Some( + ParquetPrimitiveField( + fieldNames = parentFieldNames :+ p.getName, + fieldType = ParquetSchemaType( + p.getLogicalTypeAnnotation, + p.getPrimitiveTypeName, + p.getTypeLength))) + // Note that when g is a `Struct`, `g.getOriginalType` is `null`. + // When g is a `Map`, `g.getOriginalType` is `MAP`. + // When g is a `List`, `g.getOriginalType` is `LIST`. + case g: GroupType if g.getOriginalType == null => + getPrimitiveFields(g.getFields.asScala.toSeq, parentFieldNames :+ g.getName) + // Parquet only supports push-down for primitive types; as a result, Map and List types + // are removed. + case _ => None + } + } + + val primitiveFields = getPrimitiveFields(schema.getFields.asScala.toSeq).map { field => + (field.fieldNames.toSeq.map(quoteIfNeeded).mkString("."), field) + } + if (caseSensitive) { + primitiveFields.toMap + } else { + // Don't consider ambiguity here, i.e. more than one field is matched in case insensitive + // mode, just skip pushdown for these fields, they will trigger Exception when reading, + // See: SPARK-25132. + val dedupPrimitiveFields = + primitiveFields + .groupBy(_._1.toLowerCase(Locale.ROOT)) + .filter(_._2.size == 1) + .mapValues(_.head._2) + CaseInsensitiveMap(dedupPrimitiveFields.toMap) + } + } + + /** + * Holds a single primitive field information stored in the underlying parquet file. + * + * @param fieldNames + * a field name as an array of string multi-identifier in parquet file + * @param fieldType + * field type related info in parquet file + */ + private case class ParquetPrimitiveField( + fieldNames: Array[String], + fieldType: ParquetSchemaType) + + private case class ParquetSchemaType( + logicalTypeAnnotation: LogicalTypeAnnotation, + primitiveTypeName: PrimitiveTypeName, + length: Int) + + private val ParquetBooleanType = ParquetSchemaType(null, BOOLEAN, 0) + private val ParquetByteType = + ParquetSchemaType(LogicalTypeAnnotation.intType(8, true), INT32, 0) + private val ParquetShortType = + ParquetSchemaType(LogicalTypeAnnotation.intType(16, true), INT32, 0) + private val ParquetIntegerType = ParquetSchemaType(null, INT32, 0) + private val ParquetLongType = ParquetSchemaType(null, INT64, 0) + private val ParquetFloatType = ParquetSchemaType(null, FLOAT, 0) + private val ParquetDoubleType = ParquetSchemaType(null, DOUBLE, 0) + private val ParquetStringType = + ParquetSchemaType(LogicalTypeAnnotation.stringType(), BINARY, 0) + private val ParquetBinaryType = ParquetSchemaType(null, BINARY, 0) + private val ParquetDateType = + ParquetSchemaType(LogicalTypeAnnotation.dateType(), INT32, 0) + private val ParquetTimestampMicrosType = + ParquetSchemaType(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS), INT64, 0) + private val ParquetTimestampMillisType = + ParquetSchemaType(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS), INT64, 0) + + private def dateToDays(date: Any): Int = { + val gregorianDays = date match { + case d: Date => DateTimeUtils.fromJavaDate(d) + case ld: LocalDate => DateTimeUtils.localDateToDays(ld) + } + datetimeRebaseSpec.mode match { + case LegacyBehaviorPolicy.LEGACY => rebaseGregorianToJulianDays(gregorianDays) + case _ => gregorianDays + } + } + + private def timestampToMicros(v: Any): JLong = { + val gregorianMicros = v match { + case i: Instant => DateTimeUtils.instantToMicros(i) + case t: Timestamp => DateTimeUtils.fromJavaTimestamp(t) + } + datetimeRebaseSpec.mode match { + case LegacyBehaviorPolicy.LEGACY => + rebaseGregorianToJulianMicros(datetimeRebaseSpec.timeZone, gregorianMicros) + case _ => gregorianMicros + } + } + + private def decimalToInt32(decimal: JBigDecimal): Integer = decimal.unscaledValue().intValue() + + private def decimalToInt64(decimal: JBigDecimal): JLong = decimal.unscaledValue().longValue() + + private def decimalToByteArray(decimal: JBigDecimal, numBytes: Int): Binary = { + val decimalBuffer = new Array[Byte](numBytes) + val bytes = decimal.unscaledValue().toByteArray + + val fixedLengthBytes = if (bytes.length == numBytes) { + bytes + } else { + val signByte = if (bytes.head < 0) -1: Byte else 0: Byte + java.util.Arrays.fill(decimalBuffer, 0, numBytes - bytes.length, signByte) + System.arraycopy(bytes, 0, decimalBuffer, numBytes - bytes.length, bytes.length) + decimalBuffer + } + Binary.fromConstantByteArray(fixedLengthBytes, 0, numBytes) + } + + private def timestampToMillis(v: Any): JLong = { + val micros = timestampToMicros(v) + val millis = DateTimeUtils.microsToMillis(micros) + millis.asInstanceOf[JLong] + } + + private def toIntValue(v: Any): Integer = { + Option(v) + .map { + case p: Period => IntervalUtils.periodToMonths(p) + case n => n.asInstanceOf[Number].intValue + } + .map(_.asInstanceOf[Integer]) + .orNull + } + + private def toLongValue(v: Any): JLong = v match { + case d: Duration => IntervalUtils.durationToMicros(d) + case l => l.asInstanceOf[JLong] + } + + private val makeEq + : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { + case ParquetBooleanType => + (n: Array[String], v: Any) => FilterApi.eq(booleanColumn(n), v.asInstanceOf[JBoolean]) + case ParquetByteType | ParquetShortType | ParquetIntegerType => + (n: Array[String], v: Any) => + FilterApi.eq( + intColumn(n), + Option(v).map(_.asInstanceOf[Number].intValue.asInstanceOf[Integer]).orNull) + case ParquetLongType => + (n: Array[String], v: Any) => FilterApi.eq(longColumn(n), v.asInstanceOf[JLong]) + case ParquetFloatType => + (n: Array[String], v: Any) => FilterApi.eq(floatColumn(n), v.asInstanceOf[JFloat]) + case ParquetDoubleType => + (n: Array[String], v: Any) => FilterApi.eq(doubleColumn(n), v.asInstanceOf[JDouble]) + + // Binary.fromString and Binary.fromByteArray don't accept null values + case ParquetStringType => + (n: Array[String], v: Any) => + FilterApi.eq( + binaryColumn(n), + Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + case ParquetBinaryType => + (n: Array[String], v: Any) => + FilterApi.eq( + binaryColumn(n), + Option(v).map(_ => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) + case ParquetDateType if pushDownDate => + (n: Array[String], v: Any) => + FilterApi.eq( + intColumn(n), + Option(v).map(date => dateToDays(date).asInstanceOf[Integer]).orNull) + case ParquetTimestampMicrosType if pushDownTimestamp => + (n: Array[String], v: Any) => + FilterApi.eq(longColumn(n), Option(v).map(timestampToMicros).orNull) + case ParquetTimestampMillisType if pushDownTimestamp => + (n: Array[String], v: Any) => + FilterApi.eq(longColumn(n), Option(v).map(timestampToMillis).orNull) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.eq( + intColumn(n), + Option(v).map(d => decimalToInt32(d.asInstanceOf[JBigDecimal])).orNull) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.eq( + longColumn(n), + Option(v).map(d => decimalToInt64(d.asInstanceOf[JBigDecimal])).orNull) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) + if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.eq( + binaryColumn(n), + Option(v).map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)).orNull) + } + + private val makeNotEq + : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { + case ParquetBooleanType => + (n: Array[String], v: Any) => FilterApi.notEq(booleanColumn(n), v.asInstanceOf[JBoolean]) + case ParquetByteType | ParquetShortType | ParquetIntegerType => + (n: Array[String], v: Any) => + FilterApi.notEq( + intColumn(n), + Option(v).map(_.asInstanceOf[Number].intValue.asInstanceOf[Integer]).orNull) + case ParquetLongType => + (n: Array[String], v: Any) => FilterApi.notEq(longColumn(n), v.asInstanceOf[JLong]) + case ParquetFloatType => + (n: Array[String], v: Any) => FilterApi.notEq(floatColumn(n), v.asInstanceOf[JFloat]) + case ParquetDoubleType => + (n: Array[String], v: Any) => FilterApi.notEq(doubleColumn(n), v.asInstanceOf[JDouble]) + + case ParquetStringType => + (n: Array[String], v: Any) => + FilterApi.notEq( + binaryColumn(n), + Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + case ParquetBinaryType => + (n: Array[String], v: Any) => + FilterApi.notEq( + binaryColumn(n), + Option(v).map(_ => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) + case ParquetDateType if pushDownDate => + (n: Array[String], v: Any) => + FilterApi.notEq( + intColumn(n), + Option(v).map(date => dateToDays(date).asInstanceOf[Integer]).orNull) + case ParquetTimestampMicrosType if pushDownTimestamp => + (n: Array[String], v: Any) => + FilterApi.notEq(longColumn(n), Option(v).map(timestampToMicros).orNull) + case ParquetTimestampMillisType if pushDownTimestamp => + (n: Array[String], v: Any) => + FilterApi.notEq(longColumn(n), Option(v).map(timestampToMillis).orNull) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.notEq( + intColumn(n), + Option(v).map(d => decimalToInt32(d.asInstanceOf[JBigDecimal])).orNull) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.notEq( + longColumn(n), + Option(v).map(d => decimalToInt64(d.asInstanceOf[JBigDecimal])).orNull) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) + if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.notEq( + binaryColumn(n), + Option(v).map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)).orNull) + } + + private val makeLt + : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { + case ParquetByteType | ParquetShortType | ParquetIntegerType => + (n: Array[String], v: Any) => + FilterApi.lt(intColumn(n), v.asInstanceOf[Number].intValue.asInstanceOf[Integer]) + case ParquetLongType => + (n: Array[String], v: Any) => FilterApi.lt(longColumn(n), v.asInstanceOf[JLong]) + case ParquetFloatType => + (n: Array[String], v: Any) => FilterApi.lt(floatColumn(n), v.asInstanceOf[JFloat]) + case ParquetDoubleType => + (n: Array[String], v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[JDouble]) + + case ParquetStringType => + (n: Array[String], v: Any) => + FilterApi.lt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + case ParquetBinaryType => + (n: Array[String], v: Any) => + FilterApi.lt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + case ParquetDateType if pushDownDate => + (n: Array[String], v: Any) => + FilterApi.lt(intColumn(n), dateToDays(v).asInstanceOf[Integer]) + case ParquetTimestampMicrosType if pushDownTimestamp => + (n: Array[String], v: Any) => FilterApi.lt(longColumn(n), timestampToMicros(v)) + case ParquetTimestampMillisType if pushDownTimestamp => + (n: Array[String], v: Any) => FilterApi.lt(longColumn(n), timestampToMillis(v)) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.lt(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.lt(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) + if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.lt(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) + } + + private val makeLtEq + : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { + case ParquetByteType | ParquetShortType | ParquetIntegerType => + (n: Array[String], v: Any) => + FilterApi.ltEq(intColumn(n), v.asInstanceOf[Number].intValue.asInstanceOf[Integer]) + case ParquetLongType => + (n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), v.asInstanceOf[JLong]) + case ParquetFloatType => + (n: Array[String], v: Any) => FilterApi.ltEq(floatColumn(n), v.asInstanceOf[JFloat]) + case ParquetDoubleType => + (n: Array[String], v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[JDouble]) + + case ParquetStringType => + (n: Array[String], v: Any) => + FilterApi.ltEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + case ParquetBinaryType => + (n: Array[String], v: Any) => + FilterApi.ltEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + case ParquetDateType if pushDownDate => + (n: Array[String], v: Any) => + FilterApi.ltEq(intColumn(n), dateToDays(v).asInstanceOf[Integer]) + case ParquetTimestampMicrosType if pushDownTimestamp => + (n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), timestampToMicros(v)) + case ParquetTimestampMillisType if pushDownTimestamp => + (n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), timestampToMillis(v)) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.ltEq(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.ltEq(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) + if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.ltEq(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) + } + + private val makeGt + : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { + case ParquetByteType | ParquetShortType | ParquetIntegerType => + (n: Array[String], v: Any) => + FilterApi.gt(intColumn(n), v.asInstanceOf[Number].intValue.asInstanceOf[Integer]) + case ParquetLongType => + (n: Array[String], v: Any) => FilterApi.gt(longColumn(n), v.asInstanceOf[JLong]) + case ParquetFloatType => + (n: Array[String], v: Any) => FilterApi.gt(floatColumn(n), v.asInstanceOf[JFloat]) + case ParquetDoubleType => + (n: Array[String], v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[JDouble]) + + case ParquetStringType => + (n: Array[String], v: Any) => + FilterApi.gt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + case ParquetBinaryType => + (n: Array[String], v: Any) => + FilterApi.gt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + case ParquetDateType if pushDownDate => + (n: Array[String], v: Any) => + FilterApi.gt(intColumn(n), dateToDays(v).asInstanceOf[Integer]) + case ParquetTimestampMicrosType if pushDownTimestamp => + (n: Array[String], v: Any) => FilterApi.gt(longColumn(n), timestampToMicros(v)) + case ParquetTimestampMillisType if pushDownTimestamp => + (n: Array[String], v: Any) => FilterApi.gt(longColumn(n), timestampToMillis(v)) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.gt(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.gt(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) + if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.gt(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) + } + + private val makeGtEq + : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { + case ParquetByteType | ParquetShortType | ParquetIntegerType => + (n: Array[String], v: Any) => + FilterApi.gtEq(intColumn(n), v.asInstanceOf[Number].intValue.asInstanceOf[Integer]) + case ParquetLongType => + (n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), v.asInstanceOf[JLong]) + case ParquetFloatType => + (n: Array[String], v: Any) => FilterApi.gtEq(floatColumn(n), v.asInstanceOf[JFloat]) + case ParquetDoubleType => + (n: Array[String], v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[JDouble]) + + case ParquetStringType => + (n: Array[String], v: Any) => + FilterApi.gtEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + case ParquetBinaryType => + (n: Array[String], v: Any) => + FilterApi.gtEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + case ParquetDateType if pushDownDate => + (n: Array[String], v: Any) => + FilterApi.gtEq(intColumn(n), dateToDays(v).asInstanceOf[Integer]) + case ParquetTimestampMicrosType if pushDownTimestamp => + (n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), timestampToMicros(v)) + case ParquetTimestampMillisType if pushDownTimestamp => + (n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), timestampToMillis(v)) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.gtEq(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.gtEq(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) + if pushDownDecimal => + (n: Array[String], v: Any) => + FilterApi.gtEq(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) + } + + private val makeInPredicate: PartialFunction[ + ParquetSchemaType, + (Array[String], Array[Any], ParquetStatistics[_]) => FilterPredicate] = { + case ParquetByteType | ParquetShortType | ParquetIntegerType => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(toIntValue(_).toInt).foreach(statistics.updateStats) + FilterApi.and( + FilterApi.gtEq(intColumn(n), statistics.genericGetMin().asInstanceOf[Integer]), + FilterApi.ltEq(intColumn(n), statistics.genericGetMax().asInstanceOf[Integer])) + + case ParquetLongType => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(toLongValue).foreach(statistics.updateStats(_)) + FilterApi.and( + FilterApi.gtEq(longColumn(n), statistics.genericGetMin().asInstanceOf[JLong]), + FilterApi.ltEq(longColumn(n), statistics.genericGetMax().asInstanceOf[JLong])) + + case ParquetFloatType => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(_.asInstanceOf[JFloat]).foreach(statistics.updateStats(_)) + FilterApi.and( + FilterApi.gtEq(floatColumn(n), statistics.genericGetMin().asInstanceOf[JFloat]), + FilterApi.ltEq(floatColumn(n), statistics.genericGetMax().asInstanceOf[JFloat])) + + case ParquetDoubleType => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(_.asInstanceOf[JDouble]).foreach(statistics.updateStats(_)) + FilterApi.and( + FilterApi.gtEq(doubleColumn(n), statistics.genericGetMin().asInstanceOf[JDouble]), + FilterApi.ltEq(doubleColumn(n), statistics.genericGetMax().asInstanceOf[JDouble])) + + case ParquetStringType => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(s => Binary.fromString(s.asInstanceOf[String])).foreach(statistics.updateStats) + FilterApi.and( + FilterApi.gtEq(binaryColumn(n), statistics.genericGetMin().asInstanceOf[Binary]), + FilterApi.ltEq(binaryColumn(n), statistics.genericGetMax().asInstanceOf[Binary])) + + case ParquetBinaryType => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(b => Binary.fromReusedByteArray(b.asInstanceOf[Array[Byte]])) + .foreach(statistics.updateStats) + FilterApi.and( + FilterApi.gtEq(binaryColumn(n), statistics.genericGetMin().asInstanceOf[Binary]), + FilterApi.ltEq(binaryColumn(n), statistics.genericGetMax().asInstanceOf[Binary])) + + case ParquetDateType if pushDownDate => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(dateToDays).map(_.asInstanceOf[Integer]).foreach(statistics.updateStats(_)) + FilterApi.and( + FilterApi.gtEq(intColumn(n), statistics.genericGetMin().asInstanceOf[Integer]), + FilterApi.ltEq(intColumn(n), statistics.genericGetMax().asInstanceOf[Integer])) + + case ParquetTimestampMicrosType if pushDownTimestamp => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(timestampToMicros).foreach(statistics.updateStats(_)) + FilterApi.and( + FilterApi.gtEq(longColumn(n), statistics.genericGetMin().asInstanceOf[JLong]), + FilterApi.ltEq(longColumn(n), statistics.genericGetMax().asInstanceOf[JLong])) + + case ParquetTimestampMillisType if pushDownTimestamp => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(timestampToMillis).foreach(statistics.updateStats(_)) + FilterApi.and( + FilterApi.gtEq(longColumn(n), statistics.genericGetMin().asInstanceOf[JLong]), + FilterApi.ltEq(longColumn(n), statistics.genericGetMax().asInstanceOf[JLong])) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(_.asInstanceOf[JBigDecimal]).map(decimalToInt32).foreach(statistics.updateStats(_)) + FilterApi.and( + FilterApi.gtEq(intColumn(n), statistics.genericGetMin().asInstanceOf[Integer]), + FilterApi.ltEq(intColumn(n), statistics.genericGetMax().asInstanceOf[Integer])) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => + (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(_.asInstanceOf[JBigDecimal]).map(decimalToInt64).foreach(statistics.updateStats(_)) + FilterApi.and( + FilterApi.gtEq(longColumn(n), statistics.genericGetMin().asInstanceOf[JLong]), + FilterApi.ltEq(longColumn(n), statistics.genericGetMax().asInstanceOf[JLong])) + + case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) + if pushDownDecimal => + (path: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => + v.map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)) + .foreach(statistics.updateStats) + FilterApi.and( + FilterApi.gtEq(binaryColumn(path), statistics.genericGetMin().asInstanceOf[Binary]), + FilterApi.ltEq(binaryColumn(path), statistics.genericGetMax().asInstanceOf[Binary])) + } + + // Returns filters that can be pushed down when reading Parquet files. + def convertibleFilters(filters: Seq[sources.Filter]): Seq[sources.Filter] = { + filters.flatMap(convertibleFiltersHelper(_, canPartialPushDown = true)) + } + + private def convertibleFiltersHelper( + predicate: sources.Filter, + canPartialPushDown: Boolean): Option[sources.Filter] = { + predicate match { + case sources.And(left, right) => + val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) + val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) + (leftResultOptional, rightResultOptional) match { + case (Some(leftResult), Some(rightResult)) => Some(sources.And(leftResult, rightResult)) + case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) + case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) + case _ => None + } + + case sources.Or(left, right) => + val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) + val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) + if (leftResultOptional.isEmpty || rightResultOptional.isEmpty) { + None + } else { + Some(sources.Or(leftResultOptional.get, rightResultOptional.get)) + } + case sources.Not(pred) => + val resultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) + resultOptional.map(sources.Not) + + case other => + if (createFilter(other).isDefined) { + Some(other) + } else { + None + } + } + } + + /** + * Converts data sources filters to Parquet filter predicates. + */ + def createFilter(predicate: sources.Filter): Option[FilterPredicate] = { + createFilterHelper(predicate, canPartialPushDownConjuncts = true) + } + + // Parquet's type in the given file should be matched to the value's type + // in the pushed filter in order to push down the filter to Parquet. + private def valueCanMakeFilterOn(name: String, value: Any): Boolean = { + value == null || (nameToParquetField(name).fieldType match { + case ParquetBooleanType => value.isInstanceOf[JBoolean] + case ParquetByteType | ParquetShortType | ParquetIntegerType => + if (isSpark34Plus) { + value match { + // Byte/Short/Int are all stored as INT32 in Parquet so filters are built using type + // Int. We don't create a filter if the value would overflow. + case _: JByte | _: JShort | _: Integer => true + case v: JLong => v.longValue() >= Int.MinValue && v.longValue() <= Int.MaxValue + case _ => false + } + } else { + // If not Spark 3.4+, we still following the old behavior as Spark does. + value.isInstanceOf[Number] + } + case ParquetLongType => value.isInstanceOf[JLong] + case ParquetFloatType => value.isInstanceOf[JFloat] + case ParquetDoubleType => value.isInstanceOf[JDouble] + case ParquetStringType => value.isInstanceOf[String] + case ParquetBinaryType => value.isInstanceOf[Array[Byte]] + case ParquetDateType => + value.isInstanceOf[Date] || value.isInstanceOf[LocalDate] + case ParquetTimestampMicrosType | ParquetTimestampMillisType => + value.isInstanceOf[Timestamp] || value.isInstanceOf[Instant] + case ParquetSchemaType(decimalType: DecimalLogicalTypeAnnotation, INT32, _) => + isDecimalMatched(value, decimalType) + case ParquetSchemaType(decimalType: DecimalLogicalTypeAnnotation, INT64, _) => + isDecimalMatched(value, decimalType) + case ParquetSchemaType( + decimalType: DecimalLogicalTypeAnnotation, + FIXED_LEN_BYTE_ARRAY, + _) => + isDecimalMatched(value, decimalType) + case _ => false + }) + } + + // Decimal type must make sure that filter value's scale matched the file. + // If doesn't matched, which would cause data corruption. + private def isDecimalMatched( + value: Any, + decimalLogicalType: DecimalLogicalTypeAnnotation): Boolean = value match { + case decimal: JBigDecimal => + decimal.scale == decimalLogicalType.getScale + case _ => false + } + + private def canMakeFilterOn(name: String, value: Any): Boolean = { + nameToParquetField.contains(name) && valueCanMakeFilterOn(name, value) + } + + /** + * @param predicate + * the input filter predicates. Not all the predicates can be pushed down. + * @param canPartialPushDownConjuncts + * whether a subset of conjuncts of predicates can be pushed down safely. Pushing ONLY one + * side of AND down is safe to do at the top level or none of its ancestors is NOT and OR. + * @return + * the Parquet-native filter predicates that are eligible for pushdown. + */ + private def createFilterHelper( + predicate: sources.Filter, + canPartialPushDownConjuncts: Boolean): Option[FilterPredicate] = { + // NOTE: + // + // For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`, + // which can be casted to `false` implicitly. Please refer to the `eval` method of these + // operators and the `PruneFilters` rule for details. + + // Hyukjin: + // I added [[EqualNullSafe]] with [[org.apache.parquet.filter2.predicate.Operators.Eq]]. + // So, it performs equality comparison identically when given [[sources.Filter]] is [[EqualTo]]. + // The reason why I did this is, that the actual Parquet filter checks null-safe equality + // comparison. + // So I added this and maybe [[EqualTo]] should be changed. It still seems fine though, because + // physical planning does not set `NULL` to [[EqualTo]] but changes it to [[IsNull]] and etc. + // Probably I missed something and obviously this should be changed. + + predicate match { + case sources.IsNull(name) if canMakeFilterOn(name, null) => + makeEq + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, null)) + case sources.IsNotNull(name) if canMakeFilterOn(name, null) => + makeNotEq + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, null)) + + case sources.EqualTo(name, value) if canMakeFilterOn(name, value) => + makeEq + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, value)) + case sources.Not(sources.EqualTo(name, value)) if canMakeFilterOn(name, value) => + makeNotEq + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, value)) + + case sources.EqualNullSafe(name, value) if canMakeFilterOn(name, value) => + makeEq + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, value)) + case sources.Not(sources.EqualNullSafe(name, value)) if canMakeFilterOn(name, value) => + makeNotEq + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, value)) + + case sources.LessThan(name, value) if canMakeFilterOn(name, value) => + makeLt + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, value)) + case sources.LessThanOrEqual(name, value) if canMakeFilterOn(name, value) => + makeLtEq + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, value)) + + case sources.GreaterThan(name, value) if canMakeFilterOn(name, value) => + makeGt + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, value)) + case sources.GreaterThanOrEqual(name, value) if canMakeFilterOn(name, value) => + makeGtEq + .lift(nameToParquetField(name).fieldType) + .map(_(nameToParquetField(name).fieldNames, value)) + + case sources.And(lhs, rhs) => + // At here, it is not safe to just convert one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. + val lhsFilterOption = + createFilterHelper(lhs, canPartialPushDownConjuncts) + val rhsFilterOption = + createFilterHelper(rhs, canPartialPushDownConjuncts) + + (lhsFilterOption, rhsFilterOption) match { + case (Some(lhsFilter), Some(rhsFilter)) => Some(FilterApi.and(lhsFilter, rhsFilter)) + case (Some(lhsFilter), None) if canPartialPushDownConjuncts => Some(lhsFilter) + case (None, Some(rhsFilter)) if canPartialPushDownConjuncts => Some(rhsFilter) + case _ => None + } + + case sources.Or(lhs, rhs) => + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + for { + lhsFilter <- createFilterHelper(lhs, canPartialPushDownConjuncts) + rhsFilter <- createFilterHelper(rhs, canPartialPushDownConjuncts) + } yield FilterApi.or(lhsFilter, rhsFilter) + + case sources.Not(pred) => + createFilterHelper(pred, canPartialPushDownConjuncts = false) + .map(FilterApi.not) + + case sources.In(name, values) + if pushDownInFilterThreshold > 0 && values.nonEmpty && + canMakeFilterOn(name, values.head) => + val fieldType = nameToParquetField(name).fieldType + val fieldNames = nameToParquetField(name).fieldNames + if (values.length <= pushDownInFilterThreshold) { + values.distinct + .flatMap { v => + makeEq.lift(fieldType).map(_(fieldNames, v)) + } + .reduceLeftOption(FilterApi.or) + } else if (canPartialPushDownConjuncts) { + val primitiveType = schema.getColumnDescription(fieldNames).getPrimitiveType + val statistics: ParquetStatistics[_] = ParquetStatistics.createStats(primitiveType) + if (values.contains(null)) { + Seq( + makeEq.lift(fieldType).map(_(fieldNames, null)), + makeInPredicate + .lift(fieldType) + .map(_(fieldNames, values.filter(_ != null), statistics))).flatten + .reduceLeftOption(FilterApi.or) + } else { + makeInPredicate.lift(fieldType).map(_(fieldNames, values, statistics)) + } + } else { + None + } + + case sources.StringStartsWith(name, prefix) + if pushDownStringPredicate && canMakeFilterOn(name, prefix) => + Option(prefix).map { v => + FilterApi.userDefined( + binaryColumn(nameToParquetField(name).fieldNames), + new UserDefinedPredicate[Binary] with Serializable { + private val strToBinary = Binary.fromReusedByteArray(v.getBytes) + private val size = strToBinary.length + + override def canDrop(statistics: Statistics[Binary]): Boolean = { + val comparator = PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR + val max = statistics.getMax + val min = statistics.getMin + comparator.compare(max.slice(0, math.min(size, max.length)), strToBinary) < 0 || + comparator.compare(min.slice(0, math.min(size, min.length)), strToBinary) > 0 + } + + override def inverseCanDrop(statistics: Statistics[Binary]): Boolean = { + val comparator = PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR + val max = statistics.getMax + val min = statistics.getMin + comparator.compare(max.slice(0, math.min(size, max.length)), strToBinary) == 0 && + comparator.compare(min.slice(0, math.min(size, min.length)), strToBinary) == 0 + } + + override def keep(value: Binary): Boolean = { + value != null && UTF8String + .fromBytes(value.getBytes) + .startsWith(UTF8String.fromBytes(strToBinary.getBytes)) + } + }) + } + + case sources.StringEndsWith(name, suffix) + if pushDownStringPredicate && canMakeFilterOn(name, suffix) => + Option(suffix).map { v => + FilterApi.userDefined( + binaryColumn(nameToParquetField(name).fieldNames), + new UserDefinedPredicate[Binary] with Serializable { + private val suffixStr = UTF8String.fromString(v) + override def canDrop(statistics: Statistics[Binary]): Boolean = false + override def inverseCanDrop(statistics: Statistics[Binary]): Boolean = false + override def keep(value: Binary): Boolean = { + value != null && UTF8String.fromBytes(value.getBytes).endsWith(suffixStr) + } + }) + } + + case sources.StringContains(name, value) + if pushDownStringPredicate && canMakeFilterOn(name, value) => + Option(value).map { v => + FilterApi.userDefined( + binaryColumn(nameToParquetField(name).fieldNames), + new UserDefinedPredicate[Binary] with Serializable { + private val subStr = UTF8String.fromString(v) + override def canDrop(statistics: Statistics[Binary]): Boolean = false + override def inverseCanDrop(statistics: Statistics[Binary]): Boolean = false + override def keep(value: Binary): Boolean = { + value != null && UTF8String.fromBytes(value.getBytes).contains(subStr) + } + }) + } + + case _ => None + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala new file mode 100644 index 0000000000..609026ee0a --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -0,0 +1,1806 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.serde + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average, Count, Final, Max, Min, Partial, Sum} +import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke +import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, SinglePartition} +import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils +import org.apache.spark.sql.comet.{CometHashAggregateExec, CometSinkPlaceHolder, DecimalPrecision} +import org.apache.spark.sql.execution +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import org.apache.comet.CometSparkSessionExtensions.{isCometOperatorEnabled, isCometScan, isSpark32, isSpark34Plus} +import org.apache.comet.serde.ExprOuterClass.{AggExpr, DataType => ProtoDataType, Expr, ScalarFunc} +import org.apache.comet.serde.ExprOuterClass.DataType.{DataTypeInfo, DecimalInfo, ListInfo, StructInfo} +import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, Operator} +import org.apache.comet.shims.ShimQueryPlanSerde + +/** + * An utility object for query plan and expression serialization. + */ +object QueryPlanSerde extends Logging with ShimQueryPlanSerde { + def emitWarning(reason: String): Unit = { + logWarning(s"Comet native execution is disabled due to: $reason") + } + + def supportedDataType(dt: DataType): Boolean = dt match { + case _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | + _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: DecimalType | + _: DateType | _: BooleanType | _: NullType => + true + // `TimestampNTZType` is private in Spark 3.2. + case dt if dt.typeName == "timestamp_ntz" => true + case dt => + emitWarning(s"unsupported Spark data type: $dt") + false + } + + /** + * Serializes Spark datatype to protobuf. Note that, a datatype can be serialized by this method + * doesn't mean it is supported by Comet native execution, i.e., `supportedDataType` may return + * false for it. + */ + def serializeDataType(dt: DataType): Option[ExprOuterClass.DataType] = { + val typeId = dt match { + case _: BooleanType => 0 + case _: ByteType => 1 + case _: ShortType => 2 + case _: IntegerType => 3 + case _: LongType => 4 + case _: FloatType => 5 + case _: DoubleType => 6 + case _: StringType => 7 + case _: BinaryType => 8 + case _: TimestampType => 9 + case _: DecimalType => 10 + case dt if dt.typeName == "timestamp_ntz" => 11 + case _: DateType => 12 + case _: NullType => 13 + case _: ArrayType => 14 + case _: StructType => 15 + case dt => + emitWarning(s"Cannot serialize Spark data type: $dt") + return None + } + + val builder = ProtoDataType.newBuilder() + builder.setTypeIdValue(typeId) + + // Decimal + val dataType = dt match { + case t: DecimalType => + val info = DataTypeInfo.newBuilder() + val decimal = DecimalInfo.newBuilder() + decimal.setPrecision(t.precision) + decimal.setScale(t.scale) + info.setDecimal(decimal) + builder.setTypeInfo(info.build()).build() + + case a: ArrayType => + val elementType = serializeDataType(a.elementType) + + if (elementType.isEmpty) { + return None + } + + val info = DataTypeInfo.newBuilder() + val list = ListInfo.newBuilder() + list.setElementType(elementType.get) + list.setContainsNull(a.containsNull) + + info.setList(list) + builder.setTypeInfo(info.build()).build() + + case s: StructType => + val info = DataTypeInfo.newBuilder() + val struct = StructInfo.newBuilder() + + val fieldNames = s.fields.map(_.name).toIterable.asJava + val fieldDatatypes = s.fields.map(f => serializeDataType(f.dataType)).toSeq + val fieldNullable = s.fields.map(f => Boolean.box(f.nullable)).toIterable.asJava + + if (fieldDatatypes.exists(_.isEmpty)) { + return None + } + + struct.addAllFieldNames(fieldNames) + struct.addAllFieldDatatypes(fieldDatatypes.map(_.get).asJava) + struct.addAllFieldNullable(fieldNullable) + + info.setStruct(struct) + builder.setTypeInfo(info.build()).build() + case _ => builder.build() + } + + Some(dataType) + } + + private def sumDataTypeSupported(dt: DataType): Boolean = { + dt match { + case _: NumericType => true + case _ => false + } + } + + private def avgDataTypeSupported(dt: DataType): Boolean = { + dt match { + case _: NumericType => true + // TODO: implement support for interval types + case _ => false + } + } + + private def minMaxDataTypeSupported(dt: DataType): Boolean = { + dt match { + case _: NumericType | DateType | TimestampType => true + case _ => false + } + } + + def aggExprToProto(aggExpr: AggregateExpression, inputs: Seq[Attribute]): Option[AggExpr] = { + aggExpr.aggregateFunction match { + case s @ Sum(child, _) if sumDataTypeSupported(s.dataType) => + val childExpr = exprToProto(child, inputs) + val dataType = serializeDataType(s.dataType) + + if (childExpr.isDefined && dataType.isDefined) { + val sumBuilder = ExprOuterClass.Sum.newBuilder() + sumBuilder.setChild(childExpr.get) + sumBuilder.setDatatype(dataType.get) + sumBuilder.setFailOnError(getFailOnError(s)) + + Some( + ExprOuterClass.AggExpr + .newBuilder() + .setSum(sumBuilder) + .build()) + } else { + None + } + case s @ Average(child, _) if avgDataTypeSupported(s.dataType) => + val childExpr = exprToProto(child, inputs) + val dataType = serializeDataType(s.dataType) + + val sumDataType = if (child.dataType.isInstanceOf[DecimalType]) { + + // This is input precision + 10 to be consistent with Spark + val precision = Math.min( + DecimalType.MAX_PRECISION, + child.dataType.asInstanceOf[DecimalType].precision + 10) + val newType = + DecimalType.apply(precision, child.dataType.asInstanceOf[DecimalType].scale) + serializeDataType(newType) + } else { + serializeDataType(child.dataType) + } + + if (childExpr.isDefined && dataType.isDefined) { + val builder = ExprOuterClass.Avg.newBuilder() + builder.setChild(childExpr.get) + builder.setDatatype(dataType.get) + builder.setFailOnError(getFailOnError(s)) + builder.setSumDatatype(sumDataType.get) + + Some( + ExprOuterClass.AggExpr + .newBuilder() + .setAvg(builder) + .build()) + } else { + None + } + case Count(children) => + val exprChildren = children.map(exprToProto(_, inputs)) + + if (exprChildren.forall(_.isDefined)) { + val countBuilder = ExprOuterClass.Count.newBuilder() + countBuilder.addAllChildren(exprChildren.map(_.get).asJava) + + Some( + ExprOuterClass.AggExpr + .newBuilder() + .setCount(countBuilder) + .build()) + } else { + None + } + case min @ Min(child) if minMaxDataTypeSupported(min.dataType) => + val childExpr = exprToProto(child, inputs) + val dataType = serializeDataType(min.dataType) + + if (childExpr.isDefined && dataType.isDefined) { + val minBuilder = ExprOuterClass.Min.newBuilder() + minBuilder.setChild(childExpr.get) + minBuilder.setDatatype(dataType.get) + + Some( + ExprOuterClass.AggExpr + .newBuilder() + .setMin(minBuilder) + .build()) + } else { + None + } + case max @ Max(child) if minMaxDataTypeSupported(max.dataType) => + val childExpr = exprToProto(child, inputs) + val dataType = serializeDataType(max.dataType) + + if (childExpr.isDefined && dataType.isDefined) { + val maxBuilder = ExprOuterClass.Max.newBuilder() + maxBuilder.setChild(childExpr.get) + maxBuilder.setDatatype(dataType.get) + + Some( + ExprOuterClass.AggExpr + .newBuilder() + .setMax(maxBuilder) + .build()) + } else { + None + } + + case fn => + emitWarning(s"unsupported Spark aggregate function: $fn") + None + } + } + + def exprToProto(expr: Expression, input: Seq[Attribute]): Option[Expr] = { + val conf = SQLConf.get + val newExpr = + DecimalPrecision.promote(conf.decimalOperationsAllowPrecisionLoss, expr, !conf.ansiEnabled) + exprToProtoInternal(newExpr, input) + } + + def exprToProtoInternal(expr: Expression, inputs: Seq[Attribute]): Option[Expr] = { + SQLConf.get + expr match { + case a @ Alias(_, _) => + exprToProtoInternal(a.child, inputs) + + case cast @ Cast(_: Literal, dataType, _, _) => + // This can happen after promoting decimal precisions + val value = cast.eval() + exprToProtoInternal(Literal(value, dataType), inputs) + + case Cast(child, dt, timeZoneId, _) => + val childExpr = exprToProtoInternal(child, inputs) + val dataType = serializeDataType(dt) + + if (childExpr.isDefined && dataType.isDefined) { + val castBuilder = ExprOuterClass.Cast.newBuilder() + castBuilder.setChild(childExpr.get) + castBuilder.setDatatype(dataType.get) + + val timeZone = timeZoneId.getOrElse("UTC") + castBuilder.setTimezone(timeZone) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setCast(castBuilder) + .build()) + } else { + None + } + + case add @ Add(left, right, _) if supportedDataType(left.dataType) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val addBuilder = ExprOuterClass.Add.newBuilder() + addBuilder.setLeft(leftExpr.get) + addBuilder.setRight(rightExpr.get) + addBuilder.setFailOnError(getFailOnError(add)) + serializeDataType(add.dataType).foreach { t => + addBuilder.setReturnType(t) + } + + Some( + ExprOuterClass.Expr + .newBuilder() + .setAdd(addBuilder) + .build()) + } else { + None + } + + case sub @ Subtract(left, right, _) if supportedDataType(left.dataType) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Subtract.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + builder.setFailOnError(getFailOnError(sub)) + serializeDataType(sub.dataType).foreach { t => + builder.setReturnType(t) + } + + Some( + ExprOuterClass.Expr + .newBuilder() + .setSubtract(builder) + .build()) + } else { + None + } + + case mul @ Multiply(left, right, _) + if supportedDataType(left.dataType) && !decimalBeforeSpark34(left.dataType) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Multiply.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + builder.setFailOnError(getFailOnError(mul)) + serializeDataType(mul.dataType).foreach { t => + builder.setReturnType(t) + } + + Some( + ExprOuterClass.Expr + .newBuilder() + .setMultiply(builder) + .build()) + } else { + None + } + + case div @ Divide(left, right, _) + if supportedDataType(left.dataType) && !decimalBeforeSpark34(left.dataType) => + val leftExpr = exprToProtoInternal(left, inputs) + // Datafusion now throws an exception for dividing by zero + // See https://github.com/apache/arrow-datafusion/pull/6792 + // For now, use NullIf to swap zeros with nulls. + val rightExpr = exprToProtoInternal(nullIfWhenPrimitive(right), inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Divide.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + builder.setFailOnError(getFailOnError(div)) + serializeDataType(div.dataType).foreach { t => + builder.setReturnType(t) + } + + Some( + ExprOuterClass.Expr + .newBuilder() + .setDivide(builder) + .build()) + } else { + None + } + + case rem @ Remainder(left, right, _) + if supportedDataType(left.dataType) && !decimalBeforeSpark34(left.dataType) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(nullIfWhenPrimitive(right), inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Remainder.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + builder.setFailOnError(getFailOnError(rem)) + serializeDataType(rem.dataType).foreach { t => + builder.setReturnType(t) + } + + Some( + ExprOuterClass.Expr + .newBuilder() + .setRemainder(builder) + .build()) + } else { + None + } + + case EqualTo(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Equal.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setEq(builder) + .build()) + } else { + None + } + + case Not(EqualTo(left, right)) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.NotEqual.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setNeq(builder) + .build()) + } else { + None + } + + case EqualNullSafe(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.EqualNullSafe.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setEqNullSafe(builder) + .build()) + } else { + None + } + + case Not(EqualNullSafe(left, right)) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.NotEqualNullSafe.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setNeqNullSafe(builder) + .build()) + } else { + None + } + + case GreaterThan(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.GreaterThan.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setGt(builder) + .build()) + } else { + None + } + + case GreaterThanOrEqual(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.GreaterThanEqual.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setGtEq(builder) + .build()) + } else { + None + } + + case LessThan(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.LessThan.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setLt(builder) + .build()) + } else { + None + } + + case LessThanOrEqual(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.LessThanEqual.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setLtEq(builder) + .build()) + } else { + None + } + + case Literal(value, dataType) if supportedDataType(dataType) => + val exprBuilder = ExprOuterClass.Literal.newBuilder() + + if (value == null) { + exprBuilder.setIsNull(true) + } else { + exprBuilder.setIsNull(false) + dataType match { + case _: BooleanType => exprBuilder.setBoolVal(value.asInstanceOf[Boolean]) + case _: ByteType => exprBuilder.setByteVal(value.asInstanceOf[Byte]) + case _: ShortType => exprBuilder.setShortVal(value.asInstanceOf[Short]) + case _: IntegerType => exprBuilder.setIntVal(value.asInstanceOf[Int]) + case _: LongType => exprBuilder.setLongVal(value.asInstanceOf[Long]) + case _: FloatType => exprBuilder.setFloatVal(value.asInstanceOf[Float]) + case _: DoubleType => exprBuilder.setDoubleVal(value.asInstanceOf[Double]) + case _: StringType => + exprBuilder.setStringVal(value.asInstanceOf[UTF8String].toString) + case _: TimestampType => exprBuilder.setLongVal(value.asInstanceOf[Long]) + case _: DecimalType => + // Pass decimal literal as bytes. + val unscaled = value.asInstanceOf[Decimal].toBigDecimal.underlying.unscaledValue + exprBuilder.setDecimalVal( + com.google.protobuf.ByteString.copyFrom(unscaled.toByteArray)) + case _: BinaryType => + val byteStr = + com.google.protobuf.ByteString.copyFrom(value.asInstanceOf[Array[Byte]]) + exprBuilder.setBytesVal(byteStr) + case _: DateType => exprBuilder.setIntVal(value.asInstanceOf[Int]) + case dt => + logWarning(s"Unexpected date type '$dt' for literal value '$value'") + } + } + + val dt = serializeDataType(dataType) + + if (dt.isDefined) { + exprBuilder.setDatatype(dt.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setLiteral(exprBuilder) + .build()) + } else { + None + } + + case Substring(str, Literal(pos, _), Literal(len, _)) => + val strExpr = exprToProtoInternal(str, inputs) + + if (strExpr.isDefined) { + val builder = ExprOuterClass.Substring.newBuilder() + builder.setChild(strExpr.get) + builder.setStart(pos.asInstanceOf[Int]) + builder.setLen(len.asInstanceOf[Int]) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setSubstring(builder) + .build()) + } else { + None + } + + case Like(left, right, _) => + // TODO escapeChar + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Like.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setLike(builder) + .build()) + } else { + None + } + + // TODO waiting for arrow-rs update +// case RLike(left, right) => +// val leftExpr = exprToProtoInternal(left, inputs) +// val rightExpr = exprToProtoInternal(right, inputs) +// +// if (leftExpr.isDefined && rightExpr.isDefined) { +// val builder = ExprOuterClass.RLike.newBuilder() +// builder.setLeft(leftExpr.get) +// builder.setRight(rightExpr.get) +// +// Some( +// ExprOuterClass.Expr +// .newBuilder() +// .setRlike(builder) +// .build()) +// } else { +// None +// } + + case StartsWith(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.StartsWith.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setStartsWith(builder) + .build()) + } else { + None + } + + case EndsWith(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.EndsWith.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setEndsWith(builder) + .build()) + } else { + None + } + + case Contains(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Contains.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setContains(builder) + .build()) + } else { + None + } + + case StringSpace(child) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val builder = ExprOuterClass.StringSpace.newBuilder() + builder.setChild(childExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setStringSpace(builder) + .build()) + } else { + None + } + + case Hour(child, timeZoneId) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val builder = ExprOuterClass.Hour.newBuilder() + builder.setChild(childExpr.get) + + val timeZone = timeZoneId.getOrElse("UTC") + builder.setTimezone(timeZone) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setHour(builder) + .build()) + } else { + None + } + + case Minute(child, timeZoneId) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val builder = ExprOuterClass.Minute.newBuilder() + builder.setChild(childExpr.get) + + val timeZone = timeZoneId.getOrElse("UTC") + builder.setTimezone(timeZone) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setMinute(builder) + .build()) + } else { + None + } + + case TruncDate(child, format) => + val childExpr = exprToProtoInternal(child, inputs) + val formatExpr = exprToProtoInternal(format, inputs) + + if (childExpr.isDefined && formatExpr.isDefined) { + val builder = ExprOuterClass.TruncDate.newBuilder() + builder.setChild(childExpr.get) + builder.setFormat(formatExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setTruncDate(builder) + .build()) + } else { + None + } + + case TruncTimestamp(format, child, timeZoneId) => + val childExpr = exprToProtoInternal(child, inputs) + val formatExpr = exprToProtoInternal(format, inputs) + + if (childExpr.isDefined && formatExpr.isDefined) { + val builder = ExprOuterClass.TruncTimestamp.newBuilder() + builder.setChild(childExpr.get) + builder.setFormat(formatExpr.get) + + val timeZone = timeZoneId.getOrElse("UTC") + builder.setTimezone(timeZone) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setTruncTimestamp(builder) + .build()) + } else { + None + } + + case Second(child, timeZoneId) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val builder = ExprOuterClass.Second.newBuilder() + builder.setChild(childExpr.get) + + val timeZone = timeZoneId.getOrElse("UTC") + builder.setTimezone(timeZone) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setSecond(builder) + .build()) + } else { + None + } + + case Year(child) => + val periodType = exprToProtoInternal(Literal("year"), inputs) + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("datepart", Seq(periodType, childExpr): _*) + .map(e => { + Expr + .newBuilder() + .setCast( + ExprOuterClass.Cast + .newBuilder() + .setChild(e) + .setDatatype(serializeDataType(IntegerType).get) + .build()) + .build() + }) + + case IsNull(child) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val castBuilder = ExprOuterClass.IsNull.newBuilder() + castBuilder.setChild(childExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setIsNull(castBuilder) + .build()) + } else { + None + } + + case IsNotNull(child) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val castBuilder = ExprOuterClass.IsNotNull.newBuilder() + castBuilder.setChild(childExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setIsNotNull(castBuilder) + .build()) + } else { + None + } + + case SortOrder(child, direction, nullOrdering, _) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val sortOrderBuilder = ExprOuterClass.SortOrder.newBuilder() + sortOrderBuilder.setChild(childExpr.get) + + direction match { + case Ascending => sortOrderBuilder.setDirectionValue(0) + case Descending => sortOrderBuilder.setDirectionValue(1) + } + + nullOrdering match { + case NullsFirst => sortOrderBuilder.setNullOrderingValue(0) + case NullsLast => sortOrderBuilder.setNullOrderingValue(1) + } + + Some( + ExprOuterClass.Expr + .newBuilder() + .setSortOrder(sortOrderBuilder) + .build()) + } else { + None + } + + case And(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.And.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setAnd(builder) + .build()) + } else { + None + } + + case Or(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Or.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setOr(builder) + .build()) + } else { + None + } + + case UnaryExpression(child) if expr.prettyName == "promote_precision" => + // `UnaryExpression` includes `PromotePrecision` for Spark 3.2 & 3.3 + // `PromotePrecision` is just a wrapper, don't need to serialize it. + exprToProtoInternal(child, inputs) + + case CheckOverflow(child, dt, nullOnOverflow) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val builder = ExprOuterClass.CheckOverflow.newBuilder() + builder.setChild(childExpr.get) + builder.setFailOnError(!nullOnOverflow) + + // `dataType` must be decimal type + val dataType = serializeDataType(dt) + builder.setDatatype(dataType.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setCheckOverflow(builder) + .build()) + } else { + None + } + + case attr: AttributeReference => + val dataType = serializeDataType(attr.dataType) + + if (dataType.isDefined) { + val boundRef = BindReferences + .bindReference(attr, inputs, allowFailures = false) + .asInstanceOf[BoundReference] + val boundExpr = ExprOuterClass.BoundReference + .newBuilder() + .setIndex(boundRef.ordinal) + .setDatatype(dataType.get) + .build() + + Some( + ExprOuterClass.Expr + .newBuilder() + .setBound(boundExpr) + .build()) + } else { + None + } + + case Abs(child, _) => + exprToProtoInternal(child, inputs).map(childExpr => { + val abs = + ExprOuterClass.Abs + .newBuilder() + .setChild(childExpr) + .build() + Expr.newBuilder().setAbs(abs).build() + }) + + case Acos(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("acos", childExpr) + + case Asin(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("asin", childExpr) + + case Atan(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("atan", childExpr) + + case Atan2(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + scalarExprToProto("atan2", leftExpr, rightExpr) + + case e @ Ceil(child) => + val childExpr = exprToProtoInternal(child, inputs) + child.dataType match { + case t: DecimalType if t.scale == 0 => // zero scale is no-op + childExpr + case t: DecimalType if t.scale < 0 => // Spark disallows negative scale SPARK-30252 + None + case _ => + scalarExprToProtoWithReturnType("ceil", e.dataType, childExpr) + } + + case Cos(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("cos", childExpr) + + case Exp(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("exp", childExpr) + + case e @ Floor(child) => + val childExpr = exprToProtoInternal(child, inputs) + child.dataType match { + case t: DecimalType if t.scale == 0 => // zero scale is no-op + childExpr + case t: DecimalType if t.scale < 0 => // Spark disallows negative scale SPARK-30252 + None + case _ => + scalarExprToProtoWithReturnType("floor", e.dataType, childExpr) + } + + case Log(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("ln", childExpr) + + case Log10(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("log10", childExpr) + + case Log2(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("log2", childExpr) + + case Pow(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + scalarExprToProto("pow", leftExpr, rightExpr) + + // round function for Spark 3.2 does not allow negative round target scale. In addition, + // it has different result precision/scale for decimals. Supporting only 3.3 and above. + case r: Round if !isSpark32 => + // _scale s a constant, copied from Spark's RoundBase because it is a protected val + val scaleV: Any = r.scale.eval(EmptyRow) + val _scale: Int = scaleV.asInstanceOf[Int] + + lazy val childExpr = exprToProtoInternal(r.child, inputs) + r.child.dataType match { + case t: DecimalType if t.scale < 0 => // Spark disallows negative scale SPARK-30252 + None + case _ if scaleV == null => + exprToProtoInternal(Literal(null), inputs) + case _: ByteType | ShortType | IntegerType | LongType if _scale >= 0 => + childExpr // _scale(I.e. decimal place) >= 0 is a no-op for integer types in Spark + case _: FloatType | DoubleType => + // We cannot properly match with the Spark behavior for floating-point numbers. + // Spark uses BigDecimal for rounding float/double, and BigDecimal fist converts a + // double to string internally in order to create its own internal representation. + // The problem is BigDecimal uses java.lang.Double.toString() and it has complicated + // rounding algorithm. E.g. -5.81855622136895E8 is actually + // -581855622.13689494132995605468750. Note the 5th fractional digit is 4 instead of + // 5. Java(Scala)'s toString() rounds it up to -581855622.136895. This makes a + // difference when rounding at 5th digit, I.e. round(-5.81855622136895E8, 5) should be + // -5.818556221369E8, instead of -5.8185562213689E8. There is also an example that + // toString() does NOT round up. 6.1317116247283497E18 is 6131711624728349696. It can + // be rounded up to 6.13171162472835E18 that still represents the same double number. + // I.e. 6.13171162472835E18 == 6.1317116247283497E18. However, toString() does not. + // That results in round(6.1317116247283497E18, -5) == 6.1317116247282995E18 instead + // of 6.1317116247283999E18. + None + case _ => + // `scale` must be Int64 type in DataFusion + val scaleExpr = exprToProtoInternal(Literal(_scale.toLong, LongType), inputs) + scalarExprToProtoWithReturnType("round", r.dataType, childExpr, scaleExpr) + } + + case Signum(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("signum", childExpr) + + case Sin(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("sin", childExpr) + + case Sqrt(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("sqrt", childExpr) + + case Tan(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("tan", childExpr) + + case Ascii(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("ascii", childExpr) + + case BitLength(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("bit_length", childExpr) + + case If(predicate, trueValue, falseValue) => + val predicateExpr = exprToProtoInternal(predicate, inputs) + val trueExpr = exprToProtoInternal(trueValue, inputs) + val falseExpr = exprToProtoInternal(falseValue, inputs) + if (predicateExpr.isDefined && trueExpr.isDefined && falseExpr.isDefined) { + val builder = ExprOuterClass.IfExpr.newBuilder() + builder.setIfExpr(predicateExpr.get) + builder.setTrueExpr(trueExpr.get) + builder.setFalseExpr(falseExpr.get) + Some( + ExprOuterClass.Expr + .newBuilder() + .setIf(builder) + .build()) + } else { + None + } + + case CaseWhen(branches, elseValue) => + val whenSeq = branches.map(elements => exprToProtoInternal(elements._1, inputs)) + val thenSeq = branches.map(elements => exprToProtoInternal(elements._2, inputs)) + assert(whenSeq.length == thenSeq.length) + if (whenSeq.forall(_.isDefined) && thenSeq.forall(_.isDefined)) { + val builder = ExprOuterClass.CaseWhen.newBuilder() + builder.addAllWhen(whenSeq.map(_.get).asJava) + builder.addAllThen(thenSeq.map(_.get).asJava) + if (elseValue.isDefined) { + val elseValueExpr = exprToProtoInternal(elseValue.get, inputs) + if (elseValueExpr.isDefined) { + builder.setElseExpr(elseValueExpr.get) + } else { + return None + } + } + Some( + ExprOuterClass.Expr + .newBuilder() + .setCaseWhen(builder) + .build()) + } else { + None + } + + case ConcatWs(children) => + val exprs = children.map(e => exprToProtoInternal(Cast(e, StringType), inputs)) + scalarExprToProto("concat_ws", exprs: _*) + + case Chr(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProto("chr", childExpr) + + case InitCap(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("initcap", childExpr) + + case Length(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("length", childExpr) + + case Lower(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("lower", childExpr) + + case Md5(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("md5", childExpr) + + case OctetLength(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("octet_length", childExpr) + + case Reverse(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("reverse", childExpr) + + case StringInstr(str, substr) => + val leftExpr = exprToProtoInternal(Cast(str, StringType), inputs) + val rightExpr = exprToProtoInternal(Cast(substr, StringType), inputs) + scalarExprToProto("strpos", leftExpr, rightExpr) + + case StringRepeat(str, times) => + val leftExpr = exprToProtoInternal(Cast(str, StringType), inputs) + val rightExpr = exprToProtoInternal(Cast(times, LongType), inputs) + scalarExprToProto("repeat", leftExpr, rightExpr) + + case StringReplace(src, search, replace) => + val srcExpr = exprToProtoInternal(Cast(src, StringType), inputs) + val searchExpr = exprToProtoInternal(Cast(search, StringType), inputs) + val replaceExpr = exprToProtoInternal(Cast(replace, StringType), inputs) + scalarExprToProto("replace", srcExpr, searchExpr, replaceExpr) + + case StringTranslate(src, matching, replace) => + val srcExpr = exprToProtoInternal(Cast(src, StringType), inputs) + val matchingExpr = exprToProtoInternal(Cast(matching, StringType), inputs) + val replaceExpr = exprToProtoInternal(Cast(replace, StringType), inputs) + scalarExprToProto("translate", srcExpr, matchingExpr, replaceExpr) + + case StringTrim(srcStr, trimStr) => + trim(srcStr, trimStr, inputs, "trim") + + case StringTrimLeft(srcStr, trimStr) => + trim(srcStr, trimStr, inputs, "ltrim") + + case StringTrimRight(srcStr, trimStr) => + trim(srcStr, trimStr, inputs, "rtrim") + + case StringTrimBoth(srcStr, trimStr, _) => + trim(srcStr, trimStr, inputs, "btrim") + + case Upper(child) => + val childExpr = exprToProtoInternal(Cast(child, StringType), inputs) + scalarExprToProto("upper", childExpr) + + case BitwiseAnd(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.BitwiseAnd.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setBitwiseAnd(builder) + .build()) + } else { + None + } + + case BitwiseNot(child) => + val childExpr = exprToProtoInternal(child, inputs) + + if (childExpr.isDefined) { + val builder = ExprOuterClass.BitwiseNot.newBuilder() + builder.setChild(childExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setBitwiseNot(builder) + .build()) + } else { + None + } + + case BitwiseOr(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.BitwiseOr.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setBitwiseOr(builder) + .build()) + } else { + None + } + + case BitwiseXor(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.BitwiseXor.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setBitwiseXor(builder) + .build()) + } else { + None + } + + case ShiftRight(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.BitwiseShiftRight.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setBitwiseShiftRight(builder) + .build()) + } else { + None + } + + case ShiftLeft(left, right) => + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) + + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.BitwiseShiftLeft.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) + + Some( + ExprOuterClass.Expr + .newBuilder() + .setBitwiseShiftLeft(builder) + .build()) + } else { + None + } + + case In(value, list) => + in(value, list, inputs, false) + + case Not(In(value, list)) => + in(value, list, inputs, true) + + case Not(child) => + val childExpr = exprToProtoInternal(child, inputs) + if (childExpr.isDefined) { + val builder = ExprOuterClass.Not.newBuilder() + builder.setChild(childExpr.get) + Some( + ExprOuterClass.Expr + .newBuilder() + .setNot(builder) + .build()) + } else { + None + } + + case UnaryMinus(child, _) => + val childExpr = exprToProtoInternal(child, inputs) + if (childExpr.isDefined) { + val builder = ExprOuterClass.Negative.newBuilder() + builder.setChild(childExpr.get) + Some( + ExprOuterClass.Expr + .newBuilder() + .setNegative(builder) + .build()) + } else { + None + } + + case a @ Coalesce(_) => + val exprChildren = a.children.map(exprToProtoInternal(_, inputs)) + scalarExprToProto("coalesce", exprChildren: _*) + + // With Spark 3.4, CharVarcharCodegenUtils.readSidePadding gets called to pad spaces for char + // types. Use rpad to achieve the behavior. See https://github.com/apache/spark/pull/38151 + case StaticInvoke( + _: Class[CharVarcharCodegenUtils], + _: StringType, + "readSidePadding", + arguments, + _, + true, + false, + true) if arguments.size == 2 => + val argsExpr = Seq( + exprToProtoInternal(Cast(arguments(0), StringType), inputs), + exprToProtoInternal(arguments(1), inputs)) + + if (argsExpr.forall(_.isDefined)) { + val builder = ExprOuterClass.ScalarFunc.newBuilder() + builder.setFunc("rpad") + argsExpr.foreach(arg => builder.addArgs(arg.get)) + + Some(ExprOuterClass.Expr.newBuilder().setScalarFunc(builder).build()) + } else { + None + } + + case KnownFloatingPointNormalized(NormalizeNaNAndZero(expr)) => + val dataType = serializeDataType(expr.dataType) + if (dataType.isEmpty) { + return None + } + exprToProtoInternal(expr, inputs).map { child => + val builder = ExprOuterClass.NormalizeNaNAndZero + .newBuilder() + .setChild(child) + .setDatatype(dataType.get) + ExprOuterClass.Expr.newBuilder().setNormalizeNanAndZero(builder).build() + } + + case s @ execution.ScalarSubquery(_, _) => + val dataType = serializeDataType(s.dataType) + if (dataType.isEmpty) { + return None + } + + val builder = ExprOuterClass.Subquery + .newBuilder() + .setId(s.exprId.id) + .setDatatype(dataType.get) + Some(ExprOuterClass.Expr.newBuilder().setSubquery(builder).build()) + + case UnscaledValue(child) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProtoWithReturnType("unscaled_value", LongType, childExpr) + + case MakeDecimal(child, precision, scale, true) => + val childExpr = exprToProtoInternal(child, inputs) + scalarExprToProtoWithReturnType("make_decimal", DecimalType(precision, scale), childExpr) + + case e => + emitWarning(s"unsupported Spark expression: '$e' of class '${e.getClass.getName}") + None + } + } + + private def trim( + srcStr: Expression, + trimStr: Option[Expression], + inputs: Seq[Attribute], + trimType: String): Option[Expr] = { + val srcExpr = exprToProtoInternal(Cast(srcStr, StringType), inputs) + if (trimStr.isDefined) { + val trimExpr = exprToProtoInternal(Cast(trimStr.get, StringType), inputs) + scalarExprToProto(trimType, srcExpr, trimExpr) + } else { + scalarExprToProto(trimType, srcExpr) + } + } + + private def in( + value: Expression, + list: Seq[Expression], + inputs: Seq[Attribute], + negate: Boolean): Option[Expr] = { + val valueExpr = exprToProtoInternal(value, inputs) + val listExprs = list.map(exprToProtoInternal(_, inputs)) + if (valueExpr.isDefined && listExprs.forall(_.isDefined)) { + val builder = ExprOuterClass.In.newBuilder() + builder.setInValue(valueExpr.get) + builder.addAllLists(listExprs.map(_.get).asJava) + builder.setNegated(negate) + Some( + ExprOuterClass.Expr + .newBuilder() + .setIn(builder) + .build()) + } else { + None + } + } + + def scalarExprToProtoWithReturnType( + funcName: String, + returnType: DataType, + args: Option[Expr]*): Option[Expr] = { + val builder = ExprOuterClass.ScalarFunc.newBuilder() + builder.setFunc(funcName) + serializeDataType(returnType).flatMap { t => + builder.setReturnType(t) + scalarExprToProto0(builder, args: _*) + } + } + + def scalarExprToProto(funcName: String, args: Option[Expr]*): Option[Expr] = { + val builder = ExprOuterClass.ScalarFunc.newBuilder() + builder.setFunc(funcName) + scalarExprToProto0(builder, args: _*) + } + + private def scalarExprToProto0( + builder: ScalarFunc.Builder, + args: Option[Expr]*): Option[Expr] = { + args.foreach { + case Some(a) => builder.addArgs(a) + case _ => return None + } + Some(ExprOuterClass.Expr.newBuilder().setScalarFunc(builder).build()) + } + + def isPrimitive(expression: Expression): Boolean = expression.dataType match { + case _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | + _: DoubleType | _: TimestampType | _: DateType | _: BooleanType | _: DecimalType => + true + case _ => false + } + + def nullIfWhenPrimitive(expression: Expression): Expression = if (isPrimitive(expression)) { + new NullIf(expression, Literal.default(expression.dataType)).child + } else { + expression + } + + /** + * Convert a Spark plan operator to a protobuf Comet operator. + * + * @param op + * Spark plan operator + * @param childOp + * previously converted protobuf Comet operators, which will be consumed by the Spark plan + * operator as its children + * @return + * The converted Comet native operator for the input `op`, or `None` if the `op` cannot be + * converted to a native operator. + */ + def operator2Proto(op: SparkPlan, childOp: Operator*): Option[Operator] = { + val result = OperatorOuterClass.Operator.newBuilder() + childOp.foreach(result.addChildren) + + op match { + case ProjectExec(projectList, child) if isCometOperatorEnabled(op.conf, "project") => + val exprs = projectList.map(exprToProto(_, child.output)) + + if (exprs.forall(_.isDefined) && childOp.nonEmpty) { + val projectBuilder = OperatorOuterClass.Projection + .newBuilder() + .addAllProjectList(exprs.map(_.get).asJava) + Some(result.setProjection(projectBuilder).build()) + } else { + None + } + + case FilterExec(condition, child) if isCometOperatorEnabled(op.conf, "filter") => + val cond = exprToProto(condition, child.output) + + if (cond.isDefined && childOp.nonEmpty) { + val filterBuilder = OperatorOuterClass.Filter.newBuilder().setPredicate(cond.get) + Some(result.setFilter(filterBuilder).build()) + } else { + None + } + + case SortExec(sortOrder, _, child, _) if isCometOperatorEnabled(op.conf, "sort") => + val sortOrders = sortOrder.map(exprToProto(_, child.output)) + + if (sortOrders.forall(_.isDefined) && childOp.nonEmpty) { + val sortBuilder = OperatorOuterClass.Sort + .newBuilder() + .addAllSortOrders(sortOrders.map(_.get).asJava) + Some(result.setSort(sortBuilder).build()) + } else { + None + } + + case LocalLimitExec(limit, _) if isCometOperatorEnabled(op.conf, "local_limit") => + if (childOp.nonEmpty) { + // LocalLimit doesn't use offset, but it shares same operator serde class. + // Just set it to zero. + val limitBuilder = OperatorOuterClass.Limit + .newBuilder() + .setLimit(limit) + .setOffset(0) + Some(result.setLimit(limitBuilder).build()) + } else { + None + } + + case globalLimitExec: GlobalLimitExec if isCometOperatorEnabled(op.conf, "global_limit") => + if (childOp.nonEmpty) { + val limitBuilder = OperatorOuterClass.Limit.newBuilder() + + // Spark 3.2 doesn't support offset for GlobalLimit, but newer Spark versions + // support it. Before we upgrade to Spark 3.3, just set it zero. + // TODO: Spark 3.3 might have negative limit (-1) for Offset usage. + // When we upgrade to Spark 3.3., we need to address it here. + assert(globalLimitExec.limit >= 0, "limit should be greater or equal to zero") + limitBuilder.setLimit(globalLimitExec.limit) + limitBuilder.setOffset(0) + + Some(result.setLimit(limitBuilder).build()) + } else { + None + } + + case ExpandExec(projections, _, child) if isCometOperatorEnabled(op.conf, "expand") => + val projExprs = projections.flatMap(_.map(exprToProto(_, child.output))) + + if (projExprs.forall(_.isDefined) && childOp.nonEmpty) { + val expandBuilder = OperatorOuterClass.Expand + .newBuilder() + .addAllProjectList(projExprs.map(_.get).asJava) + .setNumExprPerProject(projections.head.size) + Some(result.setExpand(expandBuilder).build()) + } else { + None + } + + case HashAggregateExec( + _, + _, + _, + groupingExpressions, + aggregateExpressions, + _, + _, + resultExpressions, + child) if isCometOperatorEnabled(op.conf, "aggregate") => + val modes = aggregateExpressions.map(_.mode).distinct + + if (modes.size != 1) { + // This shouldn't happen as all aggregation expressions should share the same mode. + // Fallback to Spark nevertheless here. + return None + } + + val mode = modes.head match { + case Partial => CometAggregateMode.Partial + case Final => CometAggregateMode.Final + case _ => return None + } + + val output = mode match { + case CometAggregateMode.Partial => child.output + case CometAggregateMode.Final => + // Assuming `Final` always follows `Partial` aggregation, this find the first + // `Partial` aggregation and get the input attributes from it. + child.collectFirst { case CometHashAggregateExec(_, _, _, _, input, Partial, _) => + input + } match { + case Some(input) => input + case _ => return None + } + case _ => return None + } + + val aggExprs = aggregateExpressions.map(aggExprToProto(_, output)) + val groupingExprs = groupingExpressions.map(exprToProto(_, child.output)) + + if (childOp.nonEmpty && groupingExprs.forall(_.isDefined) && + aggExprs.forall(_.isDefined)) { + val hashAggBuilder = OperatorOuterClass.HashAggregate.newBuilder() + hashAggBuilder.addAllGroupingExprs(groupingExprs.map(_.get).asJava) + hashAggBuilder.addAllAggExprs(aggExprs.map(_.get).asJava) + if (mode == CometAggregateMode.Final) { + val attributes = groupingExpressions.map(_.toAttribute) ++ + aggregateExpressions.map(_.resultAttribute) + val resultExprs = resultExpressions.map(exprToProto(_, attributes)) + if (resultExprs.exists(_.isEmpty)) { + emitWarning(s"Unsupported result expressions found in: ${resultExpressions}") + return None + } + hashAggBuilder.addAllResultExprs(resultExprs.map(_.get).asJava) + } + hashAggBuilder.setModeValue(mode.getNumber) + Some(result.setHashAgg(hashAggBuilder).build()) + } else { + None + } + + case op if isCometSink(op) => + // These operators are source of Comet native execution chain + val scanBuilder = OperatorOuterClass.Scan.newBuilder() + + val scanTypes = op.output.flatten { attr => + serializeDataType(attr.dataType) + } + + if (scanTypes.length == op.output.length) { + scanBuilder.addAllFields(scanTypes.asJava) + Some(result.setScan(scanBuilder).build()) + } else { + // There are unsupported scan type + emitWarning( + s"unsupported Comet operator: ${op.nodeName}, due to unsupported data types above") + None + } + + case op => + // Emit warning if: + // 1. it is not Spark shuffle operator, which is handled separately + // 2. it is not a Comet operator + if (!op.nodeName.contains("Comet") && !op.isInstanceOf[ShuffleExchangeExec]) { + emitWarning(s"unsupported Spark operator: ${op.nodeName}") + } + None + } + } + + /** + * Whether the input Spark operator `op` can be considered as a Comet sink, i.e., the start of + * native execution. If it is true, we'll wrap `op` with `CometScanWrapper` or + * `CometSinkPlaceHolder` later in `CometSparkSessionExtensions` after `operator2proto` is + * called. + */ + private def isCometSink(op: SparkPlan): Boolean = { + op match { + case s if isCometScan(s) => true + case _: CometSinkPlaceHolder => true + case _: CoalesceExec => true + case _: UnionExec => true + case _: ShuffleExchangeExec => true + case _: TakeOrderedAndProjectExec => true + case _ => false + } + } + + /** + * Checks whether `dt` is a decimal type AND whether Spark version is before 3.4 + */ + private def decimalBeforeSpark34(dt: DataType): Boolean = { + !isSpark34Plus && (dt match { + case _: DecimalType => true + case _ => false + }) + } + + /** + * Check if the datatypes of shuffle input are supported. This is used for Columnar shuffle + * which supports struct/array. + */ + def supportPartitioningTypes(inputs: Seq[Attribute]): Boolean = { + def supportedDataType(dt: DataType): Boolean = dt match { + case _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | + _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: DecimalType | + _: DateType | _: BooleanType => + true + case StructType(fields) => + fields.forall(f => supportedDataType(f.dataType)) + case ArrayType(ArrayType(_, _), _) => false // TODO: nested array is not supported + case ArrayType(elementType, _) => + supportedDataType(elementType) + case _ => + false + } + + // Check if the datatypes of shuffle input are supported. + val supported = inputs.forall(attr => supportedDataType(attr.dataType)) + if (!supported) { + emitWarning(s"unsupported Spark partitioning: ${inputs.map(_.dataType)}") + } + supported + } + + /** + * Whether the given Spark partitioning is supported by Comet. + */ + def supportPartitioning(inputs: Seq[Attribute], partitioning: Partitioning): Boolean = { + def supportedDataType(dt: DataType): Boolean = dt match { + case _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | + _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: DecimalType | + _: DateType | _: BooleanType => + true + case _ => + // Native shuffle doesn't support struct/array yet + false + } + + // Check if the datatypes of shuffle input are supported. + val supported = inputs.forall(attr => supportedDataType(attr.dataType)) + + if (!supported) { + emitWarning(s"unsupported Spark partitioning: ${inputs.map(_.dataType)}") + false + } else { + partitioning match { + case HashPartitioning(expressions, _) => + expressions.map(QueryPlanSerde.exprToProto(_, inputs)).forall(_.isDefined) + case SinglePartition => true + case other => + emitWarning(s"unsupported Spark partitioning: ${other.getClass.getName}") + false + } + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/shims/ShimCometBatchScanExec.scala b/spark/src/main/scala/org/apache/comet/shims/ShimCometBatchScanExec.scala new file mode 100644 index 0000000000..9e7cc3ba5c --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/shims/ShimCometBatchScanExec.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.shims + +import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + +trait ShimCometBatchScanExec { + def wrapped: BatchScanExec + + // Only for Spark 3.3+ + def keyGroupedPartitioning: Option[Seq[Expression]] = wrapped.getClass.getDeclaredMethods + .filter(_.getName == "keyGroupedPartitioning") + .flatMap(_.invoke(wrapped).asInstanceOf[Option[Seq[Expression]]]) + .headOption + + // Only for Spark 3.3+ + def inputPartitions: Seq[InputPartition] = wrapped.getClass.getDeclaredMethods + .filter(_.getName == "inputPartitions") + .flatMap(_.invoke(wrapped).asInstanceOf[Seq[InputPartition]]) + + // Only for Spark 3.4+ + def ordering: Option[Seq[SortOrder]] = wrapped.getClass.getDeclaredMethods + .filter(_.getName == "ordering") + .flatMap(_.invoke(wrapped).asInstanceOf[Option[Seq[SortOrder]]]) + .headOption +} diff --git a/spark/src/main/scala/org/apache/comet/shims/ShimCometScanExec.scala b/spark/src/main/scala/org/apache/comet/shims/ShimCometScanExec.scala new file mode 100644 index 0000000000..18871b814c --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/shims/ShimCometScanExec.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.shims + +import scala.language.implicitConversions + +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile} +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions +import org.apache.spark.sql.execution.datasources.v2.DataSourceRDD +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.{LongType, StructField, StructType} + +trait ShimCometScanExec { + def wrapped: FileSourceScanExec + + // TODO: remove after dropping Spark 3.2 support and directly call wrapped.metadataColumns + lazy val metadataColumns: Seq[AttributeReference] = wrapped.getClass.getDeclaredMethods + .filter(_.getName == "metadataColumns") + .map { a => a.setAccessible(true); a } + .flatMap(_.invoke(wrapped).asInstanceOf[Seq[AttributeReference]]) + + // TODO: remove after dropping Spark 3.2 and 3.3 support and directly call + // wrapped.fileConstantMetadataColumns + lazy val fileConstantMetadataColumns: Seq[AttributeReference] = + wrapped.getClass.getDeclaredMethods + .filter(_.getName == "fileConstantMetadataColumns") + .map { a => a.setAccessible(true); a } + .flatMap(_.invoke(wrapped).asInstanceOf[Seq[AttributeReference]]) + + // TODO: remove after dropping Spark 3.2 support and directly call new DataSourceRDD + protected def newDataSourceRDD( + sc: SparkContext, + inputPartitions: Seq[Seq[InputPartition]], + partitionReaderFactory: PartitionReaderFactory, + columnarReads: Boolean, + customMetrics: Map[String, SQLMetric]): DataSourceRDD = { + implicit def flattenSeq(p: Seq[Seq[InputPartition]]): Seq[InputPartition] = p.flatten + new DataSourceRDD(sc, inputPartitions, partitionReaderFactory, columnarReads, customMetrics) + } + + // TODO: remove after dropping Spark 3.2 support and directly call new FileScanRDD + protected def newFileScanRDD( + sparkSession: SparkSession, + readFunction: PartitionedFile => Iterator[InternalRow], + filePartitions: Seq[FilePartition], + readSchema: StructType, + options: ParquetOptions): FileScanRDD = + classOf[FileScanRDD].getDeclaredConstructors + .map { c => + c.getParameterCount match { + case 3 => c.newInstance(sparkSession, readFunction, filePartitions) + case 5 => + c.newInstance(sparkSession, readFunction, filePartitions, readSchema, metadataColumns) + case 6 => + c.newInstance( + sparkSession, + readFunction, + filePartitions, + readSchema, + fileConstantMetadataColumns, + options) + } + } + .last + .asInstanceOf[FileScanRDD] + + // TODO: remove after dropping Spark 3.2 and 3.3 support and directly call + // QueryExecutionErrors.SparkException + protected def invalidBucketFile(path: String, sparkVersion: String): Throwable = { + if (sparkVersion >= "3.3") { + val messageParameters = if (sparkVersion >= "3.4") Map("path" -> path) else Array(path) + classOf[SparkException].getDeclaredConstructors + .filter(_.getParameterCount == 3) + .map(_.newInstance("INVALID_BUCKET_FILE", messageParameters, null)) + .last + .asInstanceOf[SparkException] + } else { // Spark 3.2 + new IllegalStateException(s"Invalid bucket file ${path}") + } + } + + // Copied from Spark 3.4 RowIndexUtil due to PARQUET-2161 (tracked in SPARK-39634) + // TODO: remove after PARQUET-2161 becomes available in Parquet + private def findRowIndexColumnIndexInSchema(sparkSchema: StructType): Int = { + sparkSchema.fields.zipWithIndex.find { case (field: StructField, _: Int) => + field.name == ShimFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME + } match { + case Some((field: StructField, idx: Int)) => + if (field.dataType != LongType) { + throw new RuntimeException( + s"${ShimFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME} must be of LongType") + } + idx + case _ => -1 + } + } + + protected def isNeededForSchema(sparkSchema: StructType): Boolean = { + findRowIndexColumnIndexInSchema(sparkSchema) >= 0 + } +} diff --git a/spark/src/main/scala/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala new file mode 100644 index 0000000000..8afed84ff5 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.shims + +import org.apache.spark.sql.connector.expressions.aggregate.Aggregation +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan + +trait ShimCometSparkSessionExtensions { + + /** + * TODO: delete after dropping Spark 3.2.0 support and directly call scan.pushedAggregate + */ + def getPushedAggregate(scan: ParquetScan): Option[Aggregation] = scan.getClass.getDeclaredFields + .filter(_.getName == "pushedAggregate") + .map { a => a.setAccessible(true); a } + .flatMap(_.get(scan).asInstanceOf[Option[Aggregation]]) + .headOption +} diff --git a/spark/src/main/scala/org/apache/comet/shims/ShimQueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/shims/ShimQueryPlanSerde.scala new file mode 100644 index 0000000000..c47b399cf3 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/shims/ShimQueryPlanSerde.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.shims + +import org.apache.spark.sql.catalyst.expressions.BinaryArithmetic +import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate + +trait ShimQueryPlanSerde { + def getFailOnError(b: BinaryArithmetic): Boolean = + b.getClass.getMethod("failOnError").invoke(b).asInstanceOf[Boolean] + + def getFailOnError(aggregate: DeclarativeAggregate): Boolean = { + val failOnError = aggregate.getClass.getDeclaredMethods.flatMap(m => + m.getName match { + case "failOnError" | "useAnsiAdd" => Some(m.invoke(aggregate).asInstanceOf[Boolean]) + case _ => None + }) + if (failOnError.isEmpty) { + aggregate.getClass.getDeclaredMethods + .flatMap(m => + m.getName match { + case "initQueryContext" => Some(m.invoke(aggregate).asInstanceOf[Option[_]].isDefined) + case _ => None + }) + .head + } else { + failOnError.head + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/shims/ShimSQLConf.scala b/spark/src/main/scala/org/apache/comet/shims/ShimSQLConf.scala new file mode 100644 index 0000000000..ff60ef9645 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/shims/ShimSQLConf.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.shims + +import org.apache.spark.sql.internal.SQLConf + +trait ShimSQLConf { + + /** + * Spark 3.4 renamed parquetFilterPushDownStringStartWith to + * parquetFilterPushDownStringPredicate + * + * TODO: delete after dropping Spark 3.2 & 3.3 support and simply use + * parquetFilterPushDownStringPredicate + */ + protected def getPushDownStringPredicate(sqlConf: SQLConf): Boolean = + sqlConf.getClass.getMethods + .flatMap(m => + m.getName match { + case "parquetFilterPushDownStringStartWith" | "parquetFilterPushDownStringPredicate" => + Some(m.invoke(sqlConf).asInstanceOf[Boolean]) + case _ => None + }) + .head +} diff --git a/spark/src/main/scala/org/apache/spark/Plugins.scala b/spark/src/main/scala/org/apache/spark/Plugins.scala new file mode 100644 index 0000000000..113e3a47ef --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/Plugins.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import java.{util => ju} +import java.util.Collections + +import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} +import org.apache.spark.resource.ResourceProfile + +import org.apache.comet.{CometConf, CometSparkSessionExtensions} + +/** + * Comet driver plugin. This class is loaded by Spark's plugin framework. It will be instantiated + * on driver side only. It will update the SparkConf with the extra configuration provided by + * Comet, e.g., Comet memory configurations. + * + * Note that `SparkContext.conf` is spark package only. So this plugin must be in spark package. + * Although `SparkContext.getConf` is public, it returns a copy of the SparkConf, so it cannot + * actually change Spark configs at runtime. + * + * To enable this plugin, set the config "spark.plugins" to `org.apache.spark.CometPlugin`. + */ +class CometDriverPlugin extends DriverPlugin with Logging { + import CometDriverPlugin._ + + override def init(sc: SparkContext, pluginContext: PluginContext): ju.Map[String, String] = { + logInfo("CometDriverPlugin init") + + if (shouldOverrideMemoryConf(sc.getConf)) { + val execMemOverhead = if (sc.getConf.contains(EXECUTOR_MEMORY_OVERHEAD.key)) { + sc.getConf.getSizeAsMb(EXECUTOR_MEMORY_OVERHEAD.key) + } else { + // By default, executorMemory * spark.executor.memoryOverheadFactor, with minimum of 384MB + val executorMemory = sc.getConf.getSizeAsMb(EXECUTOR_MEMORY.key) + val memoryOverheadFactor = + sc.getConf.getDouble( + EXECUTOR_MEMORY_OVERHEAD_FACTOR, + EXECUTOR_MEMORY_OVERHEAD_FACTOR_DEFAULT) + + Math.max( + (executorMemory * memoryOverheadFactor).toInt, + ResourceProfile.MEMORY_OVERHEAD_MIN_MIB) + } + + val cometMemOverhead = CometSparkSessionExtensions.getCometMemoryOverheadInMiB(sc.getConf) + sc.conf.set(EXECUTOR_MEMORY_OVERHEAD.key, s"${execMemOverhead + cometMemOverhead}M") + val newExecMemOverhead = sc.getConf.getSizeAsMb(EXECUTOR_MEMORY_OVERHEAD.key) + + logInfo(s""" + Overriding Spark memory configuration for Comet: + - Spark executor memory overhead: ${execMemOverhead}MB + - Comet memory overhead: ${cometMemOverhead}MB + - Updated Spark executor memory overhead: ${newExecMemOverhead}MB + """) + } + + Collections.emptyMap[String, String] + } + + override def receive(message: Any): AnyRef = super.receive(message) + + override def shutdown(): Unit = { + logInfo("CometDriverPlugin shutdown") + + super.shutdown() + } + + override def registerMetrics(appId: String, pluginContext: PluginContext): Unit = + super.registerMetrics(appId, pluginContext) + + /** + * Whether we should override Spark memory configuration for Comet. This only returns true when + * Comet native execution is enabled + */ + private def shouldOverrideMemoryConf(conf: SparkConf): Boolean = { + conf.getBoolean(CometConf.COMET_ENABLED.key, true) && + conf.getBoolean(CometConf.COMET_EXEC_ENABLED.key, false) + } +} + +object CometDriverPlugin { + // `org.apache.spark.internal.config.EXECUTOR_MEMORY_OVERHEAD_FACTOR` was added since Spark 3.3.0 + val EXECUTOR_MEMORY_OVERHEAD_FACTOR = "spark.executor.memoryOverheadFactor" + val EXECUTOR_MEMORY_OVERHEAD_FACTOR_DEFAULT = 0.1 +} + +/** + * The Comet plugin for Spark. To enable this plugin, set the config "spark.plugins" to + * `org.apache.spark.CometPlugin` + */ +class CometPlugin extends SparkPlugin with Logging { + override def driverPlugin(): DriverPlugin = new CometDriverPlugin + + override def executorPlugin(): ExecutorPlugin = null +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala new file mode 100644 index 0000000000..866231dea5 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import scala.concurrent.duration.NANOSECONDS + +import org.apache.spark.rdd._ +import org.apache.spark.sql.catalyst._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, DynamicPruningExpression, Expression, Literal} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.execution.metric._ +import org.apache.spark.sql.vectorized._ + +import org.apache.comet.MetricsSupport +import org.apache.comet.shims.ShimCometBatchScanExec + +import com.google.common.base.Objects + +case class CometBatchScanExec(wrapped: BatchScanExec, runtimeFilters: Seq[Expression]) + extends DataSourceV2ScanExecBase + with ShimCometBatchScanExec + with CometPlan { + + wrapped.logicalLink.foreach(setLogicalLink) + + override lazy val inputRDD: RDD[InternalRow] = wrappedScan.inputRDD + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + val scanTime = longMetric("scanTime") + inputRDD.asInstanceOf[RDD[ColumnarBatch]].mapPartitionsInternal { batches => + new Iterator[ColumnarBatch] { + + override def hasNext: Boolean = { + // The `FileScanRDD` returns an iterator which scans the file during the `hasNext` call. + val startNs = System.nanoTime() + val res = batches.hasNext + scanTime += NANOSECONDS.toMillis(System.nanoTime() - startNs) + res + } + + override def next(): ColumnarBatch = { + val batch = batches.next() + numOutputRows += batch.numRows() + batch + } + } + } + } + + // `ReusedSubqueryExec` in Spark only call non-columnar execute. + override def doExecute(): RDD[InternalRow] = { + ColumnarToRowExec(this).doExecute() + } + + override def executeCollect(): Array[InternalRow] = { + ColumnarToRowExec(this).executeCollect() + } + + override def readerFactory: PartitionReaderFactory = wrappedScan.readerFactory + + override def scan: Scan = wrapped.scan + + override def output: Seq[Attribute] = wrapped.output + + override def equals(other: Any): Boolean = other match { + case other: CometBatchScanExec => + // `wrapped` in `this` and `other` could reference to the same `BatchScanExec` object, + // therefore we need to also check `runtimeFilters` equality here. + this.wrappedScan == other.wrappedScan && this.runtimeFilters == other.runtimeFilters + case _ => + false + } + + override def hashCode(): Int = { + Objects.hashCode(wrappedScan, runtimeFilters) + } + + override def doCanonicalize(): CometBatchScanExec = { + this.copy( + wrapped = wrappedScan.doCanonicalize(), + runtimeFilters = QueryPlan.normalizePredicates( + runtimeFilters.filterNot(_ == DynamicPruningExpression(Literal.TrueLiteral)), + output)) + } + + override def nodeName: String = { + wrapped.nodeName.replace("BatchScan", "CometBatchScan") + } + + override def simpleString(maxFields: Int): String = { + val truncatedOutputString = truncatedString(output, "[", ", ", "]", maxFields) + val runtimeFiltersString = + s"RuntimeFilters: ${runtimeFilters.mkString("[", ",", "]")}" + val result = s"$nodeName$truncatedOutputString ${scan.description()} $runtimeFiltersString" + redact(result) + } + + private def wrappedScan: BatchScanExec = { + // The runtime filters in this scan could be transformed by optimizer rules such as + // `PlanAdaptiveDynamicPruningFilters`, while the one in the wrapped scan is not. And + // since `inputRDD` uses the latter and therefore will be incorrect if we don't set it here. + // + // There is, however, no good way to modify `wrapped.runtimeFilters` since it is immutable. + // It is not good to use `wrapped.copy` here since it will also re-initialize those lazy val + // in the `BatchScanExec`, e.g., metrics. + // + // TODO: find a better approach than this hack + val f = classOf[BatchScanExec].getDeclaredField("runtimeFilters") + f.setAccessible(true) + f.set(wrapped, runtimeFilters) + wrapped + } + + override lazy val metrics: Map[String, SQLMetric] = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "scanTime" -> SQLMetrics.createTimingMetric( + sparkContext, + "scan time")) ++ wrapped.customMetrics ++ { + wrapped.scan match { + case s: MetricsSupport => s.initMetrics(sparkContext) + case _ => Map.empty + } + } + + // Intentionally omitting the return type as it is different depending on Spark version + // Spark 3.2.x Seq[InputPartition] + // Spark 3.3.x Seq[Seq[InputPartition]] + // TODO: add back the return type after dropping Spark 3.2.0 support + @transient override lazy val partitions = wrappedScan.partitions + + override def supportsColumnar: Boolean = true +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala new file mode 100644 index 0000000000..fc4f90f897 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCoalesceExec.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition, UnknownPartitioning} +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.vectorized.ColumnarBatch + +import com.google.common.base.Objects + +/** + * This is basically a copy of Spark's CoalesceExec, but supports columnar processing to make it + * more efficient when including it in a Comet query plan. + */ +case class CometCoalesceExec( + override val originalPlan: SparkPlan, + numPartitions: Int, + child: SparkPlan) + extends CometExec + with UnaryExecNode { + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val rdd = child.executeColumnar() + if (numPartitions == 1 && rdd.getNumPartitions < 1) { + // Make sure we don't output an RDD with 0 partitions, when claiming that we have a + // `SinglePartition`. + new CometCoalesceExec.EmptyRDDWithPartitions(sparkContext, numPartitions) + } else { + rdd.coalesce(numPartitions, shuffle = false) + } + } + + override def outputPartitioning: Partitioning = { + if (numPartitions == 1) SinglePartition + else UnknownPartitioning(numPartitions) + } + + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometCoalesceExec => + this.numPartitions == other.numPartitions && this.child == other.child + case _ => + false + } + } + + override def hashCode(): Int = Objects.hashCode(numPartitions: java.lang.Integer, child) +} + +object CometCoalesceExec { + + /** A simple RDD with no data, but with the given number of partitions. */ + class EmptyRDDWithPartitions(@transient private val sc: SparkContext, numPartitions: Int) + extends RDD[ColumnarBatch](sc, Nil) { + + override def getPartitions: Array[Partition] = + Array.tabulate(numPartitions)(i => EmptyPartition(i)) + + override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { + Iterator.empty + } + } + + case class EmptyPartition(index: Int) extends Partition +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala new file mode 100644 index 0000000000..60b26ca00c --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** + * A node carrying SQL metrics from SparkPlan, and metrics of its children. Native code will call + * [[getChildNode]] and [[add]] to update the metrics. + * + * @param metrics + * the mapping between metric name of native operator to `SQLMetric` of Spark operator. For + * example, `numOutputRows` -> `SQLMetrics("numOutputRows")` means the native operator will + * update `numOutputRows` metric with the value of `SQLMetrics("numOutputRows")` in Spark + * operator. + */ +case class CometMetricNode(metrics: Map[String, SQLMetric], children: Seq[CometMetricNode]) + extends Logging { + + /** + * Gets a child node. Called from native. + */ + def getChildNode(i: Int): CometMetricNode = { + if (i < 0 || i >= children.length) { + // TODO: throw an exception, e.g. IllegalArgumentException, instead? + return null + } + children(i) + } + + /** + * Adds a value to a metric. Called from native. + * + * @param metricName + * the name of the metric at native operator. + * @param v + * the value to add. + */ + def add(metricName: String, v: Long): Unit = { + metrics.get(metricName) match { + case Some(metric) => metric.set(v) + case None => + // no-op + logDebug(s"Non-existing metric: $metricName. Ignored") + } + } +} + +object CometMetricNode { + + /** + * The baseline SQL metrics for DataFusion `BaselineMetrics`. + */ + def baselineMetrics(sc: SparkContext): Map[String, SQLMetric] = { + Map( + "output_rows" -> SQLMetrics.createMetric(sc, "number of output rows"), + "elapsed_compute" -> SQLMetrics.createNanoTimingMetric( + sc, + "total time (in ms) spent in this operator")) + } + + /** + * Creates a [[CometMetricNode]] from a [[CometPlan]]. + */ + def fromCometPlan(cometPlan: SparkPlan): CometMetricNode = { + val children = cometPlan.children.map(fromCometPlan) + CometMetricNode(cometPlan.metrics, children) + } + + /** + * Creates a [[CometMetricNode]] from a map of [[SQLMetric]]. + */ + def apply(metrics: Map[String, SQLMetric]): CometMetricNode = { + CometMetricNode(metrics, Nil) + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometPlan.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometPlan.scala new file mode 100644 index 0000000000..fe2ce7ed0e --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometPlan.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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.comet + +import org.apache.spark.sql.execution.SparkPlan + +/** + * The base trait for physical Comet operators. + */ +trait CometPlan extends SparkPlan diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala new file mode 100644 index 0000000000..4bf01f0f43 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala @@ -0,0 +1,458 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import scala.collection.mutable.HashMap +import scala.concurrent.duration.NANOSECONDS + +import org.apache.hadoop.fs.Path +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst._ +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions +import org.apache.spark.sql.execution.metric._ +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.collection._ + +import org.apache.comet.{CometConf, MetricsSupport} +import org.apache.comet.parquet.{CometParquetFileFormat, CometParquetPartitionReaderFactory} +import org.apache.comet.shims.{ShimCometScanExec, ShimFileFormat} + +/** + * Comet physical scan node for DataSource V1. Most of the code here follow Spark's + * [[FileSourceScanExec]], + */ +case class CometScanExec( + @transient relation: HadoopFsRelation, + output: Seq[Attribute], + requiredSchema: StructType, + partitionFilters: Seq[Expression], + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + dataFilters: Seq[Expression], + tableIdentifier: Option[TableIdentifier], + disableBucketedScan: Boolean = false, + wrapped: FileSourceScanExec) + extends DataSourceScanExec + with ShimCometScanExec + with CometPlan { + + // FIXME: ideally we should reuse wrapped.supportsColumnar, however that fails many tests + override lazy val supportsColumnar: Boolean = + relation.fileFormat.supportBatch(relation.sparkSession, schema) + + override def vectorTypes: Option[Seq[String]] = wrapped.vectorTypes + + private lazy val driverMetrics: HashMap[String, Long] = HashMap.empty + + /** + * Send the driver-side metrics. Before calling this function, selectedPartitions has been + * initialized. See SPARK-26327 for more details. + */ + private def sendDriverMetrics(): Unit = { + driverMetrics.foreach(e => metrics(e._1).add(e._2)) + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates( + sparkContext, + executionId, + metrics.filter(e => driverMetrics.contains(e._1)).values.toSeq) + } + + private def isDynamicPruningFilter(e: Expression): Boolean = + e.find(_.isInstanceOf[PlanExpression[_]]).isDefined + + @transient lazy val selectedPartitions: Array[PartitionDirectory] = { + val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) + val startTime = System.nanoTime() + val ret = + relation.location.listFiles(partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) + setFilesNumAndSizeMetric(ret, true) + val timeTakenMs = + NANOSECONDS.toMillis((System.nanoTime() - startTime) + optimizerMetadataTimeNs) + driverMetrics("metadataTime") = timeTakenMs + ret + }.toArray + + // We can only determine the actual partitions at runtime when a dynamic partition filter is + // present. This is because such a filter relies on information that is only available at run + // time (for instance the keys used in the other side of a join). + @transient private lazy val dynamicallySelectedPartitions: Array[PartitionDirectory] = { + val dynamicPartitionFilters = partitionFilters.filter(isDynamicPruningFilter) + + if (dynamicPartitionFilters.nonEmpty) { + val startTime = System.nanoTime() + // call the file index for the files matching all filters except dynamic partition filters + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil) + val ret = selectedPartitions.filter(p => boundPredicate.eval(p.values)) + setFilesNumAndSizeMetric(ret, false) + val timeTakenMs = (System.nanoTime() - startTime) / 1000 / 1000 + driverMetrics("pruningTime") = timeTakenMs + ret + } else { + selectedPartitions + } + } + + // exposed for testing + lazy val bucketedScan: Boolean = wrapped.bucketedScan + + override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = + (wrapped.outputPartitioning, wrapped.outputOrdering) + + @transient + private lazy val pushedDownFilters = { + val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation) + dataFilters.flatMap(DataSourceStrategy.translateFilter(_, supportNestedPredicatePushdown)) + } + + override lazy val metadata: Map[String, String] = + if (wrapped == null) Map.empty else wrapped.metadata + + override def verboseStringWithOperatorId(): String = wrapped.verboseStringWithOperatorId() + + lazy val inputRDD: RDD[InternalRow] = { + val options = relation.options + + (ShimFileFormat.OPTION_RETURNING_BATCH -> supportsColumnar.toString) + val readFile: (PartitionedFile) => Iterator[InternalRow] = + relation.fileFormat.buildReaderWithPartitionValues( + sparkSession = relation.sparkSession, + dataSchema = relation.dataSchema, + partitionSchema = relation.partitionSchema, + requiredSchema = requiredSchema, + filters = pushedDownFilters, + options = options, + hadoopConf = + relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options)) + + val readRDD = if (bucketedScan) { + createBucketedReadRDD( + relation.bucketSpec.get, + readFile, + dynamicallySelectedPartitions, + relation) + } else { + createReadRDD(readFile, dynamicallySelectedPartitions, relation) + } + sendDriverMetrics() + readRDD + } + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + inputRDD :: Nil + } + + /** Helper for computing total number and size of files in selected partitions. */ + private def setFilesNumAndSizeMetric( + partitions: Seq[PartitionDirectory], + static: Boolean): Unit = { + val filesNum = partitions.map(_.files.size.toLong).sum + val filesSize = partitions.map(_.files.map(_.getLen).sum).sum + if (!static || !partitionFilters.exists(isDynamicPruningFilter)) { + driverMetrics("numFiles") = filesNum + driverMetrics("filesSize") = filesSize + } else { + driverMetrics("staticFilesNum") = filesNum + driverMetrics("staticFilesSize") = filesSize + } + if (relation.partitionSchema.nonEmpty) { + driverMetrics("numPartitions") = partitions.length + } + } + + override lazy val metrics: Map[String, SQLMetric] = wrapped.metrics ++ { + // Tracking scan time has overhead, we can't afford to do it for each row, and can only do + // it for each batch. + if (supportsColumnar) { + Some("scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + } else { + None + } + } ++ { + relation.fileFormat match { + case f: MetricsSupport => f.initMetrics(sparkContext) + case _ => Map.empty + } + } + + protected override def doExecute(): RDD[InternalRow] = { + ColumnarToRowExec(this).doExecute() + } + + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + val scanTime = longMetric("scanTime") + inputRDD.asInstanceOf[RDD[ColumnarBatch]].mapPartitionsInternal { batches => + new Iterator[ColumnarBatch] { + + override def hasNext: Boolean = { + // The `FileScanRDD` returns an iterator which scans the file during the `hasNext` call. + val startNs = System.nanoTime() + val res = batches.hasNext + scanTime += NANOSECONDS.toMillis(System.nanoTime() - startNs) + res + } + + override def next(): ColumnarBatch = { + val batch = batches.next() + numOutputRows += batch.numRows() + batch + } + } + } + } + + override def executeCollect(): Array[InternalRow] = { + ColumnarToRowExec(this).executeCollect() + } + + override val nodeName: String = + s"CometScan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" + + /** + * Create an RDD for bucketed reads. The non-bucketed variant of this function is + * [[createReadRDD]]. + * + * The algorithm is pretty simple: each RDD partition being returned should include all the + * files with the same bucket id from all the given Hive partitions. + * + * @param bucketSpec + * the bucketing spec. + * @param readFile + * a function to read each (part of a) file. + * @param selectedPartitions + * Hive-style partition that are part of the read. + * @param fsRelation + * [[HadoopFsRelation]] associated with the read. + */ + private def createBucketedReadRDD( + bucketSpec: BucketSpec, + readFile: (PartitionedFile) => Iterator[InternalRow], + selectedPartitions: Array[PartitionDirectory], + fsRelation: HadoopFsRelation): RDD[InternalRow] = { + logInfo(s"Planning with ${bucketSpec.numBuckets} buckets") + val filesGroupedToBuckets = + selectedPartitions + .flatMap { p => + p.files.map { f => + PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) + } + } + .groupBy { f => + BucketingUtils + .getBucketId(new Path(f.filePath.toString()).getName) + .getOrElse(throw invalidBucketFile(f.filePath.toString(), sparkContext.version)) + } + + val prunedFilesGroupedToBuckets = if (optionalBucketSet.isDefined) { + val bucketSet = optionalBucketSet.get + filesGroupedToBuckets.filter { f => + bucketSet.get(f._1) + } + } else { + filesGroupedToBuckets + } + + val filePartitions = optionalNumCoalescedBuckets + .map { numCoalescedBuckets => + logInfo(s"Coalescing to ${numCoalescedBuckets} buckets") + val coalescedBuckets = prunedFilesGroupedToBuckets.groupBy(_._1 % numCoalescedBuckets) + Seq.tabulate(numCoalescedBuckets) { bucketId => + val partitionedFiles = coalescedBuckets + .get(bucketId) + .map { + _.values.flatten.toArray + } + .getOrElse(Array.empty) + FilePartition(bucketId, partitionedFiles) + } + } + .getOrElse { + Seq.tabulate(bucketSpec.numBuckets) { bucketId => + FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) + } + } + + prepareRDD(fsRelation, readFile, filePartitions) + } + + /** + * Create an RDD for non-bucketed reads. The bucketed variant of this function is + * [[createBucketedReadRDD]]. + * + * @param readFile + * a function to read each (part of a) file. + * @param selectedPartitions + * Hive-style partition that are part of the read. + * @param fsRelation + * [[HadoopFsRelation]] associated with the read. + */ + private def createReadRDD( + readFile: (PartitionedFile) => Iterator[InternalRow], + selectedPartitions: Array[PartitionDirectory], + fsRelation: HadoopFsRelation): RDD[InternalRow] = { + val openCostInBytes = fsRelation.sparkSession.sessionState.conf.filesOpenCostInBytes + val maxSplitBytes = + FilePartition.maxSplitBytes(fsRelation.sparkSession, selectedPartitions) + logInfo( + s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + + s"open cost is considered as scanning $openCostInBytes bytes.") + + // Filter files with bucket pruning if possible + val bucketingEnabled = fsRelation.sparkSession.sessionState.conf.bucketingEnabled + val shouldProcess: Path => Boolean = optionalBucketSet match { + case Some(bucketSet) if bucketingEnabled => + // Do not prune the file if bucket file name is invalid + filePath => BucketingUtils.getBucketId(filePath.getName).forall(bucketSet.get) + case _ => + _ => true + } + + val splitFiles = selectedPartitions + .flatMap { partition => + partition.files.flatMap { file => + // getPath() is very expensive so we only want to call it once in this block: + val filePath = file.getPath + + if (shouldProcess(filePath)) { + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, + relation.options, + filePath) && + // SPARK-39634: Allow file splitting in combination with row index generation once + // the fix for PARQUET-2161 is available. + !isNeededForSchema(requiredSchema) + PartitionedFileUtil.splitFiles( + sparkSession = relation.sparkSession, + file = file, + filePath = filePath, + isSplitable = isSplitable, + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values) + } else { + Seq.empty + } + } + } + .sortBy(_.length)(implicitly[Ordering[Long]].reverse) + + prepareRDD( + fsRelation, + readFile, + FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes)) + } + + private def prepareRDD( + fsRelation: HadoopFsRelation, + readFile: (PartitionedFile) => Iterator[InternalRow], + partitions: Seq[FilePartition]): RDD[InternalRow] = { + val hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options) + val prefetchEnabled = hadoopConf.getBoolean( + CometConf.COMET_SCAN_PREFETCH_ENABLED.key, + CometConf.COMET_SCAN_PREFETCH_ENABLED.defaultValue.get) + + val sqlConf = fsRelation.sparkSession.sessionState.conf + if (prefetchEnabled) { + CometParquetFileFormat.populateConf(sqlConf, hadoopConf) + val broadcastedConf = + fsRelation.sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + val partitionReaderFactory = CometParquetPartitionReaderFactory( + sqlConf, + broadcastedConf, + requiredSchema, + relation.partitionSchema, + pushedDownFilters.toArray, + new ParquetOptions(CaseInsensitiveMap(relation.options), sqlConf), + metrics) + + newDataSourceRDD( + fsRelation.sparkSession.sparkContext, + partitions.map(Seq(_)), + partitionReaderFactory, + true, + Map.empty) + } else { + newFileScanRDD( + fsRelation.sparkSession, + readFile, + partitions, + new StructType(requiredSchema.fields ++ fsRelation.partitionSchema.fields), + new ParquetOptions(CaseInsensitiveMap(relation.options), sqlConf)) + } + } + + // Filters unused DynamicPruningExpression expressions - one which has been replaced + // with DynamicPruningExpression(Literal.TrueLiteral) during Physical Planning + private def filterUnusedDynamicPruningExpressions( + predicates: Seq[Expression]): Seq[Expression] = { + predicates.filterNot(_ == DynamicPruningExpression(Literal.TrueLiteral)) + } + + override def doCanonicalize(): CometScanExec = { + CometScanExec( + relation, + output.map(QueryPlan.normalizeExpressions(_, output)), + requiredSchema, + QueryPlan.normalizePredicates( + filterUnusedDynamicPruningExpressions(partitionFilters), + output), + optionalBucketSet, + optionalNumCoalescedBuckets, + QueryPlan.normalizePredicates(dataFilters, output), + None, + disableBucketedScan, + null) + } +} + +object CometScanExec { + def apply(scanExec: FileSourceScanExec, session: SparkSession): CometScanExec = { + val wrapped = scanExec.copy(relation = + scanExec.relation.copy(fileFormat = new CometParquetFileFormat)(session)) + val batchScanExec = CometScanExec( + wrapped.relation, + wrapped.output, + wrapped.requiredSchema, + wrapped.partitionFilters, + wrapped.optionalBucketSet, + wrapped.optionalNumCoalescedBuckets, + wrapped.dataFilters, + wrapped.tableIdentifier, + wrapped.disableBucketedScan, + wrapped) + scanExec.logicalLink.foreach(batchScanExec.setLogicalLink) + batchScanExec + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/DataSourceHelpers.scala b/spark/src/main/scala/org/apache/spark/sql/comet/DataSourceHelpers.scala new file mode 100644 index 0000000000..5c505c4032 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/DataSourceHelpers.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources._ + +object DataSourceHelpers { + def getPushedDownFilters(relation: BaseRelation, dataFilters: Seq[Expression]): Seq[Filter] = { + val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation) + dataFilters.flatMap(DataSourceStrategy.translateFilter(_, supportNestedPredicatePushdown)) + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/DecimalPrecision.scala b/spark/src/main/scala/org/apache/spark/sql/comet/DecimalPrecision.scala new file mode 100644 index 0000000000..13f26ce585 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/DecimalPrecision.scala @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import scala.math.{max, min} + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.DecimalType + +/** + * This is mostly copied from the `decimalAndDecimal` method in Spark's [[DecimalPrecision]] which + * existed before Spark 3.4. + * + * In Spark 3.4 and up, the method `decimalAndDecimal` is removed from Spark, and for binary + * expressions with different decimal precisions from children, the difference is handled in the + * expression evaluation instead (see SPARK-39316). + * + * However in Comet, we still have to rely on the type coercion to ensure the decimal precision is + * the same for both children of a binary expression, since our arithmetic kernels do not yet + * handle the case where precision is different. Therefore, this re-apply the logic in the + * original rule, and rely on `Cast` and `CheckOverflow` for decimal binary operation. + * + * TODO: instead of relying on this rule, it's probably better to enhance arithmetic kernels to + * handle different decimal precisions + */ +object DecimalPrecision { + def promote( + allowPrecisionLoss: Boolean, + expr: Expression, + nullOnOverflow: Boolean): Expression = { + expr.transformUp { + // This means the binary expression is already optimized with the rule in Spark. This can + // happen if the Spark version is < 3.4 + case e: BinaryArithmetic if e.left.prettyName == "promote_precision" => e + + case add @ Add(DecimalExpression(p1, s1), DecimalExpression(p2, s2), _) => + val resultScale = max(s1, s2) + val resultType = if (allowPrecisionLoss) { + DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale) + } else { + DecimalType.bounded(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale) + } + CheckOverflow(add, resultType, nullOnOverflow) + + case sub @ Subtract(DecimalType.Expression(p1, s1), DecimalType.Expression(p2, s2), _) => + val resultScale = max(s1, s2) + val resultType = if (allowPrecisionLoss) { + DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale) + } else { + DecimalType.bounded(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale) + } + CheckOverflow(sub, resultType, nullOnOverflow) + + case mul @ Multiply(DecimalType.Expression(p1, s1), DecimalType.Expression(p2, s2), _) => + val resultType = if (allowPrecisionLoss) { + DecimalType.adjustPrecisionScale(p1 + p2 + 1, s1 + s2) + } else { + DecimalType.bounded(p1 + p2 + 1, s1 + s2) + } + CheckOverflow(mul, resultType, nullOnOverflow) + + case div @ Divide(DecimalType.Expression(p1, s1), DecimalType.Expression(p2, s2), _) => + val resultType = if (allowPrecisionLoss) { + // Precision: p1 - s1 + s2 + max(6, s1 + p2 + 1) + // Scale: max(6, s1 + p2 + 1) + val intDig = p1 - s1 + s2 + val scale = max(DecimalType.MINIMUM_ADJUSTED_SCALE, s1 + p2 + 1) + val prec = intDig + scale + DecimalType.adjustPrecisionScale(prec, scale) + } else { + var intDig = min(DecimalType.MAX_SCALE, p1 - s1 + s2) + var decDig = min(DecimalType.MAX_SCALE, max(6, s1 + p2 + 1)) + val diff = (intDig + decDig) - DecimalType.MAX_SCALE + if (diff > 0) { + decDig -= diff / 2 + 1 + intDig = DecimalType.MAX_SCALE - decDig + } + DecimalType.bounded(intDig + decDig, decDig) + } + CheckOverflow(div, resultType, nullOnOverflow) + + case rem @ Remainder(DecimalType.Expression(p1, s1), DecimalType.Expression(p2, s2), _) => + val resultType = if (allowPrecisionLoss) { + DecimalType.adjustPrecisionScale(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) + } else { + DecimalType.bounded(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) + } + CheckOverflow(rem, resultType, nullOnOverflow) + + case e => e + } + } + + object DecimalExpression { + def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { + case t: DecimalType => Some((t.precision, t.scale)) + case _ => None + } + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala new file mode 100644 index 0000000000..46fc27415c --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -0,0 +1,473 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.io.ByteArrayOutputStream + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, NamedExpression, SortOrder} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateMode} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, ExplainUtils, LeafExecNode, ScalarSubquery, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch + +import org.apache.comet.{CometConf, CometExecIterator, CometRuntimeException, CometSparkSessionExtensions} +import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION} +import org.apache.comet.serde.OperatorOuterClass.Operator + +import com.google.common.base.Objects + +/** + * A Comet physical operator + */ +abstract class CometExec extends CometPlan { + + /** The original Spark operator from which this Comet operator is converted from */ + def originalPlan: SparkPlan + + /** Comet always support columnar execution */ + override def supportsColumnar: Boolean = true + + override def output: Seq[Attribute] = originalPlan.output + + override def doExecute(): RDD[InternalRow] = + ColumnarToRowExec(this).doExecute() + + override def executeCollect(): Array[InternalRow] = + ColumnarToRowExec(this).executeCollect() + + override def outputOrdering: Seq[SortOrder] = originalPlan.outputOrdering + + override def outputPartitioning: Partitioning = originalPlan.outputPartitioning +} + +object CometExec { + // An unique id for each CometExecIterator, used to identify the native query execution. + private val curId = new java.util.concurrent.atomic.AtomicLong() + + def newIterId: Long = curId.getAndIncrement() + + def getCometIterator( + inputs: Seq[Iterator[ColumnarBatch]], + nativePlan: Operator): CometExecIterator = { + getCometIterator(inputs, nativePlan, CometMetricNode(Map.empty)) + } + + def getCometIterator( + inputs: Seq[Iterator[ColumnarBatch]], + nativePlan: Operator, + nativeMetrics: CometMetricNode): CometExecIterator = { + val outputStream = new ByteArrayOutputStream() + nativePlan.writeTo(outputStream) + outputStream.close() + val bytes = outputStream.toByteArray + + val configs = new java.util.HashMap[String, String]() + + val maxMemory = + CometSparkSessionExtensions.getCometMemoryOverhead(SparkEnv.get.conf) + configs.put("memory_limit", String.valueOf(maxMemory)) + configs.put("memory_fraction", String.valueOf(COMET_EXEC_MEMORY_FRACTION.get())) + configs.put("batch_size", String.valueOf(COMET_BATCH_SIZE.get())) + configs.put("debug_native", String.valueOf(COMET_DEBUG_ENABLED.get())) + + new CometExecIterator(newIterId, inputs, bytes, configs, nativeMetrics) + } +} + +/** + * A Comet native physical operator. + */ +abstract class CometNativeExec extends CometExec { + + /** + * The serialized native query plan, optional. This is only defined when the current node is the + * "boundary" node between native and Spark. + */ + private var serializedPlanOpt: Option[Array[Byte]] = None + + /** The Comet native operator */ + def nativeOp: Operator + + override protected def doPrepare(): Unit = prepareSubqueries(originalPlan) + + override lazy val metrics: Map[String, SQLMetric] = + CometMetricNode.baselineMetrics(sparkContext) + + private def prepareSubqueries(sparkPlan: SparkPlan): Unit = { + val runningSubqueries = new ArrayBuffer[ExecSubqueryExpression] + + sparkPlan.children.foreach(prepareSubqueries) + + sparkPlan.expressions.foreach { + _.collect { case e: ScalarSubquery => + runningSubqueries += e + } + } + + // fill in the result of subqueries + runningSubqueries.foreach { sub => + sub.updateResult() + } + + runningSubqueries.clear() + } + + private def setSubqueries(planId: Long, sparkPlan: SparkPlan): Unit = { + sparkPlan.children.foreach(setSubqueries(planId, _)) + + sparkPlan.expressions.foreach { + _.collect { case sub: ScalarSubquery => + CometScalarSubquery.setSubquery(planId, sub) + } + } + } + + private def cleanSubqueries(planId: Long, sparkPlan: SparkPlan): Unit = { + sparkPlan.children.foreach(cleanSubqueries(planId, _)) + + sparkPlan.expressions.foreach { + _.collect { case sub: ScalarSubquery => + CometScalarSubquery.removeSubquery(planId, sub) + } + } + } + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + serializedPlanOpt match { + case None => + assert(children.length == 1) // TODO: fix this! + children.head.executeColumnar() + case Some(serializedPlan) => + // Switch to use Decimal128 regardless of precision, since Arrow native execution + // doesn't support Decimal32 and Decimal64 yet. + conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") + + // Populate native configurations + val configs = new java.util.HashMap[String, String]() + val maxMemory = CometSparkSessionExtensions.getCometMemoryOverhead(sparkContext.getConf) + configs.put("memory_limit", String.valueOf(maxMemory)) + configs.put("memory_fraction", String.valueOf(COMET_EXEC_MEMORY_FRACTION.get())) + configs.put("batch_size", String.valueOf(COMET_BATCH_SIZE.get())) + configs.put("debug_native", String.valueOf(COMET_DEBUG_ENABLED.get())) + + // Strip mandatory prefix spark. which is not required for datafusion session params + session.conf.getAll.foreach { + case (k, v) if k.startsWith("spark.datafusion") => + configs.put(k.replaceFirst("spark\\.", ""), v) + case _ => + } + val serializedPlanCopy = serializedPlan + // TODO: support native metrics for all operators. + val nativeMetrics = CometMetricNode.fromCometPlan(this) + + def createCometExecIter(inputs: Seq[Iterator[ColumnarBatch]]): CometExecIterator = { + val it = new CometExecIterator( + CometExec.newIterId, + inputs, + serializedPlanCopy, + configs, + nativeMetrics) + + setSubqueries(it.id, originalPlan) + + Option(TaskContext.get()).foreach { context => + context.addTaskCompletionListener[Unit] { _ => + it.close() + cleanSubqueries(it.id, originalPlan) + } + } + + it + } + + children.map(_.executeColumnar) match { + case Seq(child) => + child.mapPartitionsInternal(iter => createCometExecIter(Seq(iter))) + case Seq(first, second) => + first.zipPartitions(second) { (iter1, iter2) => + createCometExecIter(Seq(iter1, iter2)) + } + case _ => + throw new CometRuntimeException( + s"Expected only two children but got s${children.size}") + } + } + } + + /** + * Converts this native Comet operator and its children into a native block which can be + * executed as a whole (i.e., in a single JNI call) from the native side. + */ + def convertBlock(): Unit = { + val out = new ByteArrayOutputStream() + nativeOp.writeTo(out) + out.close() + serializedPlanOpt = Some(out.toByteArray) + } +} + +abstract class CometUnaryExec extends CometNativeExec with UnaryExecNode + +case class CometProjectExec( + override val nativeOp: Operator, + override val originalPlan: SparkPlan, + projectList: Seq[NamedExpression], + override val output: Seq[Attribute], + child: SparkPlan) + extends CometUnaryExec { + override def producedAttributes: AttributeSet = outputSet + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + + override def stringArgs: Iterator[Any] = Iterator(output, projectList, child) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometProjectExec => + this.projectList == other.projectList && + this.output == other.output && this.child == other.child + case _ => + false + } + } + + override def hashCode(): Int = Objects.hashCode(projectList, output, child) +} + +case class CometFilterExec( + override val nativeOp: Operator, + override val originalPlan: SparkPlan, + condition: Expression, + child: SparkPlan) + extends CometUnaryExec { + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + + override def stringArgs: Iterator[Any] = + Iterator(originalPlan.output, condition, child) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometFilterExec => + this.condition == other.condition && this.child == other.child + case _ => + false + } + } + + override def hashCode(): Int = Objects.hashCode(condition, child) + + override def verboseStringWithOperatorId(): String = { + s""" + |$formattedNodeName + |${ExplainUtils.generateFieldString("Input", child.output)} + |Condition : ${condition} + |""".stripMargin + } +} + +case class CometSortExec( + override val nativeOp: Operator, + override val originalPlan: SparkPlan, + sortOrder: Seq[SortOrder], + child: SparkPlan) + extends CometUnaryExec { + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + + override def stringArgs: Iterator[Any] = + Iterator(originalPlan.output, sortOrder, child) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometSortExec => + this.sortOrder == other.sortOrder && this.child == other.child + case _ => + false + } + } + + override def hashCode(): Int = Objects.hashCode(sortOrder, child) + + override lazy val metrics: Map[String, SQLMetric] = + CometMetricNode.baselineMetrics(sparkContext) ++ + Map( + "spill_count" -> SQLMetrics.createMetric(sparkContext, "number of spills"), + "spilled_bytes" -> SQLMetrics.createNanoTimingMetric(sparkContext, "total spilled bytes")) +} + +case class CometLocalLimitExec( + override val nativeOp: Operator, + override val originalPlan: SparkPlan, + limit: Int, + child: SparkPlan) + extends CometUnaryExec { + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + + override def stringArgs: Iterator[Any] = Iterator(limit, child) + + override lazy val metrics: Map[String, SQLMetric] = Map.empty +} + +case class CometGlobalLimitExec( + override val nativeOp: Operator, + override val originalPlan: SparkPlan, + limit: Int, + child: SparkPlan) + extends CometUnaryExec { + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + + override def stringArgs: Iterator[Any] = Iterator(limit, child) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometGlobalLimitExec => + this.limit == other.limit && this.child == other.child + case _ => + false + } + } + + override def hashCode(): Int = Objects.hashCode(limit: java.lang.Integer, child) +} + +case class CometExpandExec( + override val nativeOp: Operator, + override val originalPlan: SparkPlan, + projections: Seq[Seq[Expression]], + child: SparkPlan) + extends CometUnaryExec { + override def producedAttributes: AttributeSet = outputSet + + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + + override def stringArgs: Iterator[Any] = Iterator(projections, output, child) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometExpandExec => + this.projections == other.projections && this.child == other.child + case _ => + false + } + } + + override def hashCode(): Int = Objects.hashCode(projections, child) + + // TODO: support native Expand metrics + override lazy val metrics: Map[String, SQLMetric] = Map.empty +} + +case class CometUnionExec(override val originalPlan: SparkPlan, children: Seq[SparkPlan]) + extends CometExec { + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + sparkContext.union(children.map(_.executeColumnar())) + } + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = + this.copy(children = newChildren) + + override def verboseStringWithOperatorId(): String = { + val childrenString = children.zipWithIndex + .map { case (child, index) => + s"Child $index ${ExplainUtils.generateFieldString("Input", child.output)}" + } + .mkString("\n") + s""" + |$formattedNodeName + |$childrenString + |""".stripMargin + } + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometUnionExec => this.children == other.children + case _ => false + } + } + + override def hashCode(): Int = Objects.hashCode(children) +} + +case class CometHashAggregateExec( + override val nativeOp: Operator, + override val originalPlan: SparkPlan, + groupingExpressions: Seq[NamedExpression], + aggregateExpressions: Seq[AggregateExpression], + input: Seq[Attribute], + mode: AggregateMode, + child: SparkPlan) + extends CometUnaryExec { + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + this.copy(child = newChild) + + override def stringArgs: Iterator[Any] = + Iterator(input, mode, groupingExpressions, aggregateExpressions, child) + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometHashAggregateExec => + this.groupingExpressions == other.groupingExpressions && + this.aggregateExpressions == other.aggregateExpressions && + this.input == other.input && + this.mode == other.mode && + this.child == other.child + case _ => + false + } + } + + override def hashCode(): Int = + Objects.hashCode(groupingExpressions, aggregateExpressions, input, mode, child) +} + +case class CometScanWrapper(override val nativeOp: Operator, override val originalPlan: SparkPlan) + extends CometNativeExec + with LeafExecNode { + override def stringArgs: Iterator[Any] = Iterator(originalPlan.output, originalPlan) +} + +/** + * A pseudo Comet physical scan node after Comet operators. This node is used to be a placeholder + * for chaining with following Comet native operators after previous Comet operators. This node + * will be removed after `CometExecRule`. + * + * This is very similar to `CometScanWrapper` above except it has child. + */ +case class CometSinkPlaceHolder( + override val nativeOp: Operator, // Must be a Scan + override val originalPlan: SparkPlan, + child: SparkPlan) + extends CometUnaryExec { + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { + this.copy(child = newChild) + } + + override def stringArgs: Iterator[Any] = Iterator(originalPlan.output, child) +} diff --git a/spark/src/test/java/org/apache/comet/IntegrationTestSuite.java b/spark/src/test/java/org/apache/comet/IntegrationTestSuite.java new file mode 100644 index 0000000000..ebeb41d4ec --- /dev/null +++ b/spark/src/test/java/org/apache/comet/IntegrationTestSuite.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.scalatest.TagAnnotation; + +/** + * This annotation is used on integration test suites. So we can exclude these tests from execution + * of scalatest-maven-plugin. + */ +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface IntegrationTestSuite {} diff --git a/spark/src/test/resources/log4j.properties b/spark/src/test/resources/log4j.properties new file mode 100644 index 0000000000..2f46ce1553 --- /dev/null +++ b/spark/src/test/resources/log4j.properties @@ -0,0 +1,36 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You 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. +# + +# Set everything to be logged to the file target/unit-tests.log +test.appender=file +log4j.rootCategory=INFO, ${test.appender} +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.sparkproject.jetty=WARN diff --git a/spark/src/test/resources/log4j2.properties b/spark/src/test/resources/log4j2.properties new file mode 100644 index 0000000000..04cdf85330 --- /dev/null +++ b/spark/src/test/resources/log4j2.properties @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn + diff --git a/spark/src/test/resources/test-data/before_1582_date_v2_4_5.snappy.parquet b/spark/src/test/resources/test-data/before_1582_date_v2_4_5.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..edd61c9b9fec8239636030cbda2789e223fc6320 GIT binary patch literal 660 zcmb7CJxjw-6uk+tWayB=7ZS)&9<(ULkXTHc(#1{0p}303OY^G1^eagfDP8@nTO9;v z2S@*vf5Am>nrd|sgq$1ByYHObb4hD^-ysr_NpX*eL;{eCmyhos1c*XbO)L^A5ZTbA z>@=?`to>jBAR8i9MF0h5d!&`gU?IEzC0jBSRA#v>bG|}cBW@74h&#j-kqL;XH+PE3 zfW@arN&AF%8O}+>vZzTx*iOsEoU9BY7WR;-if-s;MK(~lB^M+;zehw>QKy1B6Do4^ zk371GnI$3uX2@PR=#!9pK|$B@&&<u91@%S1Cb}(yW!qUY&<(V~0ntW%#R>30Tkprm z78@|kV$EY&$uP1o-*CiJEAF3LU7oUlrA&Je#x%VcaN3|r8jsp3HEAd0QIOEa!I<_~ zH)~~Uz{cVt3OTagD2kXLVIPIuq+g!2F`i7!Z|!_*{}UfhK-*o6yU~qHXFN{Aey`+M zj%_)mHGdRFL5bD9pi}R7UUk!VdB=6D)xdUb*W+7u+w*H)&Gxw;IP1D;0N?{}0zeD@ F$4_X9%2WUV literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_date_v2_4_6.snappy.parquet b/spark/src/test/resources/test-data/before_1582_date_v2_4_6.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..01f4887f5e994c77fc587378e7f3a19868745681 GIT binary patch literal 694 zcmb7CJ&W5w5FH8G2+oCytdKy3SvX^ZvPgC;TgFYg<8WcF4I!+obs{2vNpiwr+~(8i z?uvudDXzSqcg@|b<-`erKtOMor<wOOZ_xPpg-<l1liG-=L<7)jA8zg+2oQrjaq-Yd zmFSM8$+zWcLi0BU0J@{%X((WzY(&OV1_#-<KV(OSg2q)Tlg=LzpAerBUl3mrJA@Pv z)gGSJ@+HsSzRD+W#Ej#dL_Jk4ElwuIX{{ucWy+H|GSjdf+imC$3cFfWx63}FnufVk z%qyj#5C4(Jb}_R?RKN`K!a=_)doPe)<!AP0KNRy;!6LdHgURh28Q2cm;DBgj-r)pb zyOy=cUNgSrn1$_Qv3DZNlf~SswAnKjfYth|BVVNqH?UfZf@O>MY$ggG^MYH~%7zw~ zOF;)TFS7Ndpe~&zB8_u8c(I{#K9eQ6C@dTrQ7Dkj*J;Y56zeDyF8%3A2b=BI{iWUS z_5Z{#w_u$g#(g@=>W6%FpZOm1>&<ALq;Z}1!+6@ChGC~21!5WmolfipUJ#0|HwmL& W*z+P0#eU0n9RPgb1ot?`KlTyfmeeNz literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_date_v3_2_0.snappy.parquet b/spark/src/test/resources/test-data/before_1582_date_v3_2_0.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..25df53eb3d472c78f8ba7f216fe00e70a9468a80 GIT binary patch literal 822 zcmb7DO^ee|6uk|ZAwq=+eUAhJmcWD=h@px0D|Dl{P$^Pmb|UhcypF;2W0F)E+s;b- z7<D%;{0A=F=+<8$f@@bUT)OOLyf29kGAkh`FYn%S&%5V7GPrujB{ES+m3%ll{x0>1 zL}UQD`t05MgaA=VjePyI{&>#7Pnm2I1wb(*08lHSBB4(P<nzhXFXsgT3W9Z95QN5a z#0$hr#1Y~Z;x*z8LcbvCnk-o5LL{96{1X<uGHg(vT*Ex*23?Xv@r9Y*jcPh$$;10; z@PLmQc2A@&Nt44U$hNBeIA>vCV-~Y$Qbwq%8@g@z1`0IgO$E;vd?Hbm#x%2hOEHSh ziZ_Mryw73>x{b4ti3I3aEI_|Wq(;{j;()cxML~!bXXdI;=%KajX|nKhs(HFYQz(YL zqu5U&_Fh6U`lCWsOQF9iHMHH=jA^pxuo*+EPVU-?`5p@nZr<MQwln}uJ41NHlQf!6 zw2jM-=QKduIm1ZzV+SX=Kj)c~OdmUAp0SW+OnX~dnAyRMo4%Q5$vntR+Z;ta4pY;= zyD%qgEK*`{m?()@$Z<EB$1xkmn8!kHn}0M-f3aHGe{1tQ`M=_OE6^^U@lcH6Iv&0o zjk%b_t<ut^aZ)d5P<I{IbKLs&a2~~B-DAz5(F&TK5$kr^qfV1C&vV<2A$PBYU9TNB WN23;Rhda7$0N?{tyqy95F~0#Y%kkI% literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v2_4_5.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c7e8d3926f63ab372d898a40667cb5dc33437ede GIT binary patch literal 737 zcmbVKPm9w)6n{x$37A9FoiQURmH|r_Vn{5TZAvfRME0<F712pE)o9XYlc|3wz54+? z3Vsz&9{d7cJbLs4hzIed*>x>~f{-`K<oD+H=P}2(?}fx54snl&P7DCcgO_9RrU!8O z({&&~Y_jL!VvsH|UB~(z-Q7_b{sjd9)75dcHE7u0Yvk!s!wc|eNAi*dS9%ugVG%5f zb+u<r9dvW1&01%aTswCbAvIhawhEoj>kr?F&Zc*)x)jy3hxP1<ECh;??hV~B(rl*M z&kJQPvIS6^xy&1wF6puZvum)nhv?L%pEdeL8<^AGw7a{1WM>{GHi+KLvv~mwzG~-M zt%<GJTb+K?5S(J3KTpwb9clw}o&9C?*cYY1SpGJlU!UYYMm<?c<yXrWej$}eg%Zv? zZOzrIQt}C}RkfNa?(umh^R(uZ`)j@sMFTYxLEyAeWreJjC`$abSmn7$a%{rW(&J~u ze6rqbyk7*{IsOO3gAF)$PmSL!s{V6X)!A~<5B)Ik!~QT?WqH~c<2ap<=W#r`l|*tL kMWay~L_rkGgJ2dXhw)*M$RrK-ndbuF13xr?W4zoy0e4TQ`2YX_ literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v2_4_6.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v2_4_6.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..939e2b8088eb0f15d870f7881ac421e26187036a GIT binary patch literal 771 zcmbVKL5tHs6n;swC14IwcgBgNSOS(V#39-4W}DKBH<3LoUPW}$Of_rLW|P!aO7H#v zkAi>2lL!BR7mptO0piJ*rt4Z36oh$`nfcy)-}~O=<o+X{XoL~_n5aYpFkE;&6z@6! zH@@5f0>mV{E>0S06WwOUpXm0ELjNx)0O+=gv#CJM_E90H$2Bj&vn|Q%2Hflzu!})3 zD8{XhF=klJg|^jHM#=5V$^xW@YyC!{)q4BsCsD2O1Eb7E@$zXoc_A}_Vx)anWm=p} zs%E*9ry`jFH7(n6d%8{9)}hwcttN-4)TCb(`c3KS<88OwyMK7MTy(4vweDyA0qA{K zF14B>)?)8f`dLA6jDD`(qCZ;H#B;{x#d9ulfwuVCA%7){GMUfV&OSfj0kFL*8Rc2Z zu^DFzS@B|i&NEqwSX6?2P?k>hCYR30DXU^JsT|jtCNhmnXY_dK%tTg0b<YskBvwf# z%SvQ99?ceMDxwsV@V0cF3u0%qT&>*S1nW-!2g8#Uu!oz%?`1`I<FxB@-{XF_A1#tJ x?uua;PlwYm9PCGdoCd*Q5PN|ag!0gvgwauW<V7-y{XNUI0dPPU-{Ay*|6ee6vCIGf literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v3_2_0.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_int96_dict_v3_2_0.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..78120fdb233370826f27e80d26f1fdb5bcbe0afa GIT binary patch literal 842 zcmbVL&x_MQ6n;swrHg^B+BYVWvL#^XLJUo`Kj}sBuojU;_9mj!BweFP+f7n+t-Xi` z1&^NBtB4o93hK$bdJqN0lmCM~Icc`5OTmLMk9nE*&G)_c4cRzz$wUDqRPfiww{Mo) z$fE!t>Tst)pA-R(d|m|v%wqvRy_&vUR#+-v7IOeOg$IB<2g%7h4sYPM*B^eYv=aot zwF4A~ML1d%p}+!J5EjRZVq0SDb0s-dB6u9{e|i3B<u9c}<y0w?dH8-RE2M*G36o~o zE|2R;ioD!j7o#B!Z(fb;>ux|<512i{OM>Isahfg-ed_gr<Yh(HwT^;WxhiCHa*9Eo z<jFnqfYUl!PLafUUx@<;zX+87Q3Y9NehA1Xz9k!wpF9>o+vWc#rmf5*U-0BB$BNox zPN}h>7H&^@BtV{JNM6g4?|gw_E=bBC+%o8pvRRGU*oN<JQG4h7#m#0-0+6Hyz%4h7 zyg^@DI%QZ!1*EkFM6~T2%);%F8yn%^wh_27b!bebr`%YLcZROoQlmH=*|DmtJ<s)> zNNrsnt9=?IrNn7g*aOC1;6^bGhU}<6@_pL%8Hn9>b#+dxw#Ji*zE3cFpZ{REG689I zVVcyCb2N50Su2SroxR~s24QK>jgo1YmSL94-I3=zC5u+=O3kh|dbHW7_Zn47Ez7J| fx~_TJX<Bus+UwQay0a$h3IJPhmp$+X`-^@9clgfM literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v2_4_5.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v2_4_5.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..88a94ac482052a66482aad13059be4897b10b850 GIT binary patch literal 693 zcmbVKO>5gg5FH8E_@IOOv0DjbY8Gm2P=!=Cie-E$<PdTQCf8ER%38IelC4PVw8i*e z<R0kn=^?k|m}8GQ<`gpiift$@6n2)Gowx7JTXyjN#3Mj{Dde3^bj+%xW*HiM@AC6H zz_Y7Yuv<}R{iO>4@qvoFu544>YjCW=Qs-==<0J5o_@^2?uWRrEgJ4jMmvyag;52Jm z6LX@P{X?zHdGYmAIs76rPAcf&wQ5wNWGI*ElRV|g2&isarroSrq-yTfplU8LM5Q|Y zt<p=SS?ljQ-L3xN+qBRf4upo9*UCm@slHITQ|M0xn|-vs1*bO^s^fvt9v7b-o^v$f zEIc{oY3iW4(}|EyG5+RcLh^`9Zd@o+CV%9D^;jv3$xt$z#feCxlJ!1J*@$QJ*8B~y zu>#5@6Q$%?jz^<On({D35Z)FxTNAV1bT+de2^KH+9}FL7U>vOmf0GrBGf|Yuc+~J5 z&vm>;E1V>0)ZpD9io0<Tbl!%(h<(4)iCo|H1F`20gK$6CcS8|IUfZ-S09<gZ01WWM FJ^<Y|rBVO@ literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v2_4_6.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v2_4_6.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..68bfa33aac13f7d8af14dca353195c8f421bdea2 GIT binary patch literal 727 zcmbV~&x_MQ6vtmexAc%hT%B<SQY-_OU5G=n-OV<o7r}#gSQf7$GHE6nOxkQRU6s=R z!n@$#<H4IAJ^Ckj65qDnA4O3R^1_>W-+bTu%;fap(~to9WRXLkxR|v`$72q>9jf<T zfZZ?mV0%Sj?=MvV$R64FcCDKhKOH!6VBP2Zx=)0{Kk9EeaJ%ck9Snj&G46JqF~eyt zZ8_$|=HtgsQ>ps(i)Qjlmx{EI;J(dTX+AOQbf+p*J_YJ>kNf?ON80=d+8w^u5SzO6 zyG=h^{myusbbIv=Ztfu+4unAcuhw;wb@x@59~S*+VR4MuH{|qNi@Lyl#%A?PsVapa za>XRib=~B%Dcd=ahcW`T|C^#YD<tBd&9#yB?2Rn7QK>SDeXth7ysNYriN@6Pi4nfY za$Tg27(H8vsVXl@7pH`UjnL$!Zj34`+?vjdLL~)?@N4ahOJgxwESLV3!Rl=P!{PZ7 z*ptoR56Zf?A?<}S3}o2bOXhiz_S7&=vtbs;gM%c}SriQhX%GcbtdD|8oE*o;L86m1 R+~>XrfDi2A1)Sn9{R=(^uO$Eg literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v3_2_0.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_int96_plain_v3_2_0.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3f5f85e244c051cee27ded3cea17b27348b15af8 GIT binary patch literal 779 zcmbVL&5F}d6h4VFB?CdK_TIUHlqmtj4AjuXN&o0Zn1zZAgUn7uZj+lac++N*baW_v z0XKrUch{BA;1jrWp%383H<*QpC)0LB3JO9FoO{oA&iTIYlB0W%T>|8jLiWjrZ*M+s znWT#63ef<f0~G+O3cTv_cXfc9Uv}W94u?7{I?l+~i_hP$|D)rE4!7z$)G!DJ#n`Uv z1BSy^gAL3LlWHWSxDYc*=1Q3o)fn8@^OR>Vp5?)Fk#Mqs<Zi2MBaDM$k#44v$0MLs z!!(+<W|E5GLq#(d7^2cD{h`p%)&@*pZ7ikyd7`lUmnxD%&BzCUzEw#LZ25c7g1~Cg zT(RY#UCOxXn5%>Wy|_lJwnks8H7q`2<~TcXc*;?hbE(jg;)Dn14-SueZ3e*D8sJQ1 zc|0Dm&AX20w1DlcA@Wn{pj)RCQ8?N7rIUz)hrHnIy)w0m^Hf-UD=)H1P*_bXiiHew ztN&zbjd)U)N>j101Qu~3@`5KRUX3PF@}Wc!c8jL98f^8avsv?Mz~xi?$AqUdU^{DS z7KR+~LL4KN(vycvad(oev2>&1I<DuqjoslSmSMx=e$Z+MemCO1ZYS#coO_<zX$^(D aANIUX=togobizHOX#(H}0w47Ve}i8SEyRBS literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_5.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_5.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..62e6048354dc1026f86af85a5c8b29eb492dfaf8 GIT binary patch literal 767 zcmbV~zl+pB6vyA>;*#QufHxt*6zjk(w-7_(a=FcV2sUzv#c`Ez9x~a?YB1T&C0Rwo zf`urEomg2|SY2gf=Z{LTwiB%HudvXUY}N%=8zIb_`M#IWy!TD^uN=5krYhaIP9-V> zC?>prb8_S8#ZL;AY~do)7FA4LS!9+w4YvNGLWPlVRSBqwc~0mPQh{IrI(+)@6tDOL zw4x&7>RtnCx`K1%mzXH)<-zGA4BxwJFELyq=T9)a`gZvvQTjjL-u#MmXn0gIcK6ov zlxKHu<-<E7;W#LjHzi#TqhYbRF+ZG1V;+t1q^X8!SRKWrvazkSWMf`OB~8`d5$yx% zD7(wfm%D$MVOVIOOeMhas{qjXOy+E88T$O%8vabu9!r?4=qp~-^c58h6R}WZwNGoB zih|yqWVdaea-7w!$F}q1*v9#fr=qa4$$dK!1rK?__1C0i#e-C^0n3YQIxLvQMv;ia zoDB|3Hs(o{s%*kUQ&>ca$P1pND2=Ca%!3$<@NHqS1(*%W+06P4sIC1E!qE)$Yfa-9 zlB|7CWO+0hw_V$HY`47?Ortn#^S&RB`Xk@(T?{-i^1NOzbUerN#kMo_gB^dz2}BUO RUBfZ~@PV`V8TRql{{_RV;WPjM literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_6.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_micros_v2_4_6.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d7fdaa3e67212c1991ba75219827b490c127f43e GIT binary patch literal 801 zcmbW0zl+pB6vyA>az~0QQrv_DQ>=koZXphvTQ|2^55Yzbu{f?0&O;`fSq&z;xg@LH zg#`;y5IeE5u&}zy#?Bv=U~MN@-(O*&FWIaMt~NrLH}icjpLy>avVZNsp)ytJ#!V_w z89*`N{hPB}M=yR-C}kTLnXXaA)RlQ=vD0AdFDjH630IYXikRnwJ|QIt=Agr;r{{RZ zAD|@_5!d%>P~8<=D858QSuYOGA7S|3T6u}#3b}ZO;nlaRABocY@%Hvtq(jZ4lCitD zo+UiJe<vH<6EVj@sk|xaau5#k&5hY%HHmmQ!jq;NreQY~lgdU{UQ>)&9hEdydq=bn zq^ayKI$!MmVTNI&fijf<$1ejw^E0W~P_k#&R`6$%_E^GXNni4!qA#gnn23c6t9@G0 zR4{B^AE$R2PdLuXs<GJvk!Im|q_1wX%ghD6wN#NEM+_Z0o`{^K<A*F3IS+Ww_1C1Z z@{>eZeJjh;$so6EYZ!_s$gKWhVU2iPrb;6*Q5WW6EV7)(2}+|$6mdVoB79re)*Nj0 zi|N$<4XDij55mzD^c!{Km*TWhPug&p(`HU%%b$c%(BM5U81{yq*V*=6F?8KdCuqBE e*Aw0L!1H&!owhIhz-bw_34jlD@k8z7&+!XBE9?dU literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_micros_v3_2_0.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_micros_v3_2_0.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..809f6f50e092698ddb9d3551b88740253935e80e GIT binary patch literal 929 zcmbVL&ubG=5T4y^%L+N9koVZd1>-_P1M8AR^V3i;h*T+3N)JLU@{)aN7dJn;yHRQk zcqn@5O+9(=;6W;iSUd!@B6#rPMZ5@}`%hFvXOnGH$W2(7VZQIpH}eg<bZI$5BoZTW z^7+gDpJ;`ML;{dC`0!%)#@ge*1cu;`y~B5>G)yHjP2_+e0zgcF7!q~_@=zdeUOjm& z1QZxit}KL<uoIl_eRTvm+FL$&fZ<mv@)W~G;p{GkXFuon1-bb9?al8f3agLq9bi;v zL{*WGMmdoZQKbNraQH5SHCiQA<bkTuZGjAgAV;jS5cN9Dy?x88Z}29=g^4sPDw1W_ z{n_~9&WvUt5vf6Ss^c)bb+Sxjs-_yrszxL=Cr!)h=p_+pjQaF`vYM2&@zmpt!rbvL z<0_~Inko?y(6}G~9Yjc+eiX?(B)8E~RDnQcG|5orlRyUof)(r&?mX@|)S0NF2`GT; zqrkTaQRrbHK=*L!q)gw7akObs(c131$vO;;F^3+R4qs>Wt*h5pN_ho9Q6>m?xa-;N zmNIqT%$jLXQWFes(=pNS%`W#%w|&=aa-UhuXUcP-r~6wSu9tPsce{08H}r<h9m~_p zt3AENngJA?1%Z(0+fDBItl7a*tLr$d<{$}Qb3;FBte1QJzH!`Oc)|bZu+|6V%)~Y$ zLoPF)uiz+wC)dK|&Nbac=td%AX0m1`F<0x_j+Mx=Ts@ty=ZXzhDi#{W9AnvRrjV}j c%mu5IEm*lmBhL#~N;Nb9{NN?tX$gPMe`SXTJOBUy literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_5.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_5.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..a7cef9e60f13425560f516f4ffb32f0bb4b1831a GIT binary patch literal 761 zcmbV~F^kkd6vrnqyG(IKP$y&{#WHfs9mJ5hTz0cuBZ!D79JV4d+01IVWH*<YJvr9Z z!sZHXv=9sNOITiEA^0I|#n#sDd`Wg)aJ3P_KQsUL@_X}M=HU9hfEdIim+ugr7zklU z9-<Tf+xa;`mF<EUq)o76<5gt6ve3?7TvTCnP*X!DU|wqEX~jV+&X2A4RfRt|8!ixT z-e@?h((+69SqtMUcVB=$m#3g#-@O9ue|p`*gE!ybR$#eA5uNVe+L8q?4<E|WW07-c zkr>;$Z6xVPZC`9Qts>*;1a_859qM+mLkzly+Xij&h;EtIN6k9Zx_Ezm==JI!W1udK z&LBF1&Z`$f-OpOXMr~?0Wo!7UZvE6CSkX7eP}4VDL><7Q8f%?huuL%Q-D&yA;{}Ia zy}E5r9%UZ%e>4-yE2mGqTqvG!#qAH;oT-ySupyJGoQ)K7**FzhBH8f%oK1LMg{m<@ zV40}WTu8<90<6g_%XpkY5_~OOwgR)^e6etU1M1QK2jRg2*|(PmznYhwV^PX<I_U&n z;Cn%5C!VEQ(&2+B84t!$)VmgkVjPCOUgC#-7>PZ96vh3h@5dref?eu52*HCc!EZQ# GOaBW#&F0kr literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_6.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_millis_v2_4_6.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..4c213f4540a735c041a85fb12bc90810a3f83eb1 GIT binary patch literal 795 zcmbW0L5tHs6vtm;y5z8jAWldihZ5Pc3vo!cHf_pY1X)BB7Ox^QX{H)BX|tJBS!(s* zWe<9B56d3JFX3?y9t1yxSMloA+xjMLEm*w>;h&lJUVd-hKg_|c2LaKDL9X2+D$xLR zM;yYj|LyXUfDC&m8rdMaW9iGxYGuOqU!ou}DypG?ftWW6c_BHl<owutFFX9fS&JfJ z|Mr}-Y%RXDUN&_7`u*3aufz%J*Ymfi-B0ftdhhi6nFNakim3M9olQ}&^6;@3JmopV z5s9{?T3V70s;#T@O{2(II>Mc4*pBVBbcbkmN88Ztc^y$r!~Ce27fMUtTOE3}`iBj+ zht_FC1)N;I09v1wnhlaYZ(GApRP(2TNl9O`p{B1z!FCV}HP*biVj5t3mNhP)P*yOU zmDXd^XS@{Ycw{wp=}j7fwX;?sjxvf49Zh&e%kguX^NJ;`V%7&`>Q=`Ecl)lW%E_Q| zJ$IP$ED>)1;nW?mT&ARvm{=56Y0gE(@&ZGnNtUrV!y^2ed+rkK_NTL%_Zv`~{~v@$ zGqCP14t^sqn~O=C0S$Z_G`HhPnk7xvi;`h)7)9;fIOM}HY_}6X^uvgE{6Q3VqplzG TI0<%a&jG*(I{2dw@LK!=c(3uN literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/before_1582_timestamp_millis_v3_2_0.snappy.parquet b/spark/src/test/resources/test-data/before_1582_timestamp_millis_v3_2_0.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f9cfa4a1c0aa8334fe98dcae1cd7f4f5a62641a5 GIT binary patch literal 922 zcmbVL&x_MQ6rN<$CBlMP=?n=3ERhYnP(u^#Pl{AoL}isl_9iORBwfQMZ8u4kTC1>L z6xNF#R8YJr$g-ek5pVthA_!j9li=OU`X)`c3%z-ndA#p?@0<6{%<AdOCKj-Svv~K# zvukzCV*w#ijW$rf@MCWeqXhf%{_nd(Due=_#Udl{2qB3>5)igId<TE~`S9yu#y}&+ z+2s*q+$j32-_3~Pv5WU%y%#-z_51Tju$DhP&WM#Kzn(I==+E3swkz|zB8%hN10m)W zg3vUa)}^pY8u$$GkfM?;4)1U*2CRN=F6z>7<9cM@a03dXVqt-o1;?}Fg(EB5^QsC& zpt5YH>r=0NFij*yRq}d6#ez~4W<_OGiFqQCklfZAx~NW0J&6cf+@EEVMM@s*6flp# zGqxe}b_Qq3E{{(keTz#}5K|eQB$4^RksS`eI`m0ro^(ufrm|oH2*CIt@Novq<Tn!l z86xQ-dCh0RrWIN3gzE<FQZU9yJTiQDo!b5Lm)5E!86jDoBHVPt$m_J_%n8FX3P@g> zVnmz10e)}x+}H>^w~WAzsY7Ebzv2d3-0!+tO^f2NXUAGzYk98kL|X02Kx@;0LF`%p zNQto*xKT`lE~MH$-=|F<NKozOwQ*yuHW&```wfyC{ErS-he$p;HO<J7Wg5F{a1`dr zg><;%L6}S3$eD&|8D?&=+4FoSXVIcvDA~nIi&iV;R;5U(Wtrtd(=|^zRjce2Tdk5? Tc9xVpJU#aM-oQPr!ms=Xf$<3? literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/dec-in-fixed-len.parquet b/spark/src/test/resources/test-data/dec-in-fixed-len.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6ad37d5639511cdb430f33fa6165eb70cd9034c0 GIT binary patch literal 460 zcmZuu%SyvQ6rI#oO3~7Vw<M577-(rBhCY%QsvCDMM2ff&RLXQ_qS2&}JbX~P^8;M^ zF@As_;wSV2ypakj4s*|)nRD+w2WEJ9+Jzzr5XZ0s<i`^E`5i+sb9l*Ng-|%p;SFKK z$l)0_K?U^^3?LXaoJLHwgp{&cr#fDUp%hEC-r4;t0u(g!Xe*hq_->T&8yPVu5U4^0 z3hK5WJW4SN<zyrzZ&YyJ4yhZOuIbIP4ppsDBif%Tz=HaRg1cEh_(wbE45I03dm3L` zY{pR2`jWSjd1#@Jc_vdUo<3NSOqpOQQ*ReEeVR!+ph+5MJf$Y}0~v~h4lWjS!lIn| z<rOQ|9cVm=SlH@z%w8LxCs`P>aflr_N}BXrgbo(V<<*j?`)dokQEarvSr7`N-{ZFH k+I`P<B=gu_$KicP*phobcO9E!_Pwqv+T=tbWQhOx35yqQV*mgE literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/decimal32-written-as-64-bit-dict.snappy.parquet b/spark/src/test/resources/test-data/decimal32-written-as-64-bit-dict.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e19bd7ae01b853f9376c8e65a17fd8ab8be2bf7f GIT binary patch literal 6795 zcmZWud3+P)x}KRN&C;ZqY;BSz%_QCTBxNaxk`@I-#3U$-MM+XBTY;o4i)>8`A{<1L z0*ZQswxB48Bn3UlW?EGAh>{jW1ckQXg3D=IP_6>ad4}u#>-P71c;EMV*YBH|Z>H2< z=Cr7|s=#d3>5_@60kMip5TBL^`Y3`&a^L3g_yQtM3I&8iA|TEJj|)?{#1;uH(A<gh zNaBP<P2_?eCb1JP2~B8#A(B$!xMU6?moP+!gpu&M#3+f7C_vaWxZabPiEN3DFv7hO z;Xg=ni2G3H8gK=86KDXhLi`BHG}tupFz`><4*}n!)klzZsPilQ1&Ch&|7z5>L9PLR z9R4SOKGc>$vbGcbE+AfsnCGFdhh7ie1fLjjvym@Ftv`{U34SIp7B~oa5%UCQ{uOyv z<edli7;+jRQxX3S@^(Qlfqyb$-vp|mw}5*Gb%)=J^CFHWdcpsEubvvS2#9mQQpq%l zluN9GdlJlRXf;Y=BwmMp9i_)fmJwXYCt>G6o`K{8Cs5iVaiO&tTdO9%M7|FEaPY}! zGZEbVsPzJ595@BvLy!){jfUO>^kO-$!LGu#PJ?U*?qX?H*k06_4>*u>8?jO7xc0<@ zz!b@NA_4L+>Mw_H5`1gX_bqTgVxA(I0RI+)ZHN336-FWIS)dc#P001&vJkx)&;t+P zo@vHvI-&mxsDK<)yaZVWu&aIqIqyR+MOXHs@0WNnrJ69H2fK>3n0yd&H|&?7KLEWA z_!POb5H}HdQy}NUo{BkE0}|LV^!^3hKcF8%t{RfPvBrU8ckw0c#(%&sNBjWl%>kZ* zZwSD}jCuEVOpV?GVjkA;FZhQ7F9JUUIlv+KmI1A(!R|!`W-kT=(AB^a)L<}QHuPKQ z$?nlQ^!x(7*&B|%Q8xlp?#*0Ed<OjsT)lMI{fOHH$zI(ts8a%&3-ExIi2DmZcD<XR zF8~+CjO;`4GW2xFeZU0x1L)HY-&M#r0QP~pgxaSe-+_NU;_Q<3#2?U~1ZM}f0q-Nm z4c{Z+MX<HdOM#=fb0@%N@7C+sCl90u^@k%~0zLxyDmWVQUCiA8JdT;zYhM9wBz(&u ze}{boaxYMY9_&NOfo~Py0@_e}5Wa_@mtjw`;m?QuA^7K@7hzx7JNOOMc0#U#%)-7- zg?<_J)*z-Hz1ejZgZltF2mBY%#aQn}@LJ$O*eR$r1M(`KfD00{B%4cwalS=R(uXv5 zsdJGUgWCe`80<Tc8{un4iAAv4(L@;c3hd`0=fVCIIqbHj071!i9Hlw;j?~MjGatGT zvFwO9Q0r0TT!8&GayCH^gWCeUi`;f_%kC|gzISQav*r%$k=Tmyh}j0Pmvs^F3DAQ4 zFTt^ASroDC_MAqIrxDLS%<L`RC}}3XLfq%*&)%++A^-Dtt{gUdJFrkde-+yrg`Yhf zzC%tGYA=O88$R|inThqBfQ%si0Ju58HW~Y%Pe)`O&;;gVM16sXBdD+s+4mvi7Wh|S zzXv@7aU;OrkG!GKE1{RczYo6Wkf+BKW^iiguftyg{awI?x*K4B2Aw?^hQVg1{2Kad zfIXnb;-ca-9zdxalwpTw&z4`{lA_&C=trP`B6%2R0E1JN#yAYh39y7|7>~xq7vKzF zp9cX-+lYnWOaKq}{zbGO2i=0${qT=O`!&e_8gegu?6TNpb0A-Ve?NNu0zbQ)XHcsZ z@(ldDF<&M8JK^(yW6z<L;18ol8u&2y#o%iYw-<5&I6Hh&$lW;aC~$5pe+T4I<mSP* z4tf*x&B*x*`E$T+fIb#_Eqr4j7otZhB)jG8{3{Ui7;^m3KLGzAFbP~0?6&|0_B_Ns zjk6#N_E6vza4(?_dubNK#&-hth5gJ><0duZp<!RX+wnG2V{h5_1A}Ld#sfB<_!hDn zG2<~8tF!Nm%gC)o><Y}mz9TYWPljY?X3t3q_z*RcA;;mGZUes^^^c-X1>)HE0ejvQ zf@ePv&O%o}vS-6)_zpp`9|Usr{2g{BZ1!Cc#Mux;{z-6B*zB##1AfAsXEBQrdOrB; z$UP6P9P%r0qfsXn`Wuh~$WMX(I<Cnn_-YaV6LP!Y+W~tw>{9qYg>FD>9qO>}K=%B+ zA4mlM55(_5{z>@RgJB7{uW+w~p`Qdy(Af`@W<UwB5At7-?1z*G^$m!56sQN*fO}37 zxOZKyO8j`g{_nng0KOb~J0ZV?WWN-i!9C1=EG<C|6?*T1Zx!};G&lufG~nKWE`$Hy z;6?Di4gM_bWAN!PH~SUxHuSsb6GChV9Q#ptKVs5ge-F6`zU|N-0@y>6-PdDS6MG1q z0Z7yygBm3G4Zv2+wGlaM!F>t)ASC<M5<wihjw7&n*vGe_XG4Dh{qBQb2m58@PDjrG zZ1#E$Mch`%9?Zo+PC(3Sz+=F6<V}ZcMNSWDXQ7rJVE5=%)c6GQ4q~219(#Nd@X7Gb ze+GIr>b{2<cHe&kUIcd<Gj9RsM!l_&?DeSz*l&jp^lt#zZv_Q>1i*e6e*?P<=Sm)O zr=iz1a8`hQzCQljFT`GrNBpy}+53^bW@6<1hCVfa^G?{qG2fr?%?CdWdI+ciPs5&r z+D}6726qrKB(M+j7_jbgaJ%3e0{JI6_8IAdoQs?p_rN}<Ie325L=xtjf}ZS|_)ma6 zt6Bi|+vq9m$qM*CM@<T`e&nz-90kPS|AqL&h&vCg2kOvc7I=0)n=m8mW4}I^AWi^Z z5V8!tUI5R2$CW@%g}fngsR-2_vudxUK;>uf8&HBOfgnz(W0fBe1i`yi?;&}EeIAl? zXF16k97!rx-;T#`UyjEw4S`(>Oadm7`oVbo!@IWzZ^!RmiN}A3bM=#?bO*@z?JF;m zq;z#Wek&fYw~?Ix3?U_hgY|dgA3n_+t=0_2FZadkc}y(cS4#2+<5$L`;981Q4aNuW z*2hup_7akFc_EpI7RT?7f*1Apf8FY{Vzj~C6lpjX|L|7)_U*gzzIfl2c)hrupb5%M zaP33_J}U!+C`5>(L{f~PX;N+`r8ZLKB2)aNrkPBQkos=YKu|`8O1Dv&E-Kqk<u+6K z5vs78DkeA<hErnW2<@C<UQTI%Gcv>(73CPaIin*S8_g{@b04&G$9lQr16)fp_mO5Q zA;Os$<xYxmbp+4B@Fv@M$#&ipFK=3a_hg7SBg%U^#&dZIH<M5q;?1%rJmXE66X4AC zbLNF&@r3GV!opaBH^TEV{MrCnXXh{R@|Okp%R~H?QU2-}-%kq~{hT!sYOP(c&dYz^ zFWAs5*ccIP>K2s81OZyuMDVvToUJzgHka^azwni2VbIKZEh2oqTe#EAT}p6Wgs6oP z?Xrp9aEUg1MQ*?7?Pk%w5N~xvw7*+)faZkg#HCCkV@^C|Pki5;X!j-_4kR9l@)m~@ zLy^SeG4ez#F+z(YjQFHY{IN@X$}ip#5dS;E{X8N*9ua@hEglgQM`=lyS#r)UIq#KR z2uPeba6^)B10*jhxg3+Ih@_=-Qm;Abnmy?UZ_-bJq@P1czef47_;1}weT1~1k>2PQ z-n2{q@JdH8RFhx&XS4L5A?cl{bTB3*WHe==zqipmH_dOLg)Q`v5H0ScZ+6pCQYJIX z6c(Y<A=9~JBCkvxlxf2<U8gLqU&hGfOBs23nLN`iukgxqg7UnuTx^k5M&xt6g2Eno zF{!W^6<KD*5Qk!zPf^++9MPhv2q{K&Dm>ka(WEj>rYyH8OUjfFbqdD%l;eZS31KDE zu6(qN7~Z4Y8&ldzkt0ap{}QmZj~f%7a4769)l?&QT90gci>jiXJEK!|pj$PQOm-WS zE6bAW?8#>%oNs8(Grr_=X4TbL+@CzRGihEpxw<oXVSln$mQrI$sdJ<(@ue&arYsMq ztn5r#9p>GpW%Z<bW=y(;Nool58ZGLzZuxU=b-mxG?uw<XHzqz<mb{@wy|G=rDad=# zp~#nUHiw0LLL;*Y4tP_7M$HyCcWasIZa4Xni`?ecjP+?M9io+PVtY`tu37V%g}xlo z(4Cs^x-~xph@C#7*+I6*h@D~8R7Sh2O#6mK;c#pBScEYfHMvYH_G{mk@%Q#{6>ibK zcJ2Ni?STgEPBPUJ(A3CM+bpSvg5viq+7a&5dKu?mZq4Da^n;dEhNQB?sf+!@F^js? zC_LWJKM{*J@INf0ErhP!BJ6MrP6lNkIdobZJ-bKpzFT*yLDvuxedb8|cUX72Q}<=R zE-KTXvMamVQ_ngi=YoQ-g6hv1*?FJ-Yr87XFTW7fUkvL{NA$V~^<9sCTuk3BOY1F@ zPB06vd$mz>+BF}4OIY>2Tl+&$|D!SGMX&a2f7*3Y^>a_sPvJCMByD-W;<ui(KGHCO zHVjw{7aIgO9SJLfiNCuI6?W+@xB7pAhQGpwJDrAxs8ZWwh`ZAW6Zg?Dk2EnHkAmxD zbZ(|1#49$7`K?Mp6La22EN)@GiKZoXFhiouT}R@C9^T|0UPZ`I(ap?{#o`P>8f6{I zbBs|@lcsbUlWUCXCZo1h^uA9oYfc*FFt&w_x^n8x=H#MIV_K_vvMbR*nH+A}e%iz^ zrgS+c!)eNDO3f~3Ci_ia7)@!tvK;{v>Em5$QRI2l`Hf^jhrTardZ63%K##VVO1HG7 zy%&~Pt?7zh$-Izuh%-H(V1`+R!v~1%F3B*9(CtrWn$kzMrEfF~M-9l5y3#AU(?{13 z6|}0OpBNyt75>D<WX8Py)GZOk(FQ@1Jfqy2@t`%~V@Jl=a&^p`!8B$(Ov%+&&4jj$ zma^oDUD6xTj7b9-c6p}Slrph~a9T4Tm-C*G8)NY)6jd0YMuvIky_wTm)23T9h?Y!E zLrR55dtcO0*q(Vbrhlq8Q`Db1o6OoH%ks3!>nqE%sytaS901<rbYIpSkJ8nY^+>BG zV3BNUSI!#{&F{$4I%O5jsS6q@L(I5PE=URL7xijakop|E#M>zHcXD<#7^XJxD(LJ6 zI-{mH@u615W^;ChoUH3GgqW1YP11F4jk8JF=E@#mbfOq>sb4=PB3#;-{j5BFxttgg z<CR7-R)?}zc4hzJ%3R&6eV{+v-y~Qp6DKuN4UHMk8%2*9b9CmMwO0AlW$Dk!xwqXp zY4)VW45!bRv)&{%4`ke>#j!XU%=t&Jif+ppv<XG+Ih%TOHdDELI(K|O*F^A}thw*U zGN;;d%R;7Yo?N$4Lv>RxH|AC_xl~hbSwD5gFSuirr3JOGQre*b`D=3Kj6=X{*6)hs zuJ6j-(IoNr=N=&PT1<JnoZJk5>Z<b0-Oju>t;8Nm{gz34zDIvEpkEl{Rk%czw$#u2 zr5Q4DHN}<JD6BQw;eou<q<Bwj-bQ0uVmrC7k<@kM?XSsP-<!wxr7-<@6J43cs4yhY zf77OF8xTL&k?^`TKfx`Lm*oeVNuNB0Z_n>@=PTNHlWOv{j!Zt0bhual0hM;ds@&+) z_D9L1vA8Fxr9HpBGe66tJ28-djx5;f)+uX<xrFG*K=O!i;t*MZ)R|*&rWG;;cA4>H zdG1FY%*O+H1MY%T<(X?e%H2NQ0e?ZgRkqih`&nzjvQR;Xk)LhK<a@QVBeK(71#LYA zXQ;vr{<Pzb+HtgWuT!^x6!Gc8+iq!>Non=yKeZH|YgB(_N>%xY%iY3bjbf*(aAXbf zb#LAbdDexd!V5IhL&`3i3XXY=3u6VB+6upwlXN+k4vT(uD@$FP&E?sACgq}A?4<L) zbLL$h(4~b6f9gp)6qTIuXWnceuXG7aL{W!>cv4nW8l`GYMfhyEO6eEX#Nxa;RBpem z=v`-#JxE;Y;EoNY9BoY<=F|i{ML%|BG<A~G{6#ZjI^T6q!OtFUWi#ixUlVl{4(nF^ z(v`fxk^XBFr>L`NXK#_Ji6HywKC1XwM`kr$Jm56VcBbBFQY?=c?$~m^X%MgSNOC;% zr8a_BUcAMepmG*p>(JlnNtkRi{83Zv>CZ2T@IUELi_OWB#^OI)m8+YJkF_hlFE=g@ z7q>-<heeD1mcnIirfW`Zy+u{jl76nI_zipEYFZkn%rk1TI9-NCjG)1vKb=y>9n1?d z^JS;N<m9JVgj2&j>TCjMj9F?o-|!LjjhXE(^Ur?e1Ff;R?5Xy`?|o*$7W0q@wI!;4 z%AdX5Rfu1_UomBkj3$2P$&DCgk2ag9^=2>W)LI;-#9H1|J10LZ7Ecyg{IZmO{{E;r zsf=4;6FzR|C!I}rIjAq~DUJ@9*SZQmxASO)C9{lku~UxDMu%h%l`kJ-T4&2nD6u## zLTR~Wy2En7k$12xb8|pHzeh3CD4y#jX2r6;bXz=8(U+v^ZW$GkCI02JY;`B7Z<ycd zRCl$CPI=TxHwZ7@FF}imX0!(_8=}bvdU+!(nJ+o>2aUQvN#)ZK%MOQByNj!C&DB>@ zTY4?0?P@A!VHDQqP1f|vf{e-5*r$Tp#H^EvJGzuzorYdn(P3MT*lK>DSANPRK1An# z+fn#hSL%rx>lP-vdw_V$X3j~^eZ8ZYm~7bF#+%iw5H?w-v>Wm`n$#dC|723ZnvDH+ zQA@k@4wHM!BQTJ;(stb;hFf^mdQ+Zqz%AR^Z#9q6S`Mb}^6KW&Qijz1RhDaAlXI+w z7$-9++X|C?#S_~sB{w98eaaz|Qzv!i9CW1J>K6><2!_=rRs^zml;K;m^}Tl0ibmc} zf<LR=GW@DJuc1UmX$r<C)K<=^THsbugo>L<sW~$%&6y$<g<m?7{ftJZ(Pa@j6<0Hq zNZ@Mjc~uk~$>+9{$tS9^ZkA*Tbi+bqj^IZQem%jDn*9UJy&o6|K`@~yQ0FeHnO)@e zy3wqt*6XgBUsUU>E`s7)Tvb<8vuJ72f~q?AEO(t-J#&Sr?pbe@X{4#Pu4Zv%ovFa| z%-pK#S+%B-6)Q{&-3wS3#svR&+^nj~xeMIYIVA;_T(AolS690|)d)j=Re|aMDw{^G pSheb@RfJmo|AtvoRZ}~6(Lyz^q{vcaCUSK;OwW)6VaK1L{tM3;+?D_U literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/test-data/decimal32-written-as-64-bit.snappy.parquet b/spark/src/test/resources/test-data/decimal32-written-as-64-bit.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1068a4e4132350c073ddc9b7abe23f8d11907956 GIT binary patch literal 383 zcmZWlJxc>Y5S_gw=OHOXZqGY7K^77cT-Jn;fQOA;3c+AuBNEoTRfBi=xV<P6(r97t z53uwf2v%09Eo|)k8+PhMOyxB1&Aj<|OlS9?PJuoUdQUjDk($&5*l*_mpzxWCKAKy| z;9}`5AXf3^uCamecymY*^t)b1*!#LIAUti3JYsRuQnPQ2*JiCqR;c0ewyDE&l}Ks; z&6f-}*9~i>Wopn>aBR=_{4)47@0Gyxel==1wlgEnGbvj;Eu&)1kIzlG)?zs(C_6TI z+^r7MGa-i(ZwNh<>4ngvSRfgVRVLEGl}J=3BbiCJJ7L-NP_cmNEFFg#<LtDj;z+aL zXu|q3nM>#n>wkTb3VVr+E8DzY#cF>X$MPh`Hk?<SEjF`YGMyez!FB(cxm2m{4f<}O SA?l(ARo_SRYXqQ!um1s0Yh{=K literal 0 HcmV?d00001 diff --git a/spark/src/test/resources/tpch-extended/q1.sql b/spark/src/test/resources/tpch-extended/q1.sql new file mode 100644 index 0000000000..50eebc3422 --- /dev/null +++ b/spark/src/test/resources/tpch-extended/q1.sql @@ -0,0 +1,9 @@ +select + sum(o_custkey) +from + orders +where + o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' +group by + o_orderkey diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala new file mode 100644 index 0000000000..5ead490c24 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -0,0 +1,1262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.util + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.{CometTestBase, DataFrame, Row} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions.expr +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.SESSION_LOCAL_TIMEZONE +import org.apache.spark.sql.types.{Decimal, DecimalType, StructType} + +import org.apache.comet.CometSparkSessionExtensions.{isSpark32, isSpark34Plus} + +class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { + import testImplicits._ + + test("basic data type support") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "tbl") { + // TODO: enable test for unsigned ints + checkSparkAnswerAndOperator( + "select _1, _2, _3, _4, _5, _6, _7, _8, _13, _14, _15, _16, _17, " + + "_18, _19, _20 FROM tbl WHERE _2 > 100") + } + } + } + } + + test("null literals") { + val batchSize = 1000 + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, batchSize) + withParquetTable(path.toString, "tbl") { + val sqlString = "SELECT _4 + null, _15 - null, _16 * null FROM tbl" + val df2 = sql(sqlString) + val rows = df2.collect() + assert(rows.length == batchSize) + assert(rows.forall(_ == Row(null, null, null))) + + checkSparkAnswerAndOperator(sqlString) + } + } + } + } + + test("date and timestamp type literals") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "tbl") { + checkSparkAnswerAndOperator( + "SELECT _4 FROM tbl WHERE " + + "_20 > CAST('2020-01-01' AS DATE) AND _18 < CAST('2020-01-01' AS TIMESTAMP)") + } + } + } + } + + test("dictionary arithmetic") { + // TODO: test ANSI mode + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false", "parquet.enable.dictionary" -> "true") { + withParquetTable((0 until 10).map(i => (i % 5, i % 3)), "tbl") { + checkSparkAnswerAndOperator("SELECT _1 + _2, _1 - _2, _1 * _2, _1 / _2, _1 % _2 FROM tbl") + } + } + } + + test("dictionary arithmetic with scalar") { + withSQLConf("parquet.enable.dictionary" -> "true") { + withParquetTable((0 until 10).map(i => (i % 5, i % 3)), "tbl") { + checkSparkAnswerAndOperator("SELECT _1 + 1, _1 - 1, _1 * 2, _1 / 2, _1 % 2 FROM tbl") + } + } + } + + test("string type and substring") { + withParquetTable((0 until 5).map(i => (i.toString, (i + 100).toString)), "tbl") { + checkSparkAnswerAndOperator("SELECT _1, substring(_2, 2, 2) FROM tbl") + } + } + + test("substring with start < 1") { + withTempPath { _ => + withTable("t") { + sql("create table t (col string) using parquet") + sql("insert into t values('123456')") + checkSparkAnswerAndOperator(sql("select substring(col, 0) from t")) + checkSparkAnswerAndOperator(sql("select substring(col, -1) from t")) + } + } + } + + test("string with coalesce") { + withParquetTable( + (0 until 10).map(i => (i.toString, if (i > 5) None else Some((i + 100).toString))), + "tbl") { + checkSparkAnswerAndOperator( + "SELECT coalesce(_1), coalesce(_1, 1), coalesce(null, _1), coalesce(null, 1), coalesce(_2, _1), coalesce(null) FROM tbl") + } + } + + test("substring with dictionary") { + val data = (0 until 1000) + .map(_ % 5) // reduce value space to trigger dictionary encoding + .map(i => (i.toString, (i + 100).toString)) + withParquetTable(data, "tbl") { + checkSparkAnswerAndOperator("SELECT _1, substring(_2, 2, 2) FROM tbl") + } + } + + test("string_space") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + checkSparkAnswerAndOperator("SELECT space(_1), space(_2) FROM tbl") + } + } + + test("string_space with dictionary") { + val data = (0 until 1000).map(i => Tuple1(i % 5)) + + withSQLConf("parquet.enable.dictionary" -> "true") { + withParquetTable(data, "tbl") { + checkSparkAnswerAndOperator("SELECT space(_1) FROM tbl") + } + } + } + + test("hour, minute, second") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + val expected = makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 10000) + readParquetFile(path.toString) { df => + val query = df.select(expr("hour(_1)"), expr("minute(_1)"), expr("second(_1)")) + + checkAnswer( + query, + expected.map { + case None => + Row(null, null, null) + case Some(i) => + val hour = new java.sql.Timestamp(i).getHours + val minute = new java.sql.Timestamp(i).getMinutes + val second = new java.sql.Timestamp(i).getSeconds + + Row(hour, minute, second) + }) + } + } + } + } + + test("hour on int96 timestamp column") { + import testImplicits._ + + val N = 100 + val ts = "2020-01-01 01:02:03.123456" + Seq(true, false).foreach { dictionaryEnabled => + Seq(false, true).foreach { conversionEnabled => + withSQLConf( + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> "INT96", + SQLConf.PARQUET_INT96_TIMESTAMP_CONVERSION.key -> conversionEnabled.toString) { + withTempPath { path => + Seq + .tabulate(N)(_ => ts) + .toDF("ts1") + .select($"ts1".cast("timestamp").as("ts")) + .repartition(1) + .write + .option("parquet.enable.dictionary", dictionaryEnabled) + .parquet(path.getCanonicalPath) + + checkAnswer( + spark.read.parquet(path.getCanonicalPath).select(expr("hour(ts)")), + Seq.tabulate(N)(_ => Row(1))) + } + } + } + } + } + + test("cast timestamp and timestamp_ntz") { + withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") + makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "timetbl") { + checkSparkAnswerAndOperator( + "SELECT " + + "cast(_2 as timestamp) tz_millis, " + + "cast(_3 as timestamp) ntz_millis, " + + "cast(_4 as timestamp) tz_micros, " + + "cast(_5 as timestamp) ntz_micros " + + " from timetbl") + } + } + } + } + } + + test("cast timestamp and timestamp_ntz to string") { + // TODO: make the test pass for Spark 3.2 & 3.3 + assume(isSpark34Plus) + + withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") + makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 2001) + withParquetTable(path.toString, "timetbl") { + checkSparkAnswerAndOperator( + "SELECT " + + "cast(_2 as string) tz_millis, " + + "cast(_3 as string) ntz_millis, " + + "cast(_4 as string) tz_micros, " + + "cast(_5 as string) ntz_micros " + + " from timetbl") + } + } + } + } + } + + test("cast timestamp and timestamp_ntz to long, date") { + // TODO: make the test pass for Spark 3.2 & 3.3 + assume(isSpark34Plus) + + withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") + makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "timetbl") { + checkSparkAnswerAndOperator( + "SELECT " + + "cast(_2 as long) tz_millis, " + + "cast(_4 as long) tz_micros, " + + "cast(_2 as date) tz_millis_to_date, " + + "cast(_3 as date) ntz_millis_to_date, " + + "cast(_4 as date) tz_micros_to_date, " + + "cast(_5 as date) ntz_micros_to_date " + + " from timetbl") + } + } + } + } + } + + test("trunc") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "date_trunc.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "tbl") { + Seq("YEAR", "YYYY", "YY", "QUARTER", "MON", "MONTH", "MM", "WEEK").foreach { format => + checkSparkAnswerAndOperator(s"SELECT trunc(_20, '$format') from tbl") + } + } + } + } + } + + test("date_trunc") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") + makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "timetbl") { + Seq( + "YEAR", + "YYYY", + "YY", + "MON", + "MONTH", + "MM", + "QUARTER", + "WEEK", + "DAY", + "DD", + "HOUR", + "MINUTE", + "SECOND", + "MILLISECOND", + "MICROSECOND").foreach { format => + checkSparkAnswerAndOperator( + "SELECT " + + s"date_trunc('$format', _0), " + + s"date_trunc('$format', _1), " + + s"date_trunc('$format', _2), " + + s"date_trunc('$format', _4) " + + " from timetbl") + } + } + } + } + } + + test("date_trunc with timestamp_ntz") { + assume(!isSpark32, "timestamp functions for timestamp_ntz have incorrect behavior in 3.2") + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "timestamp_trunc.parquet") + makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 10000) + withParquetTable(path.toString, "timetbl") { + Seq( + "YEAR", + "YYYY", + "YY", + "MON", + "MONTH", + "MM", + "QUARTER", + "WEEK", + "DAY", + "DD", + "HOUR", + "MINUTE", + "SECOND", + "MILLISECOND", + "MICROSECOND").foreach { format => + checkSparkAnswerAndOperator( + "SELECT " + + s"date_trunc('$format', _3), " + + s"date_trunc('$format', _5) " + + " from timetbl") + } + } + } + } + } + + test("date_trunc on int96 timestamp column") { + import testImplicits._ + + val N = 100 + val ts = "2020-01-01 01:02:03.123456" + Seq(true, false).foreach { dictionaryEnabled => + Seq(false, true).foreach { conversionEnabled => + withSQLConf( + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> "INT96", + SQLConf.PARQUET_INT96_TIMESTAMP_CONVERSION.key -> conversionEnabled.toString) { + withTempPath { path => + Seq + .tabulate(N)(_ => ts) + .toDF("ts1") + .select($"ts1".cast("timestamp").as("ts")) + .repartition(1) + .write + .option("parquet.enable.dictionary", dictionaryEnabled) + .parquet(path.getCanonicalPath) + + withParquetTable(path.toString, "int96timetbl") { + Seq( + "YEAR", + "YYYY", + "YY", + "MON", + "MONTH", + "MM", + "QUARTER", + "WEEK", + "DAY", + "DD", + "HOUR", + "MINUTE", + "SECOND", + "MILLISECOND", + "MICROSECOND").foreach { format => + checkSparkAnswer( + "SELECT " + + s"date_trunc('$format', ts )" + + " from int96timetbl") + } + } + } + } + } + } + } + + test("charvarchar") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "char_tbl4" + withTable(table) { + val view = "str_view" + withView(view) { + sql(s"""create temporary view $view as select c, v from values + | (null, null), (null, null), + | (null, 'S'), (null, 'S'), + | ('N', 'N '), ('N', 'N '), + | ('Ne', 'Sp'), ('Ne', 'Sp'), + | ('Net ', 'Spa '), ('Net ', 'Spa '), + | ('NetE', 'Spar'), ('NetE', 'Spar'), + | ('NetEa ', 'Spark '), ('NetEa ', 'Spark '), + | ('NetEas ', 'Spark'), ('NetEas ', 'Spark'), + | ('NetEase', 'Spark-'), ('NetEase', 'Spark-') t(c, v);""".stripMargin) + sql( + s"create table $table(c7 char(7), c8 char(8), v varchar(6), s string) using parquet;") + sql(s"insert into $table select c, c, v, c from $view;") + val df = sql(s"""select substring(c7, 2), substring(c8, 2), + | substring(v, 3), substring(s, 2) from $table;""".stripMargin) + + val expected = Row(" ", " ", "", "") :: + Row(null, null, "", null) :: Row(null, null, null, null) :: + Row("e ", "e ", "", "e") :: Row("et ", "et ", "a ", "et ") :: + Row("etE ", "etE ", "ar", "etE") :: + Row("etEa ", "etEa ", "ark ", "etEa ") :: + Row("etEas ", "etEas ", "ark", "etEas ") :: + Row("etEase", "etEase ", "ark-", "etEase") :: Nil + checkAnswer(df, expected ::: expected) + } + } + } + } + } + + test("char varchar over length values") { + Seq("char", "varchar").foreach { typ => + withTempPath { dir => + withTable("t") { + sql("select '123456' as col").write.format("parquet").save(dir.toString) + sql(s"create table t (col $typ(2)) using parquet location '$dir'") + sql("insert into t values('1')") + checkSparkAnswerAndOperator(sql("select substring(col, 1) from t")) + checkSparkAnswerAndOperator(sql("select substring(col, 0) from t")) + checkSparkAnswerAndOperator(sql("select substring(col, -1) from t")) + } + } + } + } + + test("like (LikeSimplification enabled)") { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter column having values 'Rames _ose', where any character matches for '_' + val query = sql(s"select id from $table where name like 'Rames _ose'") + checkAnswer(query, Row(4) :: Row(5) :: Nil) + + // Filter rows that contains 'rose' in 'name' column + val queryContains = sql(s"select id from $table where name like '%rose%'") + checkAnswer(queryContains, Row(5) :: Nil) + + // Filter rows that starts with 'R' following by any characters + val queryStartsWith = sql(s"select id from $table where name like 'R%'") + checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) + + // Filter rows that ends with 's' following by any characters + val queryEndsWith = sql(s"select id from $table where name like '%s'") + checkAnswer(queryEndsWith, Row(3) :: Nil) + } + } + + test("contains") { + assume(!isSpark32) + + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that contains 'rose' in 'name' column + val queryContains = sql(s"select id from $table where contains (name, 'rose')") + checkAnswer(queryContains, Row(5) :: Nil) + } + } + + test("startswith") { + assume(!isSpark32) + + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that starts with 'R' following by any characters + val queryStartsWith = sql(s"select id from $table where startswith (name, 'R')") + checkAnswer(queryStartsWith, Row(3) :: Row(4) :: Row(5) :: Nil) + } + } + + test("endswith") { + assume(!isSpark32) + + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter rows that ends with 's' following by any characters + val queryEndsWith = sql(s"select id from $table where endswith (name, 's')") + checkAnswer(queryEndsWith, Row(3) :: Nil) + } + } + + test("add overflow (ANSI disable)") { + // Enabling ANSI will cause native engine failure, but as we cannot catch + // native error now, we cannot test it here. + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withParquetTable(Seq((Int.MaxValue, 1)), "tbl") { + checkSparkAnswerAndOperator("SELECT _1 + _2 FROM tbl") + } + } + } + + test("divide by zero (ANSI disable)") { + // Enabling ANSI will cause native engine failure, but as we cannot catch + // native error now, we cannot test it here. + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withParquetTable(Seq((1, 0, 1.0, 0.0)), "tbl") { + checkSparkAnswerAndOperator("SELECT _1 / _2, _3 / _4 FROM tbl") + } + } + } + + test("decimals arithmetic and comparison") { + // TODO: enable Spark 3.2 & 3.3 tests after supporting decimal reminder operation + assume(isSpark34Plus) + + def makeDecimalRDD(num: Int, decimal: DecimalType, useDictionary: Boolean): DataFrame = { + val div = if (useDictionary) 5 else num // narrow the space to make it dictionary encoded + spark + .range(num) + .map(_ % div) + // Parquet doesn't allow column names with spaces, have to add an alias here. + // Minus 500 here so that negative decimals are also tested. + .select( + (($"value" - 500) / 100.0) cast decimal as Symbol("dec1"), + (($"value" - 600) / 100.0) cast decimal as Symbol("dec2")) + .coalesce(1) + } + + Seq(true, false).foreach { dictionary => + Seq(16, 1024).foreach { batchSize => + withSQLConf( + CometConf.COMET_BATCH_SIZE.key -> batchSize.toString, + SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false", + "parquet.enable.dictionary" -> dictionary.toString) { + var combinations = Seq((5, 2), (1, 0), (18, 10), (18, 17), (19, 0), (38, 37)) + // If ANSI mode is on, the combination (1, 1) will cause a runtime error. Otherwise, the + // decimal RDD contains all null values and should be able to read back from Parquet. + + if (!SQLConf.get.ansiEnabled) { + combinations = combinations ++ Seq((1, 1)) + } + + for ((precision, scale) <- combinations) { + withTempPath { dir => + val data = makeDecimalRDD(10, DecimalType(precision, scale), dictionary) + data.write.parquet(dir.getCanonicalPath) + readParquetFile(dir.getCanonicalPath) { df => + { + val decimalLiteral1 = Decimal(1.00) + val decimalLiteral2 = Decimal(123.456789) + val cometDf = df.select( + $"dec1" + $"dec2", + $"dec1" - $"dec2", + $"dec1" % $"dec2", + $"dec1" >= $"dec1", + $"dec1" === "1.0", + $"dec1" + decimalLiteral1, + $"dec1" - decimalLiteral1, + $"dec1" + decimalLiteral2, + $"dec1" - decimalLiteral2) + + checkAnswer( + cometDf, + data + .select( + $"dec1" + $"dec2", + $"dec1" - $"dec2", + $"dec1" % $"dec2", + $"dec1" >= $"dec1", + $"dec1" === "1.0", + $"dec1" + decimalLiteral1, + $"dec1" - decimalLiteral1, + $"dec1" + decimalLiteral2, + $"dec1" - decimalLiteral2) + .collect() + .toSeq) + } + } + } + } + } + } + } + } + + test("scalar decimal arithmetic operations") { + assume(isSpark34Plus) + withTable("tbl") { + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + sql("CREATE TABLE tbl (a INT) USING PARQUET") + sql("INSERT INTO tbl VALUES (0)") + + val combinations = Seq((7, 3), (18, 10), (38, 4)) + for ((precision, scale) <- combinations) { + for (op <- Seq("+", "-", "*", "/", "%")) { + val left = s"CAST(1.00 AS DECIMAL($precision, $scale))" + val right = s"CAST(123.45 AS DECIMAL($precision, $scale))" + + withSQLConf( + "spark.sql.optimizer.excludedRules" -> + "org.apache.spark.sql.catalyst.optimizer.ConstantFolding") { + + checkSparkAnswerAndOperator(s"SELECT $left $op $right FROM tbl") + } + } + } + } + } + } + + test("cast decimals to int") { + Seq(16, 1024).foreach { batchSize => + withSQLConf( + CometConf.COMET_BATCH_SIZE.key -> batchSize.toString, + SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false") { + var combinations = Seq((5, 2), (1, 0), (18, 10), (18, 17), (19, 0), (38, 37)) + // If ANSI mode is on, the combination (1, 1) will cause a runtime error. Otherwise, the + // decimal RDD contains all null values and should be able to read back from Parquet. + + if (!SQLConf.get.ansiEnabled) { + combinations = combinations ++ Seq((1, 1)) + } + + for ((precision, scale) <- combinations; useDictionary <- Seq(false)) { + withTempPath { dir => + val data = makeDecimalRDD(10, DecimalType(precision, scale), useDictionary) + data.write.parquet(dir.getCanonicalPath) + readParquetFile(dir.getCanonicalPath) { df => + { + val cometDf = df.select($"dec".cast("int")) + + // `data` is not read from Parquet, so it doesn't go Comet exec. + checkAnswer(cometDf, data.select($"dec".cast("int")).collect().toSeq) + } + } + } + } + } + } + } + + test("various math scalar functions") { + Seq("true", "false").foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary) { + withParquetTable( + (0 until 5).map(i => (i.toDouble + 0.3, i.toDouble + 0.8)), + "tbl", + withDictionary = dictionary.toBoolean) { + checkSparkAnswerWithTol( + "SELECT abs(_1), acos(_2), asin(_1), atan(_2), atan2(_1, _2), cos(_1) FROM tbl") + checkSparkAnswerWithTol( + "SELECT exp(_1), ln(_2), log10(_1), log2(_1), pow(_1, _2) FROM tbl") + // TODO: comment in the round tests once supported + // checkSparkAnswerWithTol("SELECT round(_1), round(_2) FROM tbl") + checkSparkAnswerWithTol("SELECT signum(_1), sin(_1), sqrt(_1) FROM tbl") + checkSparkAnswerWithTol("SELECT tan(_1) FROM tbl") + } + } + } + } + + test("abs") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, 100) + withParquetTable(path.toString, "tbl") { + Seq(2, 3, 4, 5, 6, 7, 15, 16, 17).foreach { col => + checkSparkAnswerAndOperator(s"SELECT abs(_${col}) FROM tbl") + } + } + } + } + } + + test("ceil and floor") { + Seq("true", "false").foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary) { + withParquetTable( + (-5 until 5).map(i => (i.toDouble + 0.3, i.toDouble + 0.8)), + "tbl", + withDictionary = dictionary.toBoolean) { + checkSparkAnswerAndOperator("SELECT ceil(_1), ceil(_2), floor(_1), floor(_2) FROM tbl") + checkSparkAnswerAndOperator( + "SELECT ceil(0.0), ceil(-0.0), ceil(-0.5), ceil(0.5), ceil(-1.2), ceil(1.2) FROM tbl") + checkSparkAnswerAndOperator( + "SELECT floor(0.0), floor(-0.0), floor(-0.5), floor(0.5), " + + "floor(-1.2), floor(1.2) FROM tbl") + } + withParquetTable( + (-5 until 5).map(i => (i.toLong, i.toLong)), + "tbl", + withDictionary = dictionary.toBoolean) { + checkSparkAnswerAndOperator("SELECT ceil(_1), ceil(_2), floor(_1), floor(_2) FROM tbl") + checkSparkAnswerAndOperator( + "SELECT ceil(0), ceil(-0), ceil(-5), ceil(5), ceil(-1), ceil(1) FROM tbl") + checkSparkAnswerAndOperator( + "SELECT floor(0), floor(-0), floor(-5), floor(5), " + + "floor(-1), floor(1) FROM tbl") + } + withParquetTable( + (-33L to 33L by 3L).map(i => Tuple1(Decimal(i, 21, 1))), // -3.3 ~ +3.3 + "tbl", + withDictionary = dictionary.toBoolean) { + checkSparkAnswerAndOperator("SELECT ceil(_1), floor(_1) FROM tbl") + checkSparkAnswerAndOperator("SELECT ceil(cast(_1 as decimal(20, 0))) FROM tbl") + checkSparkAnswerAndOperator("SELECT floor(cast(_1 as decimal(20, 0))) FROM tbl") + withSQLConf( + // Exclude the constant folding optimizer in order to actually execute the native ceil + // and floor operations for scalar (literal) values. + "spark.sql.optimizer.excludedRules" -> "org.apache.spark.sql.catalyst.optimizer.ConstantFolding") { + for (n <- Seq("0.0", "-0.0", "0.5", "-0.5", "1.2", "-1.2")) { + checkSparkAnswerAndOperator(s"SELECT ceil(cast(${n} as decimal(38, 18))) FROM tbl") + checkSparkAnswerAndOperator(s"SELECT ceil(cast(${n} as decimal(20, 0))) FROM tbl") + checkSparkAnswerAndOperator(s"SELECT floor(cast(${n} as decimal(38, 18))) FROM tbl") + checkSparkAnswerAndOperator(s"SELECT floor(cast(${n} as decimal(20, 0))) FROM tbl") + } + } + } + } + } + } + + test("round") { + assume( + !isSpark32, + "round function for Spark 3.2 does not allow negative target scale and has different result precision/scale for decimals") + + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllTypes( + path, + dictionaryEnabled = dictionaryEnabled, + -128, + 128, + randomSize = 100) + withParquetTable(path.toString, "tbl") { + for (s <- Seq(-5, -1, 0, 1, 5, -1000, 1000, -323, -308, 308, -15, 15, -16, 16, null)) { + // array tests + // TODO: enable test for unsigned ints (_9, _10, _11, _12) + // TODO: enable test for floats (_6, _7, _8, _13) + for (c <- Seq(2, 3, 4, 5, 15, 16, 17)) { + checkSparkAnswerAndOperator(s"select _${c}, round(_${c}, ${s}) FROM tbl") + } + // scalar tests + // Exclude the constant folding optimizer in order to actually execute the native round + // operations for scalar (literal) values. + // TODO: comment in the tests for float once supported + withSQLConf( + "spark.sql.optimizer.excludedRules" -> "org.apache.spark.sql.catalyst.optimizer.ConstantFolding") { + for (n <- Seq("0.0", "-0.0", "0.5", "-0.5", "1.2", "-1.2")) { + checkSparkAnswerAndOperator(s"select round(cast(${n} as tinyint), ${s}) FROM tbl") + // checkSparkAnswerAndCometOperators(s"select round(cast(${n} as float), ${s}) FROM tbl") + checkSparkAnswerAndOperator( + s"select round(cast(${n} as decimal(38, 18)), ${s}) FROM tbl") + checkSparkAnswerAndOperator( + s"select round(cast(${n} as decimal(20, 0)), ${s}) FROM tbl") + } + // checkSparkAnswer(s"select round(double('infinity'), ${s}) FROM tbl") + // checkSparkAnswer(s"select round(double('-infinity'), ${s}) FROM tbl") + // checkSparkAnswer(s"select round(double('NaN'), ${s}) FROM tbl") + // checkSparkAnswer( + // s"select round(double('0.000000000000000000000000000000000001'), ${s}) FROM tbl") + } + } + } + } + } + } + + test("Various String scalar functions") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql( + s"insert into $table values(1, 'James Smith'), (2, 'Michael Rose')," + + " (3, 'Robert Williams'), (4, 'Rames Rose'), (5, 'James Smith')") + checkSparkAnswerAndOperator( + s"SELECT ascii(name), bit_length(name), octet_length(name), upper(name), lower(name) FROM $table") + } + } + } + } + + test("Chr") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col varchar(20)) using parquet") + sql( + s"insert into $table values('65'), ('66'), ('67'), ('68'), ('65'), ('66'), ('67'), ('68')") + checkSparkAnswerAndOperator(s"SELECT chr(col) FROM $table") + } + } + } + } + + test("InitCap") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql( + s"insert into $table values(1, 'james smith'), (2, 'michael rose'), " + + "(3, 'robert williams'), (4, 'rames rose'), (5, 'james smith')") + checkSparkAnswerAndOperator(s"SELECT initcap(name) FROM $table") + } + } + } + } + + test("trim") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col varchar(20)) using parquet") + sql(s"insert into $table values(' SparkSQL '), ('SSparkSQLS')") + + checkSparkAnswerAndOperator(s"SELECT upper(trim(col)) FROM $table") + checkSparkAnswerAndOperator(s"SELECT trim('SL', col) FROM $table") + + checkSparkAnswerAndOperator(s"SELECT upper(btrim(col)) FROM $table") + checkSparkAnswerAndOperator(s"SELECT btrim('SL', col) FROM $table") + + checkSparkAnswerAndOperator(s"SELECT upper(ltrim(col)) FROM $table") + checkSparkAnswerAndOperator(s"SELECT ltrim('SL', col) FROM $table") + + checkSparkAnswerAndOperator(s"SELECT upper(rtrim(col)) FROM $table") + checkSparkAnswerAndOperator(s"SELECT rtrim('SL', col) FROM $table") + } + } + } + } + + // TODO: enable this when we add md5 function to Comet + ignore("md5") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col String) using parquet") + sql( + s"insert into $table values ('test1'), ('test1'), ('test2'), ('test2'), (NULL), ('')") + checkSparkAnswerAndOperator(s"select md5(col) FROM $table") + } + } + } + } + + test("string concat_ws") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "names" + withTable(table) { + sql( + s"create table $table(id int, first_name varchar(20), middle_initial char(1), last_name varchar(20)) using parquet") + sql( + s"insert into $table values(1, 'James', 'B', 'Taylor'), (2, 'Smith', 'C', 'Davis')," + + " (3, NULL, NULL, NULL), (4, 'Smith', 'C', 'Davis')") + checkSparkAnswerAndOperator( + s"SELECT concat_ws(' ', first_name, middle_initial, last_name) FROM $table") + } + } + } + } + + test("string repeat") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1, 'James'), (2, 'Smith'), (3, 'Smith')") + checkSparkAnswerAndOperator(s"SELECT repeat(name, 3) FROM $table") + } + } + } + } + + test("length, reverse, instr, replace, translate") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col string) using parquet") + sql( + s"insert into $table values('Spark SQL '), (NULL), (''), ('苹果手机'), ('Spark SQL '), (NULL), (''), ('苹果手机')") + checkSparkAnswerAndOperator("select length(col), reverse(col), instr(col, 'SQL'), instr(col, '手机'), replace(col, 'SQL', '123')," + + s" replace(col, 'SQL'), replace(col, '手机', '平板'), translate(col, 'SL苹', '123') from $table") + } + } + } + } + + test("EqualNullSafe should preserve comet filter") { + Seq("true", "false").foreach(b => + withParquetTable( + data = (0 until 8).map(i => (i, if (i > 5) None else Some(i % 2 == 0))), + tableName = "tbl", + withDictionary = b.toBoolean) { + // IS TRUE + Seq("SELECT * FROM tbl where _2 is true", "SELECT * FROM tbl where _2 <=> true") + .foreach(s => checkSparkAnswerAndOperator(s)) + + // IS FALSE + Seq("SELECT * FROM tbl where _2 is false", "SELECT * FROM tbl where _2 <=> false") + .foreach(s => checkSparkAnswerAndOperator(s)) + + // IS NOT TRUE + Seq("SELECT * FROM tbl where _2 is not true", "SELECT * FROM tbl where not _2 <=> true") + .foreach(s => checkSparkAnswerAndOperator(s)) + + // IS NOT FALSE + Seq("SELECT * FROM tbl where _2 is not false", "SELECT * FROM tbl where not _2 <=> false") + .foreach(s => checkSparkAnswerAndOperator(s)) + }) + } + + test("bitwise expressions") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col1 int, col2 int) using parquet") + sql(s"insert into $table values(1111, 2)") + sql(s"insert into $table values(1111, 2)") + sql(s"insert into $table values(3333, 4)") + sql(s"insert into $table values(5555, 6)") + + checkSparkAnswerAndOperator( + s"SELECT col1 & col2, col1 | col2, col1 ^ col2 FROM $table") + checkSparkAnswerAndOperator( + s"SELECT col1 & 1234, col1 | 1234, col1 ^ 1234 FROM $table") + checkSparkAnswerAndOperator( + s"SELECT shiftright(col1, 2), shiftright(col1, col2) FROM $table") + checkSparkAnswerAndOperator( + s"SELECT shiftleft(col1, 2), shiftleft(col1, col2) FROM $table") + checkSparkAnswerAndOperator(s"SELECT ~(11), ~col1, ~col2 FROM $table") + } + } + } + } + + test("test in/not in") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql( + s"insert into $table values(1, 'James'), (1, 'Jones'), (2, 'Smith'), (3, 'Smith')," + + "(NULL, 'Jones'), (4, NULL)") + + checkSparkAnswerAndOperator(s"SELECT * FROM $table WHERE id in (1, 2, 4, NULL)") + checkSparkAnswerAndOperator( + s"SELECT * FROM $table WHERE name in ('Smith', 'Brown', NULL)") + + // TODO: why with not in, the plan is only `LocalTableScan`? + checkSparkAnswer(s"SELECT * FROM $table WHERE id not in (1)") + checkSparkAnswer(s"SELECT * FROM $table WHERE name not in ('Smith', 'Brown', NULL)") + } + } + } + } + + test("case_when") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(id int) using parquet") + sql(s"insert into $table values(1), (NULL), (2), (2), (3), (3), (4), (5), (NULL)") + checkSparkAnswerAndOperator( + s"SELECT CASE WHEN id > 2 THEN 3333 WHEN id > 1 THEN 2222 ELSE 1111 END FROM $table") + checkSparkAnswerAndOperator( + s"SELECT CASE WHEN id > 2 THEN NULL WHEN id > 1 THEN 2222 ELSE 1111 END FROM $table") + checkSparkAnswerAndOperator( + s"SELECT CASE id WHEN 1 THEN 1111 WHEN 2 THEN 2222 ELSE 3333 END FROM $table") + checkSparkAnswerAndOperator( + s"SELECT CASE id WHEN 1 THEN 1111 WHEN 2 THEN 2222 ELSE NULL END FROM $table") + checkSparkAnswerAndOperator( + s"SELECT CASE id WHEN 1 THEN 1111 WHEN 2 THEN 2222 WHEN 3 THEN 3333 WHEN 4 THEN 4444 END FROM $table") + checkSparkAnswerAndOperator( + s"SELECT CASE id WHEN NULL THEN 0 WHEN 1 THEN 1111 WHEN 2 THEN 2222 ELSE 3333 END FROM $table") + } + } + } + } + + test("not") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col1 int, col2 boolean) using parquet") + sql(s"insert into $table values(1, false), (2, true), (3, true), (3, false)") + checkSparkAnswerAndOperator(s"SELECT col1, col2, NOT(col2), !(col2) FROM $table") + } + } + } + } + + test("negative") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col1 int) using parquet") + sql(s"insert into $table values(1), (2), (3), (3)") + checkSparkAnswerAndOperator(s"SELECT negative(col1), -(col1) FROM $table") + } + } + } + } + + test("conditional expressions") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test1" + withTable(table) { + sql(s"create table $table(c1 int, c2 string, c3 int) using parquet") + sql( + s"insert into $table values(1, 'comet', 1), (2, 'comet', 3), (null, 'spark', 4)," + + " (null, null, 4), (2, 'spark', 3), (2, 'comet', 3)") + checkSparkAnswerAndOperator(s"SELECT if (c1 < 2, 1111, 2222) FROM $table") + checkSparkAnswerAndOperator(s"SELECT if (c1 < c3, 1111, 2222) FROM $table") + checkSparkAnswerAndOperator( + s"SELECT if (c2 == 'comet', 'native execution', 'non-native execution') FROM $table") + } + } + } + } + + test("basic arithmetic") { + withSQLConf("parquet.enable.dictionary" -> "false") { + withParquetTable((1 until 10).map(i => (i, i + 1)), "tbl", false) { + checkSparkAnswerAndOperator("SELECT _1 + _2, _1 - _2, _1 * _2, _1 / _2, _1 % _2 FROM tbl") + } + } + + withSQLConf("parquet.enable.dictionary" -> "false") { + withParquetTable((1 until 10).map(i => (i.toFloat, i.toFloat + 0.5)), "tbl", false) { + checkSparkAnswerAndOperator("SELECT _1 + _2, _1 - _2, _1 * _2, _1 / _2, _1 % _2 FROM tbl") + } + } + + withSQLConf("parquet.enable.dictionary" -> "false") { + withParquetTable((1 until 10).map(i => (i.toDouble, i.toDouble + 0.5d)), "tbl", false) { + checkSparkAnswerAndOperator("SELECT _1 + _2, _1 - _2, _1 * _2, _1 / _2, _1 % _2 FROM tbl") + } + } + } + + test("date partition column does not forget date type") { + withTable("t1") { + sql("CREATE TABLE t1(flag LONG, cal_dt DATE) USING PARQUET PARTITIONED BY (cal_dt)") + sql(""" + |INSERT INTO t1 VALUES + |(2, date'2021-06-27'), + |(2, date'2021-06-28'), + |(2, date'2021-06-29'), + |(2, date'2021-06-30')""".stripMargin) + checkSparkAnswerAndOperator(sql("SELECT CAST(cal_dt as STRING) FROM t1")) + checkSparkAnswer("SHOW PARTITIONS t1") + } + } + + test("Year") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col timestamp) using parquet") + sql(s"insert into $table values (now()), (null)") + checkSparkAnswerAndOperator(s"SELECT year(col) FROM $table") + } + } + } + } + + test("Decimal binary ops multiply is aligned to Spark") { + assume(isSpark34Plus) + Seq(true, false).foreach { allowPrecisionLoss => + withSQLConf( + "spark.sql.decimalOperations.allowPrecisionLoss" -> allowPrecisionLoss.toString) { + + testSingleLineQuery( + "select cast(1.23456 as decimal(10,9)) c1, cast(2.345678 as decimal(10,9)) c2", + "select a, b, typeof(a), typeof(b) from (select c1 * 2.345678 a, c2 * c1 b from tbl)", + s"basic_positive_numbers (allowPrecisionLoss = ${allowPrecisionLoss})") + + testSingleLineQuery( + "select cast(1.23456 as decimal(10,9)) c1, cast(-2.345678 as decimal(10,9)) c2", + "select a, b, typeof(a), typeof(b) from (select c1 * -2.345678 a, c2 * c1 b from tbl)", + s"basic_neg_numbers (allowPrecisionLoss = ${allowPrecisionLoss})") + + testSingleLineQuery( + "select cast(1.23456 as decimal(10,9)) c1, cast(0 as decimal(10,9)) c2", + "select a, b, typeof(a), typeof(b) from (select c1 * 0.0 a, c2 * c1 b from tbl)", + s"zero (allowPrecisionLoss = ${allowPrecisionLoss})") + + testSingleLineQuery( + "select cast(1.23456 as decimal(10,9)) c1, cast(1 as decimal(10,9)) c2", + "select a, b, typeof(a), typeof(b) from (select c1 * 1.0 a, c2 * c1 b from tbl)", + s"identity (allowPrecisionLoss = ${allowPrecisionLoss})") + + testSingleLineQuery( + "select cast(123456789.1234567890 as decimal(20,10)) c1, cast(987654321.9876543210 as decimal(20,10)) c2", + "select a, b, typeof(a), typeof(b) from (select c1 * cast(987654321.9876543210 as decimal(20,10)) a, c2 * c1 b from tbl)", + s"large_numbers (allowPrecisionLoss = ${allowPrecisionLoss})") + + testSingleLineQuery( + "select cast(0.00000000123456789 as decimal(20,19)) c1, cast(0.00000000987654321 as decimal(20,19)) c2", + "select a, b, typeof(a), typeof(b) from (select c1 * cast(0.00000000987654321 as decimal(20,19)) a, c2 * c1 b from tbl)", + s"small_numbers (allowPrecisionLoss = ${allowPrecisionLoss})") + + testSingleLineQuery( + "select cast(64053151420411946063694043751862251568 as decimal(38,0)) c1, cast(12345 as decimal(10,0)) c2", + "select a, b, typeof(a), typeof(b) from (select c1 * cast(12345 as decimal(10,0)) a, c2 * c1 b from tbl)", + s"overflow_precision (allowPrecisionLoss = ${allowPrecisionLoss})") + + testSingleLineQuery( + "select cast(6.4053151420411946063694043751862251568 as decimal(38,37)) c1, cast(1.2345 as decimal(10,9)) c2", + "select a, b, typeof(a), typeof(b) from (select c1 * cast(1.2345 as decimal(10,9)) a, c2 * c1 b from tbl)", + s"overflow_scale (allowPrecisionLoss = ${allowPrecisionLoss})") + + testSingleLineQuery( + """ + |select cast(6.4053151420411946063694043751862251568 as decimal(38,37)) c1, cast(1.2345 as decimal(10,9)) c2 + |union all + |select cast(1.23456 as decimal(10,9)) c1, cast(1 as decimal(10,9)) c2 + |""".stripMargin, + "select a, typeof(a) from (select c1 * c2 a from tbl)", + s"mixed_errs_and_results (allowPrecisionLoss = ${allowPrecisionLoss})") + } + } + } + + // tests one liner query without necessity to create external table + def testSingleLineQuery( + prepareQuery: String, + testQuery: String, + testName: String = "test", + tableName: String = "tbl"): Unit = { + + withTempDir { dir => + val path = new Path(dir.toURI.toString, testName).toUri.toString + var data: java.util.List[Row] = new util.ArrayList() + var schema: StructType = null + + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val df = spark.sql(prepareQuery) + data = df.collectAsList() + schema = df.schema + } + + spark.createDataFrame(data, schema).repartition(1).write.parquet(path) + readParquetFile(path, Some(schema)) { df => df.createOrReplaceTempView(tableName) } + checkSparkAnswerAndOperator(testQuery) + } + } + + test("Decimal random number tests") { + val rand = scala.util.Random + def makeNum(p: Int, s: Int): String = { + val int1 = rand.nextLong() + val int2 = rand.nextLong().abs + val frac1 = rand.nextLong().abs + val frac2 = rand.nextLong().abs + s"$int1$int2".take(p - s + (int1 >>> 63).toInt) + "." + s"$frac1$frac2".take(s) + } + + val table = "test" + (0 until 10).foreach { _ => + val p1 = rand.nextInt(38) + 1 // 1 <= p1 <= 38 + val s1 = rand.nextInt(p1 + 1) // 0 <= s1 <= p1 + val p2 = rand.nextInt(38) + 1 + val s2 = rand.nextInt(p2 + 1) + + withTable(table) { + sql(s"create table $table(a decimal($p1, $s1), b decimal($p2, $s2)) using parquet") + val values = + (0 until 10).map(_ => s"(${makeNum(p1, s1)}, ${makeNum(p2, s2)})").mkString(",") + sql(s"insert into $table values $values") + Seq(true, false).foreach { allowPrecisionLoss => + withSQLConf( + "spark.sql.decimalOperations.allowPrecisionLoss" -> allowPrecisionLoss.toString) { + val a = makeNum(p1, s1) + val b = makeNum(p2, s2) + var ops = Seq("+", "-") + if (isSpark34Plus) { + // These operations are only supported in Spark 3.4+ + ops = ops ++ Seq("*", "/", "%") + } + for (op <- ops) { + checkSparkAnswerAndOperator(s"select a, b, a $op b from $table") + checkSparkAnswerAndOperator(s"select $a, b, $a $op b from $table") + checkSparkAnswerAndOperator(s"select a, $b, a $op $b from $table") + checkSparkAnswerAndOperator( + s"select $a, $b, decimal($a) $op decimal($b) from $table") + } + } + } + } + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometSparkSessionExtensionsSuite.scala b/spark/src/test/scala/org/apache/comet/CometSparkSessionExtensionsSuite.scala new file mode 100644 index 0000000000..2c818b97a1 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometSparkSessionExtensionsSuite.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +class CometSparkSessionExtensionsSuite extends CometTestBase { + test("unsupported Spark types") { + Seq( + NullType -> false, + BooleanType -> true, + ByteType -> true, + ShortType -> true, + IntegerType -> true, + LongType -> true, + FloatType -> true, + DoubleType -> true, + BinaryType -> true, + StringType -> true, + ArrayType(TimestampType) -> false, + StructType( + Seq( + StructField("f1", DecimalType.SYSTEM_DEFAULT), + StructField("f2", StringType))) -> false, + MapType(keyType = LongType, valueType = DateType) -> false, + StructType(Seq(StructField("f1", ByteType), StructField("f2", StringType))) -> false, + MapType(keyType = IntegerType, valueType = BinaryType) -> false).foreach { + case (dt, expected) => + assert(CometSparkSessionExtensions.isTypeSupported(dt) == expected) + } + } + + test("unsupported Spark schema") { + Seq( + Seq(StructField("f1", IntegerType), StructField("f2", BooleanType)) -> true, + Seq(StructField("f1", IntegerType), StructField("f2", ArrayType(IntegerType))) -> false, + Seq( + StructField("f1", MapType(keyType = LongType, valueType = StringType)), + StructField("f2", ArrayType(DoubleType))) -> false).foreach { case (schema, expected) => + assert(CometSparkSessionExtensions.isSchemaSupported(StructType(schema)) == expected) + } + } + + test("isCometEnabled") { + val conf = new SQLConf + + conf.setConfString(CometConf.COMET_ENABLED.key, "false") + assert(!CometSparkSessionExtensions.isCometEnabled(conf)) + + // Since the native lib is probably already loaded due to previous tests, we reset it here + NativeBase.setLoaded(false) + + conf.setConfString(CometConf.COMET_ENABLED.key, "true") + val oldProperty = System.getProperty("os.name") + System.setProperty("os.name", "foo") + assert(!CometSparkSessionExtensions.isCometEnabled(conf)) + + System.setProperty("os.name", oldProperty) + + conf.setConf(SQLConf.PARQUET_INT96_TIMESTAMP_CONVERSION, true) + assert(!CometSparkSessionExtensions.isCometEnabled(conf)) + + // Restore the original state + NativeBase.setLoaded(true) + } + + test("Arrow properties") { + NativeBase.setLoaded(false) + NativeBase.load() + + assert(System.getProperty(NativeBase.ARROW_UNSAFE_MEMORY_ACCESS) == "true") + assert(System.getProperty(NativeBase.ARROW_NULL_CHECK_FOR_GET) == "false") + + System.setProperty(NativeBase.ARROW_UNSAFE_MEMORY_ACCESS, "false") + NativeBase.setLoaded(false) + NativeBase.load() + assert(System.getProperty(NativeBase.ARROW_UNSAFE_MEMORY_ACCESS) == "false") + + // Should not enable when debug mode is on + System.clearProperty(NativeBase.ARROW_UNSAFE_MEMORY_ACCESS) + SQLConf.get.setConfString(CometConf.COMET_DEBUG_ENABLED.key, "true") + NativeBase.setLoaded(false) + NativeBase.load() + assert(System.getProperty(NativeBase.ARROW_UNSAFE_MEMORY_ACCESS) == null) + + // Restore the original state + NativeBase.setLoaded(true) + SQLConf.get.setConfString(CometConf.COMET_DEBUG_ENABLED.key, "false") + } + + def getBytesFromMib(mib: Long): Long = mib * 1024 * 1024 + + test("Minimum Comet memory overhead") { + val conf = new SparkConf() + assert( + CometSparkSessionExtensions + .getCometMemoryOverhead(conf) == getBytesFromMib(384)) + } + + test("Comet memory overhead factor with executor memory") { + val sparkConf = new SparkConf() + sparkConf.set("spark.executor.memory", "16g") + sparkConf.set(CometConf.COMET_MEMORY_OVERHEAD_FACTOR.key, "0.5") + + assert( + CometSparkSessionExtensions + .getCometMemoryOverhead(sparkConf) == getBytesFromMib(8 * 1024)) + } + + test("Comet memory overhead factor with default executor memory") { + val sparkConf = new SparkConf() + sparkConf.set(CometConf.COMET_MEMORY_OVERHEAD_FACTOR.key, "0.5") + assert( + CometSparkSessionExtensions + .getCometMemoryOverhead(sparkConf) == getBytesFromMib(512)) + } + + test("Comet memory overhead") { + val sparkConf = new SparkConf() + sparkConf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "10g") + assert( + CometSparkSessionExtensions + .getCometMemoryOverhead(sparkConf) == getBytesFromMib(1024 * 10)) + } +} diff --git a/spark/src/test/scala/org/apache/comet/TestUtils.scala b/spark/src/test/scala/org/apache/comet/TestUtils.scala new file mode 100644 index 0000000000..d4e7715683 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/TestUtils.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.io.File + +import scala.collection.mutable.ArrayBuffer + +object TestUtils { + + /** + * Spark 3.3.0 moved {{{SpecificParquetRecordReaderBase.listDirectory}}} to + * {{{org.apache.spark.TestUtils.listDirectory}}}. Copying it here to bridge the difference + * between Spark 3.2.0 and 3.3.0 TODO: remove after dropping Spark 3.2.0 support and use + * {{{org.apache.spark.TestUtils.listDirectory}}} + */ + def listDirectory(path: File): Array[String] = { + val result = ArrayBuffer.empty[String] + if (path.isDirectory) { + path.listFiles.foreach(f => result.appendAll(listDirectory(f))) + } else { + val c = path.getName.charAt(0) + if (c != '.' && c != '_') result.append(path.getAbsolutePath) + } + result.toArray + } +} diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala new file mode 100644 index 0000000000..f0cc96a6b5 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -0,0 +1,740 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.exec + +import scala.util.Random + +import org.apache.hadoop.fs.Path +import org.apache.parquet.example.data.simple.SimpleGroup +import org.apache.parquet.schema.MessageTypeParser +import org.apache.spark.sql.{CometTestBase, DataFrame, Row} +import org.apache.spark.sql.comet.CometHashAggregateExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus + +/** + * Test suite dedicated to Comet native aggregate operator + */ +class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { + + test("Fix NPE in partial decimal sum") { + val table = "tbl" + withTable(table) { + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + withTable(table) { + sql(s"CREATE TABLE $table(col DECIMAL(5, 2)) USING PARQUET") + sql(s"INSERT INTO TABLE $table VALUES (CAST(12345.01 AS DECIMAL(5, 2)))") + val df = sql(s"SELECT SUM(col + 100000.01) FROM $table") + checkAnswer(df, Row(null)) + } + } + } + } + + test("trivial case") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable((0 until 5).map(i => (i, i)), "tbl", dictionaryEnabled) { + val df1 = sql("SELECT _2, SUM(_1) FROM tbl GROUP BY _2") + checkAnswer(df1, Row(0, 0) :: Row(1, 1) :: Row(2, 2) :: Row(3, 3) :: Row(4, 4) :: Nil) + + val df2 = sql("SELECT _2, COUNT(_1) FROM tbl GROUP BY _2") + checkAnswer(df2, Row(0, 1) :: Row(1, 1) :: Row(2, 1) :: Row(3, 1) :: Row(4, 1) :: Nil) + + val df3 = sql("SELECT COUNT(_1), COUNT(_2) FROM tbl") + checkAnswer(df3, Row(5, 5) :: Nil) + + checkSparkAnswer("SELECT _2, MIN(_1), MAX(_1) FROM tbl GROUP BY _2") + } + } + } + + test("avg") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable( + (0 until 10).map(i => ((i + 1) * (i + 1), (i + 1) / 2)), + "tbl", + dictionaryEnabled) { + + checkSparkAnswer("SELECT _2, AVG(_1) FROM tbl GROUP BY _2") + checkSparkAnswer("SELECT AVG(_2) FROM tbl") + } + } + } + + test("count, avg with null") { + Seq(false, true).foreach { dictionary => + withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + val table = "test" + withTable(table) { + sql(s"create table $table(col1 int, col2 int) using parquet") + sql(s"insert into $table values(1, 1), (2, 1), (3, 2), (null, 2), (null, 1)") + checkSparkAnswer(s"SELECT COUNT(col1) FROM $table") + checkSparkAnswer(s"SELECT col2, COUNT(col1) FROM $table GROUP BY col2") + checkSparkAnswer(s"SELECT avg(col1) FROM $table") + checkSparkAnswer(s"SELECT col2, avg(col1) FROM $table GROUP BY col2") + } + } + } + } + + test("SUM/AVG non-decimal overflow") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable(Seq((0, 100.toLong), (0, Long.MaxValue)), "tbl", dictionaryEnabled) { + checkSparkAnswer("SELECT SUM(_2) FROM tbl GROUP BY _1") + checkSparkAnswer("SELECT AVG(_2) FROM tbl GROUP BY _1") + } + } + } + + test("simple SUM, COUNT, MIN, MAX, AVG with non-distinct group keys") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable((0 until 5).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { + val df1 = sql("SELECT _2, SUM(_1) FROM tbl GROUP BY _2") + checkAnswer(df1, Row(0, 6) :: Row(1, 4) :: Nil) + val df2 = sql("SELECT _2, COUNT(_1) FROM tbl GROUP BY _2") + checkAnswer(df2, Row(0, 3) :: Row(1, 2) :: Nil) + checkSparkAnswer("SELECT _2, MIN(_1), MAX(_1) FROM tbl GROUP BY _2") + checkSparkAnswer("SELECT _2, AVG(_1) FROM tbl GROUP BY _2") + } + } + } + + test("group-by on variable length types") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable((0 until 100).map(i => (i, (i % 10).toString)), "tbl", dictionaryEnabled) { + val n = 1 + checkSparkAnswerAndNumOfAggregates("SELECT _2, SUM(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, COUNT(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, MIN(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, MAX(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, AVG(_1) FROM tbl GROUP BY _2", n) + } + } + } + + test("simple SUM, COUNT, MIN, MAX, AVG with non-distinct + null group keys") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable( + (0 until 10).map { i => + (i, if (i % 3 == 0) null.asInstanceOf[Int] else i % 3) + }, + "tbl", + dictionaryEnabled) { + val df1 = sql("SELECT _2, SUM(_1) FROM tbl GROUP BY _2") + checkAnswer(df1, Row(null.asInstanceOf[Int], 18) :: Row(1, 12) :: Row(2, 15) :: Nil) + + val df2 = sql("SELECT _2, COUNT(_1) FROM tbl GROUP BY _2") + checkAnswer(df2, Row(null.asInstanceOf[Int], 4) :: Row(1, 3) :: Row(2, 3) :: Nil) + + val df3 = sql("SELECT _2, MIN(_1), MAX(_1) FROM tbl GROUP BY _2") + checkAnswer(df3, Row(null.asInstanceOf[Int], 0, 9) :: Row(1, 1, 7) :: Row(2, 2, 8) :: Nil) + checkSparkAnswer(sql("SELECT _2, AVG(_1) FROM tbl GROUP BY _2")) + } + } + } + + test("simple SUM, COUNT, MIN, MAX, AVG with null aggregates") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable( + (0 until 10).map { i => + ( + if (i % 2 == 0) null.asInstanceOf[Int] else i, + if (i % 3 == 0) null.asInstanceOf[Int] else i % 3) + }, + "tbl", + dictionaryEnabled) { + val df1 = sql("SELECT _2, SUM(_1) FROM tbl GROUP BY _2") + checkAnswer(df1, Row(null.asInstanceOf[Int], 12) :: Row(1, 8) :: Row(2, 5) :: Nil) + + val df2 = sql("SELECT _2, COUNT(_1) FROM tbl GROUP BY _2") + checkAnswer(df2, Row(null.asInstanceOf[Int], 4) :: Row(1, 3) :: Row(2, 3) :: Nil) + + val df3 = sql("SELECT _2, MIN(_1), MAX(_1) FROM tbl GROUP BY _2") + checkAnswer(df3, Row(null.asInstanceOf[Int], 0, 9) :: Row(1, 0, 7) :: Row(2, 0, 5) :: Nil) + + checkSparkAnswer(sql("SELECT _2, AVG(_1) FROM tbl GROUP BY _2")) + } + } + } + + test("simple SUM, MIN, MAX, AVG with all nulls") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable( + (0 until 10).map { i => + (null.asInstanceOf[Int], if (i % 3 == 0) null.asInstanceOf[Int] else i % 3) + }, + "tbl", + dictionaryEnabled) { + val df = sql("SELECT _2, SUM(_1) FROM tbl GROUP BY _2") + checkAnswer( + df, + Seq( + Row(null.asInstanceOf[Int], null.asInstanceOf[Int]), + Row(1, null.asInstanceOf[Int]), + Row(2, null.asInstanceOf[Int]))) + + val df2 = sql("SELECT _2, MIN(_1), MAX(_1) FROM tbl GROUP BY _2") + checkAnswer( + df2, + Seq( + Row(null.asInstanceOf[Int], null.asInstanceOf[Int], null.asInstanceOf[Int]), + Row(1, null.asInstanceOf[Int], null.asInstanceOf[Int]), + Row(2, null.asInstanceOf[Int], null.asInstanceOf[Int]))) + checkSparkAnswer(sql("SELECT _2, SUM(_1) FROM tbl GROUP BY _2")) + } + } + } + + test("SUM, COUNT, MIN, MAX, AVG on float & double") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test") + makeParquetFile(path, 1000, 10, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + checkSparkAnswer( + "SELECT _g5, SUM(_5), COUNT(_5), MIN(_5), MAX(_5), AVG(_5) FROM tbl GROUP BY _g5") + checkSparkAnswer( + "SELECT _g6, SUM(_6), COUNT(_6), MIN(_6), MAX(_6), AVG(_6) FROM tbl GROUP BY _g6") + } + } + } + } + + test("SUM, MIN, MAX, AVG for NaN, -0.0 and 0.0") { + // NaN should be grouped together, and -0.0 and 0.0 should also be grouped together + Seq(true, false).foreach { dictionaryEnabled => + val data: Seq[(Float, Int)] = Seq( + (Float.NaN, 1), + (-0.0.asInstanceOf[Float], 2), + (0.0.asInstanceOf[Float], 3), + (Float.NaN, 4)) + withParquetTable(data, "tbl", dictionaryEnabled) { + checkSparkAnswer("SELECT SUM(_2), MIN(_2), MAX(_2), _1 FROM tbl GROUP BY _1") + checkSparkAnswer("SELECT MIN(_1), MAX(_1), MIN(_2), MAX(_2) FROM tbl") + checkSparkAnswer("SELECT AVG(_2), _1 FROM tbl GROUP BY _1") + checkSparkAnswer("SELECT AVG(_1), AVG(_2) FROM tbl") + } + } + } + + test("SUM/MIN/MAX/AVG on decimal") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test") + makeParquetFile(path, 1000, 10, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + checkSparkAnswer("SELECT _g1, SUM(_7), MIN(_7), MAX(_7), AVG(_7) FROM tbl GROUP BY _g1") + checkSparkAnswer("SELECT _g1, SUM(_8), MIN(_8), MAX(_8), AVG(_8) FROM tbl GROUP BY _g1") + checkSparkAnswer("SELECT _g1, SUM(_9), MIN(_9), MAX(_9), AVG(_9) FROM tbl GROUP BY _g1") + } + } + } + } + + test("multiple SUM/MIN/MAX/AVG on decimal and non-decimal") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test") + makeParquetFile(path, 1000, 10, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + checkSparkAnswer( + "SELECT _g1, COUNT(_6), COUNT(_7), SUM(_6), SUM(_7), MIN(_6), MIN(_7), MAX(_6), MAX(_7), AVG(_6), AVG(_7) FROM tbl GROUP BY _g1") + checkSparkAnswer( + "SELECT _g1, COUNT(_7), COUNT(_8), SUM(_7), SUM(_8), MIN(_7), MIN(_8), MAX(_7), MAX(_8), AVG(_7), AVG(_8) FROM tbl GROUP BY _g1") + checkSparkAnswer( + "SELECT _g1, COUNT(_8), COUNT(_9), SUM(_8), SUM(_9), MIN(_8), MIN(_9), MAX(_8), MAX(_9), AVG(_8), AVG(_9) FROM tbl GROUP BY _g1") + checkSparkAnswer( + "SELECT _g1, COUNT(_9), COUNT(_1), SUM(_9), SUM(_1), MIN(_9), MIN(_1), MAX(_9), MAX(_1), AVG(_9), AVG(_1) FROM tbl GROUP BY _g1") + } + } + } + } + + test("SUM/AVG on decimal with different precisions") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test") + makeParquetFile(path, 1000, 10, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + Seq("SUM", "AVG").foreach { FN => + checkSparkAnswer( + s"SELECT _g1, $FN(_8 + CAST(1 AS DECIMAL(20, 10))) FROM tbl GROUP BY _g1") + checkSparkAnswer( + s"SELECT _g1, $FN(_8 - CAST(-1 AS DECIMAL(10, 3))) FROM tbl GROUP BY _g1") + checkSparkAnswer( + s"SELECT _g1, $FN(_9 * CAST(3.14 AS DECIMAL(4, 3))) FROM tbl GROUP BY _g1") + checkSparkAnswer( + s"SELECT _g1, $FN(_9 / CAST(1.2345 AS DECIMAL(35, 10))) FROM tbl GROUP BY _g1") + } + } + } + } + } + + test("SUM decimal with DF") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test") + makeParquetFile(path, 1000, 20, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + val expectedNumOfCometAggregates = 1 + + checkSparkAnswerAndNumOfAggregates( + "SELECT _g2, SUM(_7) FROM tbl GROUP BY _g2", + expectedNumOfCometAggregates) + checkSparkAnswerAndNumOfAggregates( + "SELECT _g3, SUM(_8) FROM tbl GROUP BY _g3", + expectedNumOfCometAggregates) + checkSparkAnswerAndNumOfAggregates( + "SELECT _g4, SUM(_9) FROM tbl GROUP BY _g4", + expectedNumOfCometAggregates) + checkSparkAnswerAndNumOfAggregates( + "SELECT SUM(_7) FROM tbl", + expectedNumOfCometAggregates) + checkSparkAnswerAndNumOfAggregates( + "SELECT SUM(_8) FROM tbl", + expectedNumOfCometAggregates) + checkSparkAnswerAndNumOfAggregates( + "SELECT SUM(_9) FROM tbl", + expectedNumOfCometAggregates) + } + } + } + } + + test("COUNT/MIN/MAX on date, timestamp") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test") + makeParquetFile(path, 1000, 10, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + checkSparkAnswer("SELECT _g1, COUNT(_10), MIN(_10), MAX(_10) FROM tbl GROUP BY _g1") + checkSparkAnswer("SELECT _g1, COUNT(_11), MIN(_11), MAX(_11) FROM tbl GROUP BY _g1") + checkSparkAnswer("SELECT _g1, COUNT(_12), MIN(_12), MAX(_12) FROM tbl GROUP BY _g1") + } + } + } + } + + test("single group-by column + aggregate column, multiple batches, no null") { + val numValues = 10000 + + Seq(1, 100, 10000).foreach { numGroups => + Seq(128, 1024, numValues + 1).foreach { batchSize => + Seq(true, false).foreach { dictionaryEnabled => + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + withParquetTable( + (0 until numValues).map(i => (i, Random.nextInt() % numGroups)), + "tbl", + dictionaryEnabled) { + checkSparkAnswer( + "SELECT _2, SUM(_1), MIN(_1), MAX(_1), COUNT(_1), AVG(_1) FROM tbl GROUP BY _2") + } + } + } + } + } + } + + test("multiple group-by columns + single aggregate column, with nulls") { + val numValues = 10000 + + Seq(1, 100, numValues).foreach { numGroups => + Seq(128, numValues + 100).foreach { batchSize => + Seq(true, false).foreach { dictionaryEnabled => + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + withTempPath { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFile(path, numValues, numGroups, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + checkSparkAnswer("SELECT _g1, _g2, SUM(_3) FROM tbl GROUP BY _g1, _g2") + checkSparkAnswer("SELECT _g1, _g2, COUNT(_3) FROM tbl GROUP BY _g1, _g2") + checkSparkAnswer("SELECT _g1, _g2, MIN(_3), MAX(_3) FROM tbl GROUP BY _g1, _g2") + checkSparkAnswer("SELECT _g1, _g2, AVG(_3) FROM tbl GROUP BY _g1, _g2") + } + } + } + } + } + } + } + + test("string should be supported") { + withTable("t") { + sql("CREATE TABLE t(v VARCHAR(3), i INT) USING PARQUET") + sql("INSERT INTO t VALUES ('c', 1)") + checkSparkAnswerAndNumOfAggregates("SELECT v, sum(i) FROM t GROUP BY v ORDER BY v", 1) + } + } + + test("multiple group-by columns + multiple aggregate column, with nulls") { + val numValues = 10000 + + Seq(1, 100, numValues).foreach { numGroups => + Seq(128, numValues + 100).foreach { batchSize => + Seq(true, false).foreach { dictionaryEnabled => + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + withTempPath { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFile(path, numValues, numGroups, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + checkSparkAnswer("SELECT _g3, _g4, SUM(_3), SUM(_4) FROM tbl GROUP BY _g3, _g4") + checkSparkAnswer( + "SELECT _g3, _g4, COUNT(_3), COUNT(_4) FROM tbl GROUP BY _g3, _g4") + checkSparkAnswer( + "SELECT _g3, _g4, MIN(_3), MAX(_3), MIN(_4), MAX(_4) FROM tbl GROUP BY _g3, _g4") + checkSparkAnswer("SELECT _g3, _g4, AVG(_3), AVG(_4) FROM tbl GROUP BY _g3, _g4") + } + } + } + } + } + } + } + + test("all types, with nulls") { + val numValues = 2048 + + Seq(1, 100, numValues).foreach { numGroups => + Seq(true, false).foreach { dictionaryEnabled => + withTempPath { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFile(path, numValues, numGroups, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + Seq(128, numValues + 100).foreach { batchSize => + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + + // Test all combinations of different aggregation & group-by types + (1 to 4).foreach { col => + (1 to 14).foreach { gCol => + checkSparkAnswer(s"SELECT _g$gCol, SUM(_$col) FROM tbl GROUP BY _g$gCol") + checkSparkAnswer(s"SELECT _g$gCol, COUNT(_$col) FROM tbl GROUP BY _g$gCol") + checkSparkAnswer( + s"SELECT _g$gCol, MIN(_$col), MAX(_$col) FROM tbl GROUP BY _g$gCol") + checkSparkAnswer(s"SELECT _g$gCol, AVG(_$col) FROM tbl GROUP BY _g$gCol") + } + } + } + } + } + } + } + } + } + + test("test final count") { + Seq(false, true).foreach { dictionaryEnabled => + val n = 1 + withParquetTable((0 until 5).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { + checkSparkAnswerAndNumOfAggregates("SELECT _2, COUNT(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("select count(_1) from tbl", n) + checkSparkAnswerAndNumOfAggregates( + "SELECT _2, COUNT(_1), SUM(_1) FROM tbl GROUP BY _2", + n) + checkSparkAnswerAndNumOfAggregates("SELECT COUNT(_1), COUNT(_2) FROM tbl", n) + } + } + } + + test("test final min/max") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable((0 until 5).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { + val n = 1 + checkSparkAnswerAndNumOfAggregates( + "SELECT _2, MIN(_1), MAX(_1), COUNT(_1) FROM tbl GROUP BY _2", + n) + checkSparkAnswerAndNumOfAggregates("SELECT MIN(_1), MAX(_1), COUNT(_1) FROM tbl", 1) + checkSparkAnswerAndNumOfAggregates( + "SELECT _2, MIN(_1), MAX(_1), COUNT(_1), SUM(_1) FROM tbl GROUP BY _2", + n) + checkSparkAnswerAndNumOfAggregates( + "SELECT MIN(_1), MIN(_2), MAX(_1), MAX(_2), COUNT(_1), COUNT(_2) FROM tbl", + n) + } + } + } + + test("test final min/max/count with result expressions") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable((0 until 5).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { + val n = 1 + checkSparkAnswerAndNumOfAggregates( + "SELECT _2, MIN(_1) + 2, COUNT(_1) FROM tbl GROUP BY _2", + n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, COUNT(_1) + 2 FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2 + 2, COUNT(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, MIN(_1) + MAX(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, MIN(_1) + _2 FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates( + "SELECT _2 + 2, MIN(_1), MAX(_1), COUNT(_1) FROM tbl GROUP BY _2", + n) + checkSparkAnswerAndNumOfAggregates( + "SELECT _2, MIN(_1), MAX(_1) + 2, COUNT(_1) FROM tbl GROUP BY _2", + n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, SUM(_1) + 2 FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2 + 2, SUM(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT _2, SUM(_1 + 1) FROM tbl GROUP BY _2", n) + + // result expression is unsupported by Comet, so only partial aggregation should be used + val df = sql( + "SELECT _2, MIN(_1) + java_method('java.lang.Math', 'random') " + + "FROM tbl GROUP BY _2") + assert(getNumCometHashAggregate(df) == 1) + } + } + } + + test("test final sum") { + Seq(false, true).foreach { dictionaryEnabled => + val n = 1 + withParquetTable((0L until 5L).map(i => (i, i % 2)), "tbl", dictionaryEnabled) { + checkSparkAnswerAndNumOfAggregates("SELECT _2, SUM(_1), MIN(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT SUM(_1) FROM tbl", n) + checkSparkAnswerAndNumOfAggregates( + "SELECT _2, MIN(_1), MAX(_1), COUNT(_1), SUM(_1), AVG(_1) FROM tbl GROUP BY _2", + n) + checkSparkAnswerAndNumOfAggregates( + "SELECT MIN(_1), MIN(_2), MAX(_1), MAX(_2), COUNT(_1), COUNT(_2), SUM(_1), SUM(_2) FROM tbl", + n) + } + } + } + + test("test final avg") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable( + (0 until 5).map(i => (i.toDouble, i.toDouble % 2)), + "tbl", + dictionaryEnabled) { + val n = 1 + checkSparkAnswerAndNumOfAggregates("SELECT _2 , AVG(_1) FROM tbl GROUP BY _2", n) + checkSparkAnswerAndNumOfAggregates("SELECT AVG(_1) FROM tbl", n) + checkSparkAnswerAndNumOfAggregates( + "SELECT _2, MIN(_1), MAX(_1), COUNT(_1), SUM(_1), AVG(_1) FROM tbl GROUP BY _2", + n) + checkSparkAnswerAndNumOfAggregates( + "SELECT MIN(_1), MIN(_2), MAX(_1), MAX(_2), COUNT(_1), COUNT(_2), SUM(_1), SUM(_2), AVG(_1), AVG(_2) FROM tbl", + n) + } + } + } + + test("final decimal avg") { + // TODO: enable decimal average for Spark 3.2 & 3.3 + assume(isSpark34Plus) + + Seq(true, false).foreach { dictionaryEnabled => + withSQLConf("parquet.enable.dictionary" -> dictionaryEnabled.toString) { + val table = "t1" + withTable(table) { + sql(s"create table $table(a decimal(38, 37), b INT) using parquet") + sql(s"insert into $table values(-0.0000000000000000000000000000000000002, 1)") + sql(s"insert into $table values(-0.0000000000000000000000000000000000002, 1)") + sql(s"insert into $table values(-0.0000000000000000000000000000000000004, 2)") + sql(s"insert into $table values(-0.0000000000000000000000000000000000004, 2)") + sql(s"insert into $table values(-0.00000000000000000000000000000000000002, 3)") + sql(s"insert into $table values(-0.00000000000000000000000000000000000002, 3)") + sql(s"insert into $table values(-0.00000000000000000000000000000000000004, 4)") + sql(s"insert into $table values(-0.00000000000000000000000000000000000004, 4)") + sql(s"insert into $table values(0.13344406545919155429936259114971302408, 5)") + sql(s"insert into $table values(0.13344406545919155429936259114971302408, 5)") + + val n = 1 + checkSparkAnswerAndNumOfAggregates("SELECT b , AVG(a) FROM t1 GROUP BY b", n) + checkSparkAnswerAndNumOfAggregates("SELECT AVG(a) FROM t1", n) + checkSparkAnswerAndNumOfAggregates( + "SELECT b, MIN(a), MAX(a), COUNT(a), SUM(a), AVG(a) FROM t1 GROUP BY b", + n) + checkSparkAnswerAndNumOfAggregates( + "SELECT MIN(a), MAX(a), COUNT(a), SUM(a), AVG(a) FROM t1", + n) + } + } + } + } + + test("test partial avg") { + Seq(true, false).foreach { dictionaryEnabled => + withParquetTable( + (0 until 5).map(i => (i.toDouble, i.toDouble % 2)), + "tbl", + dictionaryEnabled) { + checkSparkAnswerAndNumOfAggregates("SELECT _2 , AVG(_1) FROM tbl GROUP BY _2", 1) + } + } + } + + test("avg null handling") { + val table = "t1" + withTable(table) { + sql(s"create table $table(a double, b double) using parquet") + sql(s"insert into $table values(1, 1.0)") + sql(s"insert into $table values(null, null)") + sql(s"insert into $table values(1, 2.0)") + sql(s"insert into $table values(null, null)") + sql(s"insert into $table values(2, null)") + sql(s"insert into $table values(2, null)") + + val query = sql(s"select a, AVG(b) from $table GROUP BY a") + checkSparkAnswer(query) + } + } + + test("Decimal Avg with DF") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test") + makeParquetFile(path, 1000, 20, dictionaryEnabled) + withParquetTable(path.toUri.toString, "tbl") { + val expectedNumOfCometAggregates = 1 + + checkSparkAnswerAndNumOfAggregates( + "SELECT _g2, AVG(_7) FROM tbl GROUP BY _g2", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTol("SELECT _g3, AVG(_8) FROM tbl GROUP BY _g3") + assert( + getNumCometHashAggregate( + sql("SELECT _g3, AVG(_8) FROM tbl GROUP BY _g3")) == expectedNumOfCometAggregates) + + checkSparkAnswerAndNumOfAggregates( + "SELECT _g4, AVG(_9) FROM tbl GROUP BY _g4", + expectedNumOfCometAggregates) + + checkSparkAnswerAndNumOfAggregates( + "SELECT AVG(_7) FROM tbl", + expectedNumOfCometAggregates) + + checkSparkAnswerWithTol("SELECT AVG(_8) FROM tbl") + assert( + getNumCometHashAggregate( + sql("SELECT AVG(_8) FROM tbl")) == expectedNumOfCometAggregates) + + checkSparkAnswerAndNumOfAggregates( + "SELECT AVG(_9) FROM tbl", + expectedNumOfCometAggregates) + } + } + } + } + + protected def checkSparkAnswerAndNumOfAggregates(query: String, numAggregates: Int): Unit = { + val df = sql(query) + checkSparkAnswer(df) + val actualNumAggregates = getNumCometHashAggregate(df) + assert( + actualNumAggregates == numAggregates, + s"Expected $numAggregates Comet aggregate operators, but found $actualNumAggregates") + } + + def getNumCometHashAggregate(df: DataFrame): Int = { + val sparkPlan = stripAQEPlan(df.queryExecution.executedPlan) + sparkPlan.collect { case s: CometHashAggregateExec => s }.size + } + + def makeParquetFile( + path: Path, + total: Int, + numGroups: Int, + dictionaryEnabled: Boolean): Unit = { + val schemaStr = + """ + |message root { + | optional INT32 _1(INT_8); + | optional INT32 _2(INT_16); + | optional INT32 _3; + | optional INT64 _4; + | optional FLOAT _5; + | optional DOUBLE _6; + | optional INT32 _7(DECIMAL(5, 2)); + | optional INT64 _8(DECIMAL(18, 10)); + | optional FIXED_LEN_BYTE_ARRAY(16) _9(DECIMAL(38, 37)); + | optional INT64 _10(TIMESTAMP(MILLIS,true)); + | optional INT64 _11(TIMESTAMP(MICROS,true)); + | optional INT32 _12(DATE); + | optional INT32 _g1(INT_8); + | optional INT32 _g2(INT_16); + | optional INT32 _g3; + | optional INT64 _g4; + | optional FLOAT _g5; + | optional DOUBLE _g6; + | optional INT32 _g7(DECIMAL(5, 2)); + | optional INT64 _g8(DECIMAL(18, 10)); + | optional FIXED_LEN_BYTE_ARRAY(16) _g9(DECIMAL(38, 37)); + | optional INT64 _g10(TIMESTAMP(MILLIS,true)); + | optional INT64 _g11(TIMESTAMP(MICROS,true)); + | optional INT32 _g12(DATE); + | optional BINARY _g13(UTF8); + | optional BINARY _g14; + |} + """.stripMargin + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter(schema, path, dictionaryEnabled = true) + + val rand = scala.util.Random + val expected = (0 until total).map { i => + // use a single value for the first page, to make sure dictionary encoding kicks in + if (rand.nextBoolean()) None + else { + if (dictionaryEnabled) Some(i % 10) else Some(i) + } + } + + expected.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i.toByte) + record.add(1, i.toShort) + record.add(2, i) + record.add(3, i.toLong) + record.add(4, rand.nextFloat()) + record.add(5, rand.nextDouble()) + record.add(6, i) + record.add(7, i.toLong) + record.add(8, (i % 10).toString * 16) + record.add(9, i.toLong) + record.add(10, i.toLong) + record.add(11, i) + record.add(12, i.toByte % numGroups) + record.add(13, i.toShort % numGroups) + record.add(14, i % numGroups) + record.add(15, i.toLong % numGroups) + record.add(16, rand.nextFloat()) + record.add(17, rand.nextDouble()) + record.add(18, i) + record.add(19, i.toLong) + record.add(20, (i % 10).toString * 16) + record.add(21, i.toLong) + record.add(22, i.toLong) + record.add(23, i) + record.add(24, (i % 10).toString * 24) + record.add(25, (i % 10).toString * 36) + case _ => + } + writer.write(record) + } + + writer.close() + } +} diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala new file mode 100644 index 0000000000..c8e31ef4c3 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -0,0 +1,804 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.exec + +import scala.util.Random + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.{AnalysisException, Column, CometTestBase, DataFrame, DataFrameWriter, Row, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStatistics, CatalogTable} +import org.apache.spark.sql.catalyst.expressions.Hex +import org.apache.spark.sql.comet.{CometFilterExec, CometHashAggregateExec, CometProjectExec, CometScanExec} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.joins.{BroadcastNestedLoopJoinExec, CartesianProductExec, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.SESSION_LOCAL_TIMEZONE +import org.apache.spark.unsafe.types.UTF8String + +import org.apache.comet.CometConf + +class CometExecSuite extends CometTestBase { + import testImplicits._ + + test("scalar subquery") { + val dataTypes = + Seq( + "BOOLEAN", + "BYTE", + "SHORT", + "INT", + "BIGINT", + "FLOAT", + "DOUBLE", + // "DATE": TODO: needs to address issue #1364 first + // "TIMESTAMP", TODO: needs to address issue #1364 first + "STRING", + "BINARY", + "DECIMAL(38, 10)") + dataTypes.map { subqueryType => + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + var column1 = s"CAST(max(_1) AS $subqueryType)" + if (subqueryType == "BINARY") { + // arrow-rs doesn't support casting integer to binary yet. + // We added it to upstream but it's not released yet. + column1 = "CAST(CAST(max(_1) AS STRING) AS BINARY)" + } + + val df1 = sql(s"SELECT (SELECT $column1 FROM tbl) AS a, _1, _2 FROM tbl") + checkSparkAnswerAndOperator(df1) + + var column2 = s"CAST(_1 AS $subqueryType)" + if (subqueryType == "BINARY") { + // arrow-rs doesn't support casting integer to binary yet. + // We added it to upstream but it's not released yet. + column2 = "CAST(CAST(_1 AS STRING) AS BINARY)" + } + + val df2 = sql(s"SELECT _1, _2 FROM tbl WHERE $column2 > (SELECT $column1 FROM tbl)") + checkSparkAnswerAndOperator(df2) + + // Non-correlated exists subquery will be rewritten to scalar subquery + val df3 = sql( + "SELECT * FROM tbl WHERE EXISTS " + + s"(SELECT $column2 FROM tbl WHERE _1 > 1)") + checkSparkAnswerAndOperator(df3) + + // Null value + column1 = s"CAST(NULL AS $subqueryType)" + if (subqueryType == "BINARY") { + column1 = "CAST(CAST(NULL AS STRING) AS BINARY)" + } + + val df4 = sql(s"SELECT (SELECT $column1 FROM tbl LIMIT 1) AS a, _1, _2 FROM tbl") + checkSparkAnswerAndOperator(df4) + } + } + } + + test("Comet native metrics: project and filter") { + withSQLConf( + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + val df = sql("SELECT _1 + 1, _2 + 2 FROM tbl WHERE _1 > 3") + df.collect() + + var metrics = find(df.queryExecution.executedPlan) { + case _: CometProjectExec => true + case _ => false + }.map(_.metrics).get + + assert(metrics.contains("output_rows")) + assert(metrics("output_rows").value == 1L) + + metrics = find(df.queryExecution.executedPlan) { + case _: CometFilterExec => true + case _ => false + }.map(_.metrics).get + + assert(metrics.contains("output_rows")) + assert(metrics("output_rows").value == 1L) + } + } + } + + test("expand operator") { + val data1 = (0 until 1000) + .map(_ % 5) // reduce value space to trigger dictionary encoding + .map(i => (i, i + 100, i + 10)) + val data2 = (0 until 5).map(i => (i, i + 1, i * 1000)) + + Seq(data1, data2).foreach { tableData => + withParquetTable(tableData, "tbl") { + val df = sql("SELECT _1, _2, SUM(_3) FROM tbl GROUP BY _1, _2 GROUPING SETS ((_1), (_2))") + checkSparkAnswerAndOperator(df, classOf[HashAggregateExec], classOf[ShuffleExchangeExec]) + } + } + } + + test("multiple distinct multiple columns sets") { + withTable("agg2") { + val data2 = Seq[(Integer, Integer, Integer)]( + (1, 10, -10), + (null, -60, 60), + (1, 30, -30), + (1, 30, 30), + (2, 1, 1), + (null, -10, 10), + (2, -1, null), + (2, 1, 1), + (2, null, 1), + (null, 100, -10), + (3, null, 3), + (null, null, null), + (3, null, null)).toDF("key", "value1", "value2") + data2.write.saveAsTable("agg2") + + val df = spark.sql(""" + |SELECT + | key, + | count(distinct value1), + | sum(distinct value1), + | count(distinct value2), + | sum(distinct value2), + | count(distinct value1, value2), + | count(value1), + | sum(value1), + | count(value2), + | sum(value2), + | count(*), + | count(1) + |FROM agg2 + |GROUP BY key + """.stripMargin) + + // The above query uses COUNT(DISTINCT) which Comet doesn't support yet, so the plan will + // have a mix of `HashAggregate` and `CometHashAggregate`. In the following we check all + // operators starting from `CometHashAggregate` are native. + checkSparkAnswer(df) + val subPlan = stripAQEPlan(df.queryExecution.executedPlan).collectFirst { + case s: CometHashAggregateExec => s + } + assert(subPlan.isDefined) + checkCometOperators(subPlan.get) + } + } + + test("transformed cometPlan") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + val df = sql("select * FROM tbl where _1 >= 2").select("_1") + checkSparkAnswerAndOperator(df) + } + } + + test("project") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + val df = sql("SELECT _1 + 1, _2 + 2, _1 - 1, _2 * 2, _2 / 2 FROM tbl") + checkSparkAnswerAndOperator(df) + } + } + + test("project + filter on arrays") { + withParquetTable((0 until 5).map(i => (i, i)), "tbl") { + val df = sql("SELECT _1 FROM tbl WHERE _1 == _2") + checkSparkAnswerAndOperator(df) + } + } + + test("project + filter") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + val df = sql("SELECT _1 + 1, _2 + 2 FROM tbl WHERE _1 > 3") + checkSparkAnswerAndOperator(df) + } + } + + test("empty projection") { + withParquetDataFrame((0 until 5).map(i => (i, i + 1))) { df => + assert(df.where("_1 IS NOT NULL").count() == 5) + checkSparkAnswerAndOperator(df) + } + } + + test("filter on string") { + withParquetTable((0 until 5).map(i => (i, i.toString)), "tbl") { + val df = sql("SELECT _1 + 1, _2 FROM tbl WHERE _2 = '3'") + checkSparkAnswerAndOperator(df) + } + } + + test("filter on dictionary string") { + val data = (0 until 1000) + .map(_ % 5) // reduce value space to trigger dictionary encoding + .map(i => (i.toString, (i + 100).toString)) + + withParquetTable(data, "tbl") { + val df = sql("SELECT _1, _2 FROM tbl WHERE _1 = '3'") + checkSparkAnswerAndOperator(df) + } + } + + test("sort with dictionary") { + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> 8192.toString) { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test") + spark + .createDataFrame((0 until 1000).map(i => (i % 5, (i % 7).toLong))) + .write + .option("compression", "none") + .parquet(path.toString) + + spark + .createDataFrame((0 until 1000).map(i => (i % 3 + 7, (i % 13 + 10).toLong))) + .write + .option("compression", "none") + .mode(SaveMode.Append) + .parquet(path.toString) + + val df = spark.read + .format("parquet") + .load(path.toString) + .sortWithinPartitions($"_1".asc, $"_2".desc) + + checkSparkAnswerAndOperator(df) + } + } + } + + test("final aggregation") { + withParquetTable( + (0 until 100) + .map(_ => (Random.nextInt(), Random.nextInt() % 5)), + "tbl") { + val df = sql("SELECT _2, COUNT(*) FROM tbl GROUP BY _2") + checkSparkAnswerAndOperator(df, classOf[HashAggregateExec], classOf[ShuffleExchangeExec]) + } + } + + test("sort (non-global)") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + val df = sql("SELECT * FROM tbl").sortWithinPartitions($"_1".desc) + checkSparkAnswerAndOperator(df) + } + } + + test("spill sort with (multiple) dictionaries") { + withSQLConf(CometConf.COMET_MEMORY_OVERHEAD.key -> "15MB") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawTimeParquetFileColumns(path, dictionaryEnabled = true, n = 1000, rowGroupSize = 10) + readParquetFile(path.toString) { df => + Seq( + $"_0".desc_nulls_first, + $"_0".desc_nulls_last, + $"_0".asc_nulls_first, + $"_0".asc_nulls_last).foreach { colOrder => + val query = df.sortWithinPartitions(colOrder) + checkSparkAnswerAndOperator(query) + } + } + } + } + } + + test("spill sort with (multiple) dictionaries on mixed columns") { + withSQLConf(CometConf.COMET_MEMORY_OVERHEAD.key -> "15MB") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawTimeParquetFile(path, dictionaryEnabled = true, n = 1000, rowGroupSize = 10) + readParquetFile(path.toString) { df => + Seq( + $"_6".desc_nulls_first, + $"_6".desc_nulls_last, + $"_6".asc_nulls_first, + $"_6".asc_nulls_last).foreach { colOrder => + // TODO: We should be able to sort on dictionary timestamp column + val query = df.sortWithinPartitions(colOrder) + checkSparkAnswerAndOperator(query) + } + } + } + } + } + + test("limit (cartesian product)") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_a") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_b") { + val df = sql("SELECT tbl_a._1, tbl_b._2 FROM tbl_a JOIN tbl_b LIMIT 2") + checkSparkAnswerAndOperator( + df, + classOf[CollectLimitExec], + classOf[CartesianProductExec]) + } + } + } + } + + test("limit with more than one batch") { + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> "1") { + withParquetTable((0 until 50).map(i => (i, i + 1)), "tbl_a") { + withParquetTable((0 until 50).map(i => (i, i + 1)), "tbl_b") { + val df = sql("SELECT tbl_a._1, tbl_b._2 FROM tbl_a JOIN tbl_b LIMIT 2") + checkSparkAnswerAndOperator( + df, + classOf[CollectLimitExec], + classOf[BroadcastNestedLoopJoinExec], + classOf[BroadcastExchangeExec]) + } + } + } + } + + test("limit less than rows") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_a") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl_b") { + val df = sql( + "SELECT tbl_a._1, tbl_b._2 FROM tbl_a JOIN tbl_b " + + "WHERE tbl_a._1 > tbl_a._2 LIMIT 2") + checkSparkAnswerAndOperator( + df, + classOf[CollectLimitExec], + classOf[BroadcastNestedLoopJoinExec], + classOf[BroadcastExchangeExec]) + } + } + } + + test("empty-column input (read schema is empty)") { + withTable("t1") { + Seq((1, true), (2, false)) + .toDF("l", "b") + .repartition(2) + .write + .saveAsTable("t1") + val query = spark.table("t1").selectExpr("IF(l > 1 AND null, 5, 1) AS out") + checkSparkAnswerAndOperator(query) + } + } + + test("empty-column aggregation") { + withTable("t1") { + Seq((1, true), (2, false)) + .toDF("l", "b") + .repartition(2) + .write + .saveAsTable("t1") + val query = sql("SELECT count(1) FROM t1") + checkSparkAnswerAndOperator(query, classOf[HashAggregateExec], classOf[ShuffleExchangeExec]) + } + } + + test("null handling") { + Seq("true", "false").foreach { pushDown => + val table = "t1" + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> pushDown) { + withTable(table) { + sql(s"create table $table(a int, b int, c int) using parquet") + sql(s"insert into $table values(1,0,0)") + sql(s"insert into $table values(2,0,1)") + sql(s"insert into $table values(3,1,0)") + sql(s"insert into $table values(4,1,1)") + sql(s"insert into $table values(5,null,0)") + sql(s"insert into $table values(6,null,1)") + sql(s"insert into $table values(7,null,null)") + + val query = sql(s"select a+120 from $table where b<10 OR c=1") + checkSparkAnswerAndOperator(query) + } + } + } + } + + test("float4.sql") { + val table = "t1" + withTable(table) { + sql(s"CREATE TABLE $table (f1 float) USING parquet") + sql(s"INSERT INTO $table VALUES (float(' 0.0'))") + sql(s"INSERT INTO $table VALUES (float('1004.30 '))") + sql(s"INSERT INTO $table VALUES (float(' -34.84 '))") + sql(s"INSERT INTO $table VALUES (float('1.2345678901234e+20'))") + sql(s"INSERT INTO $table VALUES (float('1.2345678901234e-20'))") + + val query = sql(s"SELECT '' AS four, f.* FROM $table f WHERE '1004.3' > f.f1") + checkSparkAnswerAndOperator(query) + } + } + + test("NaN in predicate expression") { + val t = "test_table" + + withTable(t) { + Seq[(Integer, java.lang.Short, java.lang.Float)]( + (1, 100.toShort, 3.14.toFloat), + (2, Short.MaxValue, Float.NaN), + (3, Short.MinValue, Float.PositiveInfinity), + (4, 0.toShort, Float.MaxValue), + (5, null, null)) + .toDF("c1", "c2", "c3") + .write + .saveAsTable(t) + + val df = spark.table(t) + + var query = df.where("c3 > double('nan')").select("c1") + checkSparkAnswer(query) + // Empty result will be optimized to a local relation. No CometExec involved. + // checkCometExec(query, 0, cometExecs => {}) + + query = df.where("c3 >= double('nan')").select("c1") + checkSparkAnswerAndOperator(query) + // checkCometExec(query, 1, cometExecs => {}) + + query = df.where("c3 == double('nan')").select("c1") + checkSparkAnswerAndOperator(query) + + query = df.where("c3 <=> double('nan')").select("c1") + checkSparkAnswerAndOperator(query) + + query = df.where("c3 != double('nan')").select("c1") + checkSparkAnswerAndOperator(query) + + query = df.where("c3 <= double('nan')").select("c1") + checkSparkAnswerAndOperator(query) + + query = df.where("c3 < double('nan')").select("c1") + checkSparkAnswerAndOperator(query) + } + } + + test("table statistics") { + withTempDatabase { database => + spark.catalog.setCurrentDatabase(database) + withTempDir { dir => + withTable("t1", "t2") { + spark.range(10).write.saveAsTable("t1") + sql( + s"CREATE EXTERNAL TABLE t2 USING parquet LOCATION '${dir.toURI}' " + + "AS SELECT * FROM range(20)") + + sql(s"ANALYZE TABLES IN $database COMPUTE STATISTICS NOSCAN") + checkTableStats("t1", hasSizeInBytes = true, expectedRowCounts = None) + checkTableStats("t2", hasSizeInBytes = true, expectedRowCounts = None) + + sql("ANALYZE TABLES COMPUTE STATISTICS") + checkTableStats("t1", hasSizeInBytes = true, expectedRowCounts = Some(10)) + checkTableStats("t2", hasSizeInBytes = true, expectedRowCounts = Some(20)) + } + } + } + } + + test("like (LikeSimplification disabled)") { + val table = "names" + withSQLConf( + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "org.apache.spark.sql.catalyst.optimizer.LikeSimplification") { + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael Rose')") + sql(s"insert into $table values(3,'Robert Williams')") + sql(s"insert into $table values(4,'Rames Rose')") + sql(s"insert into $table values(5,'Rames rose')") + + // Filter column having values 'Rames _ose', where any character matches for '_' + val query = sql(s"select id from $table where name like 'Rames _ose'") + checkSparkAnswerAndOperator(query) + + // Filter rows that contains 'rose' in 'name' column + val queryContains = sql(s"select id from $table where name like '%rose%'") + checkSparkAnswerAndOperator(queryContains) + + // Filter rows that starts with 'R' following by any characters + val queryStartsWith = sql(s"select id from $table where name like 'R%'") + checkSparkAnswerAndOperator(queryStartsWith) + + // Filter rows that ends with 's' following by any characters + val queryEndsWith = sql(s"select id from $table where name like '%s'") + checkSparkAnswerAndOperator(queryEndsWith) + } + } + } + + test("sum overflow (ANSI disable)") { + Seq("true", "false").foreach { dictionary => + withSQLConf( + SQLConf.ANSI_ENABLED.key -> "false", + "parquet.enable.dictionary" -> dictionary) { + withParquetTable(Seq((Long.MaxValue, 1), (Long.MaxValue, 2)), "tbl") { + val df = sql("SELECT sum(_1) FROM tbl") + checkSparkAnswerAndOperator( + df, + classOf[HashAggregateExec], + classOf[ShuffleExchangeExec]) + } + } + } + } + + test("partition col") { + withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + withTable("t1") { + sql(""" + | CREATE TABLE t1(name STRING, part1 TIMESTAMP) + | USING PARQUET PARTITIONED BY (part1) + """.stripMargin) + + sql(""" + | INSERT OVERWRITE t1 PARTITION( + | part1 = timestamp'2019-01-01 11:11:11' + | ) VALUES('a') + """.stripMargin) + checkSparkAnswerAndOperator(sql(""" + | SELECT + | name, + | CAST(part1 AS STRING) + | FROM t1 + """.stripMargin)) + } + } + } + + test("SPARK-33474: Support typed literals as partition spec values") { + withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "Asia/Kathmandu") { + withTable("t1") { + val binaryStr = "Spark SQL" + val binaryHexStr = Hex.hex(UTF8String.fromString(binaryStr).getBytes).toString + sql(""" + | CREATE TABLE t1(name STRING, part1 DATE, part2 TIMESTAMP, part3 BINARY, + | part4 STRING, part5 STRING, part6 STRING, part7 STRING) + | USING PARQUET PARTITIONED BY (part1, part2, part3, part4, part5, part6, part7) + """.stripMargin) + + sql(s""" + | INSERT OVERWRITE t1 PARTITION( + | part1 = date'2019-01-01', + | part2 = timestamp'2019-01-01 11:11:11', + | part3 = X'$binaryHexStr', + | part4 = 'p1', + | part5 = date'2019-01-01', + | part6 = timestamp'2019-01-01 11:11:11', + | part7 = X'$binaryHexStr' + | ) VALUES('a') + """.stripMargin) + checkSparkAnswerAndOperator(sql(""" + | SELECT + | name, + | CAST(part1 AS STRING), + | CAST(part2 as STRING), + | CAST(part3 as STRING), + | part4, + | part5, + | part6, + | part7 + | FROM t1 + """.stripMargin)) + + val e = intercept[AnalysisException] { + sql("CREATE TABLE t2(name STRING, part INTERVAL) USING PARQUET PARTITIONED BY (part)") + }.getMessage + assert(e.contains("Cannot use interval")) + } + } + } + + def getCatalogTable(tableName: String): CatalogTable = { + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + } + + def checkTableStats( + tableName: String, + hasSizeInBytes: Boolean, + expectedRowCounts: Option[Int]): Option[CatalogStatistics] = { + val stats = getCatalogTable(tableName).stats + if (hasSizeInBytes || expectedRowCounts.nonEmpty) { + assert(stats.isDefined) + assert(stats.get.sizeInBytes >= 0) + assert(stats.get.rowCount === expectedRowCounts) + } else { + assert(stats.isEmpty) + } + + stats + } + + def joinCondition(joinCols: Seq[String])(left: DataFrame, right: DataFrame): Column = { + joinCols.map(col => left(col) === right(col)).reduce(_ && _) + } + + def testBucketing( + bucketedTableTestSpecLeft: BucketedTableTestSpec, + bucketedTableTestSpecRight: BucketedTableTestSpec, + joinType: String = "inner", + joinCondition: (DataFrame, DataFrame) => Column): Unit = { + val df1 = + (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k").as("df1") + val df2 = + (0 until 50).map(i => (i % 7, i % 11, i.toString)).toDF("i", "j", "k").as("df2") + + val BucketedTableTestSpec(bucketSpecLeft, numPartitionsLeft, _, _, _) = + bucketedTableTestSpecLeft + + val BucketedTableTestSpec(bucketSpecRight, numPartitionsRight, _, _, _) = + bucketedTableTestSpecRight + + withTable("bucketed_table1", "bucketed_table2") { + withBucket(df1.repartition(numPartitionsLeft).write.format("parquet"), bucketSpecLeft) + .saveAsTable("bucketed_table1") + withBucket(df2.repartition(numPartitionsRight).write.format("parquet"), bucketSpecRight) + .saveAsTable("bucketed_table2") + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val t1 = spark.table("bucketed_table1") + val t2 = spark.table("bucketed_table2") + val joined = t1.join(t2, joinCondition(t1, t2), joinType) + + val df = joined.sort("bucketed_table1.k", "bucketed_table2.k") + checkSparkAnswer(df) + + // The sub-plan contains should contain all native operators except a SMJ + val subPlan = stripAQEPlan(df.queryExecution.executedPlan).collectFirst { + case s: SortMergeJoinExec => s + } + assert(subPlan.isDefined) + checkCometOperators(subPlan.get, classOf[SortMergeJoinExec]) + } + } + } + + test("bucketed table") { + val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j"))) + } + + def withBucket( + writer: DataFrameWriter[Row], + bucketSpec: Option[BucketSpec]): DataFrameWriter[Row] = { + bucketSpec + .map { spec => + writer.bucketBy( + spec.numBuckets, + spec.bucketColumnNames.head, + spec.bucketColumnNames.tail: _*) + + if (spec.sortColumnNames.nonEmpty) { + writer.sortBy(spec.sortColumnNames.head, spec.sortColumnNames.tail: _*) + } else { + writer + } + } + .getOrElse(writer) + } + + test("union") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + val df1 = sql("select * FROM tbl where _1 >= 2").select("_1") + val df2 = sql("select * FROM tbl where _1 >= 2").select("_2") + val df3 = sql("select * FROM tbl where _1 >= 3").select("_2") + + val unionDf1 = df1.union(df2) + checkSparkAnswerAndOperator(unionDf1) + + // Test union with different number of rows from inputs + val unionDf2 = df1.union(df3) + checkSparkAnswerAndOperator(unionDf2) + + val unionDf3 = df1.union(df2).union(df3) + checkSparkAnswerAndOperator(unionDf3) + } + } + + test("native execution after union") { + withParquetTable((0 until 5).map(i => (i, i + 1)), "tbl") { + val df1 = sql("select * FROM tbl where _1 >= 2").select("_1") + val df2 = sql("select * FROM tbl where _1 >= 2").select("_2") + val df3 = sql("select * FROM tbl where _1 >= 3").select("_2") + + val unionDf1 = df1.union(df2).select($"_1" + 1).sortWithinPartitions($"_1") + checkSparkAnswerAndOperator(unionDf1) + + // Test union with different number of rows from inputs + val unionDf2 = df1.union(df3).select($"_1" + 1).sortWithinPartitions($"_1") + checkSparkAnswerAndOperator(unionDf2) + + val unionDf3 = df1.union(df2).union(df3).select($"_1" + 1).sortWithinPartitions($"_1") + checkSparkAnswerAndOperator(unionDf3) + } + } + + test("native execution after coalesce") { + withTable("t1") { + (0 until 5) + .map(i => (i, (i + 1).toLong)) + .toDF("l", "b") + .write + .saveAsTable("t1") + + val df = sql("SELECT * FROM t1") + .sortWithinPartitions($"l".desc) + .repartition(10, $"l") + + val rdd = df.rdd + assert(rdd.partitions.length == 10) + + val coalesced = df.coalesce(2).select($"l" + 1).sortWithinPartitions($"l") + checkSparkAnswerAndOperator( + coalesced, + classOf[ProjectExec], + classOf[SortExec], + classOf[CoalesceExec], + classOf[ShuffleExchangeExec]) + } + } + + test("disabled/unsupported exec with multiple children should not disappear") { + withSQLConf( + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "false", + CometConf.COMET_EXEC_CONFIG_PREFIX + ".project.enabled" -> "true", + CometConf.COMET_EXEC_CONFIG_PREFIX + ".union.enabled" -> "false") { + withParquetDataFrame((0 until 5).map(Tuple1(_))) { df => + val projected = df.selectExpr("_1 as x") + val unioned = projected.union(df) + val p = unioned.queryExecution.executedPlan.find(_.isInstanceOf[UnionExec]) + assert(p.get.collectLeaves().forall(_.isInstanceOf[CometScanExec])) + } + } + } + + test("coalesce") { + withTable("t1") { + (0 until 5) + .map(i => (i, (i + 1).toLong)) + .toDF("l", "b") + .write + .saveAsTable("t1") + + val df = sql("SELECT * FROM t1") + .sortWithinPartitions($"l".desc) + .repartition(10, $"l") + + val rdd = df.rdd + assert(rdd.getNumPartitions == 10) + + val coalesced = df.coalesce(2) + checkSparkAnswerAndOperator(coalesced, classOf[CoalesceExec], classOf[ShuffleExchangeExec]) + + val coalescedRdd = coalesced.rdd + assert(coalescedRdd.getNumPartitions == 2) + } + } +} + +case class BucketedTableTestSpec( + bucketSpec: Option[BucketSpec], + numPartitions: Int = 10, + expectedShuffle: Boolean = true, + expectedSort: Boolean = true, + expectedNumOutputPartitions: Option[Int] = None) diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala new file mode 100644 index 0000000000..4f92242efb --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala @@ -0,0 +1,1351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.parquet + +import java.io.{File, FileFilter} +import java.math.BigDecimal +import java.time.{ZoneId, ZoneOffset} + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + +import org.scalactic.source.Position +import org.scalatest.Tag + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.parquet.example.data.simple.SimpleGroup +import org.apache.parquet.schema.MessageTypeParser +import org.apache.spark.SparkException +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.comet.CometBatchScanExec +import org.apache.spark.sql.comet.CometScanExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus + +import com.google.common.primitives.UnsignedLong + +abstract class ParquetReadSuite extends CometTestBase { + import testImplicits._ + + testStandardAndLegacyModes("decimals") { + Seq(true, false).foreach { useDecimal128 => + Seq(16, 1024).foreach { batchSize => + withSQLConf( + CometConf.COMET_EXEC_ENABLED.key -> false.toString, + CometConf.COMET_USE_DECIMAL_128.key -> useDecimal128.toString, + CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + var combinations = Seq((5, 2), (1, 0), (18, 10), (18, 17), (19, 0), (38, 37)) + // If ANSI mode is on, the combination (1, 1) will cause a runtime error. Otherwise, the + // decimal RDD contains all null values and should be able to read back from Parquet. + + if (!SQLConf.get.ansiEnabled) { + combinations = combinations ++ Seq((1, 1)) + } + for ((precision, scale) <- combinations; useDictionary <- Seq(false, true)) { + withTempPath { dir => + val data = makeDecimalRDD(1000, DecimalType(precision, scale), useDictionary) + data.write.parquet(dir.getCanonicalPath) + readParquetFile(dir.getCanonicalPath) { df => + { + checkAnswer(df, data.collect().toSeq) + } + } + } + } + } + } + } + } + + test("simple count") { + withParquetTable((0 until 10).map(i => (i, i.toString)), "tbl") { + assert(sql("SELECT * FROM tbl WHERE _1 % 2 == 0").count() == 5) + } + } + + test("basic data types") { + Seq(7, 1024).foreach { batchSize => + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + val data = (-100 to 100).map { i => + ( + i % 2 == 0, + i, + i.toByte, + i.toShort, + i.toLong, + i.toFloat, + i.toDouble, + DateTimeUtils.toJavaDate(i)) + } + checkParquetScan(data) + checkParquetFile(data) + } + } + } + + test("basic data types with dictionary") { + Seq(7, 1024).foreach { batchSize => + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + val data = (-100 to 100).map(_ % 4).map { i => + ( + i % 2 == 0, + i, + i.toByte, + i.toShort, + i.toLong, + i.toFloat, + i.toDouble, + DateTimeUtils.toJavaDate(i)) + } + checkParquetScan(data) + checkParquetFile(data) + } + } + } + + test("basic filters") { + val data = (-100 to 100).map { i => + ( + i % 2 == 0, + i, + i.toByte, + i.toShort, + i.toLong, + i.toFloat, + i.toDouble, + DateTimeUtils.toJavaDate(i)) + } + val filter = (row: Row) => row.getBoolean(0) + checkParquetScan(data, filter) + checkParquetFile(data, filter) + } + + test("raw binary test") { + val data = (1 to 4).map(i => Tuple1(Array.fill(3)(i.toByte))) + withParquetDataFrame(data) { df => + assertResult(data.map(_._1.mkString(",")).sorted) { + df.collect().map(_.getAs[Array[Byte]](0).mkString(",")).sorted + } + } + } + + test("string") { + val data = (1 to 4).map(i => Tuple1(i.toString)) + // Property spark.sql.parquet.binaryAsString shouldn't affect Parquet files written by Spark SQL + // as we store Spark SQL schema in the extra metadata. + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "false")(checkParquetFile(data)) + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "true")(checkParquetFile(data)) + } + + test("string with dictionary") { + Seq((100, 5), (1000, 10)).foreach { case (total, divisor) => + val data = (1 to total).map(i => Tuple1((i % divisor).toString)) + // Property spark.sql.parquet.binaryAsString shouldn't affect Parquet files written by Spark SQL + // as we store Spark SQL schema in the extra metadata. + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "false")(checkParquetFile(data)) + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "true")(checkParquetFile(data)) + } + } + + test("long string + reserve additional space for value buffer") { + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> 16.toString) { + val data = (1 to 100).map(i => (i, i.toString * 10)) + checkParquetFile(data) + } + } + + test("timestamp") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + val expected = makeRawTimeParquetFile(path, dictionaryEnabled = dictionaryEnabled, 10000) + val useLocalDateTime = spark.version >= "3.3" + readParquetFile(path.toString) { df => + checkAnswer( + df.select($"_0", $"_1", $"_2", $"_3", $"_4", $"_5"), + expected.map { + case None => + Row(null, null, null, null, null, null) + case Some(i) => + // use `LocalDateTime` for `TimestampNTZType` with Spark 3.3 and above. At the moment, + // Spark reads Parquet timestamp values into `Timestamp` (with local timezone) + // regardless of whether `isAdjustedToUTC` is true or false. See SPARK-36182. + // TODO: make `LocalDateTime` default after dropping Spark 3.2.0 support + val ts = new java.sql.Timestamp(i) + val ldt = if (useLocalDateTime) { + ts.toLocalDateTime + .atZone(ZoneId.systemDefault()) + .withZoneSameInstant(ZoneOffset.UTC) + .toLocalDateTime + } else ts + Row(ts, ts, ts, ldt, ts, ldt) + }) + } + } + } + } + + test("timestamp as int96") { + import testImplicits._ + + val N = 100 + val ts = "2020-01-01 01:02:03.123456" + Seq(false, true).foreach { dictionaryEnabled => + Seq(false, true).foreach { conversionEnabled => + withSQLConf( + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> "INT96", + SQLConf.PARQUET_INT96_TIMESTAMP_CONVERSION.key -> conversionEnabled.toString) { + withTempPath { path => + Seq + .tabulate(N)(_ => ts) + .toDF("ts1") + .select($"ts1".cast("timestamp").as("ts")) + .repartition(1) + .write + .option("parquet.enable.dictionary", dictionaryEnabled) + .parquet(path.getCanonicalPath) + + checkAnswer( + spark.read.parquet(path.getCanonicalPath).select($"ts".cast("string")), + Seq.tabulate(N)(_ => Row(ts))) + } + } + } + } + } + + test("batch paging on basic types") { + Seq(1, 2, 4, 9).foreach { batchSize => + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + val data = (1 to 10).map(i => (i, i.toByte, i.toShort, i.toFloat, i.toDouble, i.toString)) + checkParquetFile(data) + } + } + } + + test("nulls") { + val allNulls = ( + null.asInstanceOf[java.lang.Boolean], + null.asInstanceOf[Integer], + null.asInstanceOf[java.lang.Long], + null.asInstanceOf[java.lang.Float], + null.asInstanceOf[java.lang.Double], + null.asInstanceOf[java.lang.String]) + + withParquetDataFrame(allNulls :: Nil) { df => + val rows = df.collect() + assert(rows.length === 1) + assert(rows.head === Row(Seq.fill(6)(null): _*)) + assert(df.where("_1 is null").count() == 1) + } + } + + test("mixed nulls and non-nulls") { + val rand = scala.util.Random + val data = (0 to 100).map { i => + val row: (Boolean, Integer, java.lang.Long, java.lang.Float, java.lang.Double, String) = { + if (rand.nextBoolean()) { + (i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble, i.toString) + } else { + ( + null.asInstanceOf[java.lang.Boolean], + null.asInstanceOf[Integer], + null.asInstanceOf[java.lang.Long], + null.asInstanceOf[java.lang.Float], + null.asInstanceOf[java.lang.Double], + null.asInstanceOf[String]) + } + } + row + } + checkParquetFile(data) + } + + test("test multiple pages with different sizes and nulls") { + def makeRawParquetFile( + path: Path, + dictionaryEnabled: Boolean, + n: Int, + pageSize: Int): Seq[Option[Int]] = { + val schemaStr = { + if (isSpark34Plus) { + """ + |message root { + | optional boolean _1; + | optional int32 _2(INT_8); + | optional int32 _3(INT_16); + | optional int32 _4; + | optional int64 _5; + | optional float _6; + | optional double _7; + | optional binary _8(UTF8); + | optional int32 _9(UINT_8); + | optional int32 _10(UINT_16); + | optional int32 _11(UINT_32); + | optional int64 _12(UINT_64); + | optional binary _13(ENUM); + | optional FIXED_LEN_BYTE_ARRAY(3) _14; + |} + """.stripMargin + } else { + """ + |message root { + | optional boolean _1; + | optional int32 _2(INT_8); + | optional int32 _3(INT_16); + | optional int32 _4; + | optional int64 _5; + | optional float _6; + | optional double _7; + | optional binary _8(UTF8); + | optional int32 _9(UINT_8); + | optional int32 _10(UINT_16); + | optional int32 _11(UINT_32); + | optional int64 _12(UINT_64); + | optional binary _13(ENUM); + | optional binary _14(UTF8); + |} + """.stripMargin + } + } + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter( + schema, + path, + dictionaryEnabled = dictionaryEnabled, + pageSize = pageSize, + dictionaryPageSize = pageSize) + + val rand = scala.util.Random + val expected = (0 until n).map { i => + if (rand.nextBoolean()) { + None + } else { + Some(i) + } + } + expected.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i % 2 == 0) + record.add(1, i.toByte) + record.add(2, i.toShort) + record.add(3, i) + record.add(4, i.toLong) + record.add(5, i.toFloat) + record.add(6, i.toDouble) + record.add(7, i.toString * 48) + record.add(8, (-i).toByte) + record.add(9, (-i).toShort) + record.add(10, -i) + record.add(11, (-i).toLong) + record.add(12, i.toString) + record.add(13, (i % 10).toString * 3) + case _ => + } + writer.write(record) + } + + writer.close() + expected + } + + Seq(64, 128, 256, 512, 1024, 4096, 5000).foreach { pageSize => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + val expected = makeRawParquetFile(path, dictionaryEnabled = false, 10000, pageSize) + readParquetFile(path.toString) { df => + checkAnswer( + df, + expected.map { + case None => + Row(null, null, null, null, null, null, null, null, null, null, null, null, null, + null) + case Some(i) => + val flba_field = if (isSpark34Plus) { + Array.fill(3)(i % 10 + 48) // char '0' is 48 in ascii + } else { + (i % 10).toString * 3 + } + Row( + i % 2 == 0, + i.toByte, + i.toShort, + i, + i.toLong, + i.toFloat, + i.toDouble, + i.toString * 48, + java.lang.Byte.toUnsignedInt((-i).toByte), + java.lang.Short.toUnsignedInt((-i).toShort), + java.lang.Integer.toUnsignedLong(-i), + new BigDecimal(UnsignedLong.fromLongBits((-i).toLong).bigIntegerValue()), + i.toString, + flba_field) + }) + } + readParquetFile(path.toString) { df => + assert( + df.filter("_8 IS NOT NULL AND _4 % 256 == 255").count() == + expected.flatten.count(_ % 256 == 255)) + } + } + } + } + + test("vector reloading with all non-null values") { + def makeRawParquetFile( + path: Path, + dictionaryEnabled: Boolean, + n: Int, + numNonNulls: Int): Seq[Option[Int]] = { + val schemaStr = + """ + |message root { + | optional int32 _1; + |} + """.stripMargin + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter(schema, path, dictionaryEnabled = dictionaryEnabled) + + val expected = (0 until n).map { i => + if (i >= numNonNulls) { + None + } else { + Some(i) + } + } + expected.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i) + case _ => + } + writer.write(record) + } + + writer.close() + expected + } + + Seq(2, 99, 1024).foreach { numNonNulls => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + val expected = makeRawParquetFile(path, dictionaryEnabled = false, 1024, numNonNulls) + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> "2") { + readParquetFile(path.toString) { df => + checkAnswer( + df, + expected.map { + case None => + Row(null) + case Some(i) => + Row(i) + }) + } + } + } + } + } + + test("test lazy materialization skipping") { + def makeRawParquetFile( + path: Path, + dictionaryEnabled: Boolean, + pageSize: Int, + pageRowCountLimit: Int, + expected: Seq[Row]): Unit = { + val schemaStr = + """ + |message root { + | optional int32 _1; + | optional binary _2(UTF8); + |} + """.stripMargin + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter( + schema, + path, + dictionaryEnabled = dictionaryEnabled, + pageSize = pageSize, + dictionaryPageSize = pageSize, + pageRowCountLimit = pageRowCountLimit) + + expected.foreach { row => + val record = new SimpleGroup(schema) + record.add(0, row.getInt(0)) + record.add(1, row.getString(1)) + writer.write(record) + } + + writer.close() + } + + val skip = Row(0, "a") // row to skip by lazy materialization + val read = Row(1, "b") // row not to skip + // The initial page row count is always 100 in ParquetWriter, even with pageRowCountLimit config + // Thus, use this header to fill in the first 100 + val header = Seq.fill(100)(skip) + + val expected = Seq( // spotless:off + read, read, read, read, // read all rows in the page + skip, skip, skip, skip, // skip all rows in the page + skip, skip, skip, skip, // consecutively skip all rows in the page + read, skip, skip, read, // skip middle rows in the page + skip, read, read, skip, // read middle rows in the page + skip, read, skip, read, // skip and read in turns + read, skip, read, skip // skip and read in turns + ) // spotless:on + + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + withSQLConf( + CometConf.COMET_BATCH_SIZE.key -> "4", + CometConf.COMET_EXEC_ENABLED.key -> "false") { + makeRawParquetFile(path, dictionaryEnabled = false, 1024, 4, header ++ expected) + readParquetFile(path.toString) { df => + checkAnswer(df.filter("_1 != 0"), expected.filter(_.getInt(0) != 0)) + } + } + } + } + + test("test multiple pages with mixed PLAIN_DICTIONARY and PLAIN encoding") { + // TODO: consider merging this with the same method above + def makeRawParquetFile(path: Path, n: Int): Seq[Option[Int]] = { + val dictionaryPageSize = 1024 + val pageRowCount = 500 + val schemaStr = + """ + |message root { + | optional boolean _1; + | optional int32 _2(INT_8); + | optional int32 _3(INT_16); + | optional int32 _4; + | optional int64 _5; + | optional float _6; + | optional double _7; + | optional binary _8(UTF8); + |} + """.stripMargin + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter( + schema, + path, + dictionaryEnabled = true, + dictionaryPageSize = dictionaryPageSize, + pageRowCountLimit = pageRowCount) + + val rand = scala.util.Random + val expected = (0 until n).map { i => + // use a single value for the first page, to make sure dictionary encoding kicks in + val value = if (i < pageRowCount) i % 8 else i + if (rand.nextBoolean()) None + else Some(value) + } + + expected.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i % 2 == 0) + record.add(1, i.toByte) + record.add(2, i.toShort) + record.add(3, i) + record.add(4, i.toLong) + record.add(5, i.toFloat) + record.add(6, i.toDouble) + record.add(7, i.toString * 100) + case _ => + } + writer.write(record) + } + + writer.close() + expected + } + + Seq(16, 128).foreach { batchSize => + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> batchSize.toString) { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + val expected = makeRawParquetFile(path, 10000) + readParquetFile(path.toString) { df => + checkAnswer( + df, + expected.map { + case None => + Row(null, null, null, null, null, null, null, null) + case Some(i) => + Row( + i % 2 == 0, + i.toByte, + i.toShort, + i, + i.toLong, + i.toFloat, + i.toDouble, + i.toString * 100) + }) + } + } + } + } + } + + test("skip vector re-loading") { + Seq(false, true).foreach { enableDictionary => + withSQLConf( + CometConf.COMET_BATCH_SIZE.key -> 7.toString, + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + // Make sure this works with Comet native execution too + val data = (1 to 100) + .map(_ % 5) // trigger dictionary encoding + .map(i => (i, i.toByte, i.toShort, i.toFloat, i.toDouble, i.toString)) + withParquetTable(data, "tbl", withDictionary = enableDictionary) { + val df = sql("SELECT count(*) FROM tbl WHERE _1 >= 0") + checkAnswer(df, Row(100) :: Nil) + } + } + } + } + + test("partition column types") { + withTempPath { dir => + Seq(1).toDF().repartition(1).write.parquet(dir.getCanonicalPath) + + val dataTypes = + Seq( + StringType, + BooleanType, + ByteType, + BinaryType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + DecimalType(25, 5), + DateType, + TimestampType) + + // TODO: support `NullType` here, after we add the support in `ColumnarBatchRow` + val constantValues = + Seq( + UTF8String.fromString("a string"), + true, + 1.toByte, + "Spark SQL".getBytes, + 2.toShort, + 3, + Long.MaxValue, + 0.25.toFloat, + 0.75d, + Decimal("1234.23456"), + DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("2015-01-01")), + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"))) + + dataTypes.zip(constantValues).foreach { case (dt, v) => + val schema = StructType(StructField("pcol", dt) :: Nil) + val conf = SQLConf.get + val partitionValues = new GenericInternalRow(Array(v)) + val file = dir + .listFiles(new FileFilter { + override def accept(pathname: File): Boolean = + pathname.isFile && pathname.toString.endsWith("parquet") + }) + .head + val reader = new BatchReader( + file.toString, + CometConf.COMET_BATCH_SIZE.get(conf), + schema, + partitionValues) + reader.init() + + try { + reader.nextBatch() + val batch = reader.currentBatch() + val actual = batch.getRow(0).get(1, dt) + val expected = v + if (dt.isInstanceOf[BinaryType]) { + assert( + actual.asInstanceOf[Array[Byte]] sameElements expected.asInstanceOf[Array[Byte]]) + } else { + assert(actual == expected) + } + } finally { + reader.close() + } + } + } + } + + test("partition columns - multiple batch") { + withSQLConf( + CometConf.COMET_BATCH_SIZE.key -> 7.toString, + CometConf.COMET_EXEC_ENABLED.key -> "false", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", + CometConf.COMET_ENABLED.key -> "true") { + Seq("a", null).foreach { partValue => + withTempPath { dir => + (1 to 100) + .map(v => (partValue.asInstanceOf[String], v)) + .toDF("pcol", "col") + .repartition(1) + .write + .format("parquet") + .partitionBy("pcol") + .save(dir.getCanonicalPath) + val df = spark.read.format("parquet").load(dir.getCanonicalPath) + assert(df.filter("col > 90").count() == 10) + } + } + } + } + + test("fix: string partition column with incorrect offset buffer") { + def makeRawParquetFile( + path: Path, + dictionaryEnabled: Boolean, + n: Int, + pageSize: Int): Seq[Option[Int]] = { + val schemaStr = + """ + |message root { + | optional binary _1(UTF8); + |} + """.stripMargin + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter( + schema, + path, + dictionaryEnabled = dictionaryEnabled, + pageSize = pageSize, + dictionaryPageSize = pageSize, + rowGroupSize = 1024 * 128) + + val rand = scala.util.Random + val expected = (0 until n).map { i => + if (rand.nextBoolean()) { + None + } else { + Some(i) + } + } + expected.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i.toString * 48) + case _ => + } + writer.write(record) + } + + writer.close() + expected + } + + withTable("tbl") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawParquetFile(path, false, 10000, 128) + + sql("CREATE TABLE tbl (value STRING, p STRING) USING PARQUET PARTITIONED BY (p) ") + sql(s"ALTER TABLE tbl ADD PARTITION (p='a') LOCATION '$dir'") + assert(sql("SELECT DISTINCT p FROM tbl").count() == 1) + } + } + + } + + test("missing columns") { + withTempPath { dir => + Seq("a", "b").toDF("col1").write.parquet(dir.getCanonicalPath) + + // Create a schema where `col2` doesn't exist in the file schema + var schema = + StructType(Seq(StructField("col1", StringType), StructField("col2", IntegerType))) + var df = spark.read.schema(schema).parquet(dir.getCanonicalPath) + checkAnswer(df, Row("a", null) :: Row("b", null) :: Nil) + + // Should be the same when the missing column is at the beginning of the schema + + schema = StructType(Seq(StructField("col0", BooleanType), StructField("col1", StringType))) + df = spark.read.schema(schema).parquet(dir.getCanonicalPath) + checkAnswer(df, Row(null, "a") :: Row(null, "b") :: Nil) + } + } + + test("unsigned int supported") { + Seq(true, false).foreach { dictionaryEnabled => + def makeRawParquetFile(path: Path): Unit = { + val schemaStr = + """message root { + | required INT32 a(UINT_8); + | required INT32 b(UINT_16); + | required INT32 c(UINT_32); + |} + """.stripMargin + val schema = MessageTypeParser.parseMessageType(schemaStr) + + val writer = createParquetWriter(schema, path, dictionaryEnabled) + + (0 until 10).foreach { n => + val record = new SimpleGroup(schema) + record.add(0, n.toByte + Byte.MaxValue) + record.add(1, n.toShort + Short.MaxValue) + record.add(2, n + Int.MaxValue) + writer.write(record) + } + writer.close() + } + + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawParquetFile(path) + readParquetFile(path.toString) { df => + checkAnswer( + df, + (0 until 10).map(n => + Row(n.toByte + Byte.MaxValue, n.toShort + Short.MaxValue, n + Int.MaxValue.toLong))) + } + } + } + } + + test("unsigned long supported") { + Seq(true, false).foreach { dictionaryEnabled => + def makeRawParquetFile(path: Path): Unit = { + val schemaStr = + """message root { + | required INT64 a(UINT_64); + |} + """.stripMargin + val schema = MessageTypeParser.parseMessageType(schemaStr) + + val writer = createParquetWriter(schema, path, dictionaryEnabled) + + (0 until 10).map(_.toLong).foreach { n => + val record = new SimpleGroup(schema) + record.add(0, n + Long.MaxValue) + writer.write(record) + } + writer.close() + } + + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawParquetFile(path) + readParquetFile(path.toString) { df => + checkAnswer( + df, + (0 until 10).map(n => + Row( + new BigDecimal(UnsignedLong.fromLongBits(n + Long.MaxValue).bigIntegerValue())))) + } + } + } + } + + test("enum support") { + // https://github.com/apache/parquet-format/blob/master/LogicalTypes.md + // "enum type should interpret ENUM annotated field as a UTF-8" + Seq(true, false).foreach { dictionaryEnabled => + def makeRawParquetFile(path: Path): Unit = { + val schemaStr = + """message root { + | required BINARY a(ENUM); + |} + """.stripMargin + val schema = MessageTypeParser.parseMessageType(schemaStr) + + val writer = createParquetWriter(schema, path, dictionaryEnabled) + + (0 until 10).map(_.toLong).foreach { n => + val record = new SimpleGroup(schema) + record.add(0, n.toString) + writer.write(record) + } + writer.close() + } + + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawParquetFile(path) + readParquetFile(path.toString) { df => + checkAnswer(df, (0 until 10).map(n => Row(n.toString))) + } + } + } + } + + test("FIXED_LEN_BYTE_ARRAY support") { + assume(isSpark34Plus) + Seq(true, false).foreach { dictionaryEnabled => + def makeRawParquetFile(path: Path): Unit = { + val schemaStr = + """message root { + | required FIXED_LEN_BYTE_ARRAY(1) a; + | required FIXED_LEN_BYTE_ARRAY(3) b; + |} + """.stripMargin + val schema = MessageTypeParser.parseMessageType(schemaStr) + + val writer = createParquetWriter(schema, path, dictionaryEnabled) + + (0 until 10).map(_.toString).foreach { n => + val record = new SimpleGroup(schema) + record.add(0, n) + record.add(1, n + n + n) + writer.write(record) + } + writer.close() + } + + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawParquetFile(path) + readParquetFile(path.toString) { df => + checkAnswer( + df, + (48 until 58).map(n => // char '0' is 48 in ascii + Row(Array(n), Array(n, n, n)))) + } + } + } + } + + test("schema evolution") { + Seq(true, false).foreach { enableSchemaEvolution => + Seq(true, false).foreach { useDictionary => + { + withSQLConf( + CometConf.COMET_SCHEMA_EVOLUTION_ENABLED.key -> enableSchemaEvolution.toString) { + val data = (0 until 100).map(i => { + val v = if (useDictionary) i % 5 else i + (v, v.toFloat) + }) + val readSchema = + StructType( + Seq(StructField("_1", LongType, false), StructField("_2", DoubleType, false))) + + withParquetDataFrame(data, schema = Some(readSchema)) { df => + if (enableSchemaEvolution) { + checkAnswer(df, data.map(Row.fromTuple)) + } else { + assertThrows[SparkException](df.collect()) + } + } + } + } + } + } + } + + test("scan metrics") { + val metricNames = Seq( + "ParquetRowGroups", + "ParquetNativeDecodeTime", + "ParquetNativeLoadTime", + "ParquetLoadRowGroupTime", + "ParquetInputFileReadTime", + "ParquetInputFileReadSize", + "ParquetInputFileReadThroughput") + + withParquetTable((0 until 10000).map(i => (i, i.toDouble)), "tbl") { + val df = sql("SELECT * FROM tbl WHERE _1 > 0") + val scans = df.queryExecution.executedPlan collect { + case s: CometScanExec => s + case s: CometBatchScanExec => s + } + assert(scans.size == 1, s"Expect one scan node but found ${scans.size}") + val metrics = scans.head.metrics + metricNames.foreach { metricName => + assert(metrics.contains(metricName), s"metric $metricName was not found") + } + + df.collect() + + metricNames.foreach { metricName => + assert( + metrics(metricName).value > 0, + s"Expect metric value for $metricName to be positive") + } + } + } + + test("read dictionary encoded decimals written as FIXED_LEN_BYTE_ARRAY") { + // In this test, data is encoded using Parquet page v2 format, but with PLAIN encoding + checkAnswer( + // Decimal column in this file is encoded using plain dictionary + readResourceParquetFile("test-data/dec-in-fixed-len.parquet"), + spark.range(1 << 4).select('id % 10 cast DecimalType(10, 2) as 'fixed_len_dec)) + } + + test("read long decimals with precision <= 9") { + // decimal32-written-as-64-bit.snappy.parquet was generated using a 3rd-party library. It has + // 10 rows of Decimal(9, 1) written as LongDecimal instead of an IntDecimal + var df = readResourceParquetFile("test-data/decimal32-written-as-64-bit.snappy.parquet") + assert(10 == df.collect().length) + var first10Df = df.head(10) + assert( + Seq(792059492, 986842987, 540247998, null, 357991078, 494131059, 92536396, 426847157, + -999999999, 204486094) + .zip(first10Df) + .forall(d => + d._2.isNullAt(0) && d._1 == null || + d._1 == d._2.getDecimal(0).unscaledValue().intValue())) + + // decimal32-written-as-64-bit-dict.snappy.parquet was generated using a 3rd-party library. It + // has 2048 rows of Decimal(3, 1) written as LongDecimal instead of an IntDecimal + df = readResourceParquetFile("test-data/decimal32-written-as-64-bit-dict.snappy.parquet") + assert(2048 == df.collect().length) + first10Df = df.head(10) + assert( + Seq(751, 937, 511, null, 337, 467, 84, 403, -999, 190) + .zip(first10Df) + .forall(d => + d._2.isNullAt(0) && d._1 == null || + d._1 == d._2.getDecimal(0).unscaledValue().intValue())) + + val last10Df = df.tail(10) + assert( + Seq(866, 20, 492, 76, 824, 604, 343, 820, 864, 243) + .zip(last10Df) + .forall(d => d._1 == d._2.getDecimal(0).unscaledValue().intValue())) + } + + private val actions: Seq[DataFrame => DataFrame] = Seq( + "_1 = 500", + "_1 = 500 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1000 or _1 = 1500", + "_1 >= 500 and _1 < 1000", + "(_1 >= 500 and _1 < 1000) or (_1 >= 1500 and _1 < 1600)").map(f => + (df: DataFrame) => df.filter(f)) + + test("test lazy materialization when batch size is small") { + val df = spark.range(0, 2000).selectExpr("id as _1", "cast(id as string) as _11") + checkParquetDataFrame(df)(actions: _*) + } + + test("test lazy materialization when batch size is small (dict encode)") { + val df = spark.range(0, 2000).selectExpr("id as _1", "cast(id % 10 as string) as _11") + checkParquetDataFrame(df)(actions: _*) + } + + private def testStandardAndLegacyModes(testName: String)(f: => Unit): Unit = { + test(s"Standard mode - $testName") { + withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false") { f } + } + + test(s"Legacy mode - $testName") { + withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "true") { f } + } + } + + private def checkParquetFile[T <: Product: ClassTag: TypeTag]( + data: Seq[T], + f: Row => Boolean = _ => true): Unit = { + withParquetDataFrame(data)(r => checkAnswer(r.filter(f), data.map(Row.fromTuple).filter(f))) + } + + protected def checkParquetScan[T <: Product: ClassTag: TypeTag]( + data: Seq[T], + f: Row => Boolean = _ => true): Unit + + /** + * create parquet file with various page sizes and batch sizes + */ + private def checkParquetDataFrame(df: DataFrame)(actions: (DataFrame => DataFrame)*): Unit = { + Seq(true, false).foreach { enableDictionary => + Seq(64, 127, 4049).foreach { pageSize => + withTempPath(file => { + df.coalesce(1) + .write + .option("parquet.page.size", pageSize.toString) + .option("parquet.enable.dictionary", enableDictionary.toString) + .parquet(file.getCanonicalPath) + + Seq(true, false).foreach { useLazyMaterialization => + Seq(true, false).foreach { enableCometExec => + Seq(4, 13, 4049).foreach { batchSize => + withSQLConf( + CometConf.COMET_BATCH_SIZE.key -> batchSize.toString, + CometConf.COMET_EXEC_ENABLED.key -> enableCometExec.toString, + CometConf.COMET_USE_LAZY_MATERIALIZATION.key -> useLazyMaterialization.toString) { + readParquetFile(file.getCanonicalPath) { parquetDf => + actions.foreach { action => + checkAnswer(action(parquetDf), action(df)) + } + } + } + } + } + } + }) + } + } + } + + test("row group skipping doesn't overflow when reading into larger type") { + assume(isSpark34Plus) + + withTempPath { path => + Seq(0).toDF("a").write.parquet(path.toString) + // Reading integer 'a' as a long isn't supported. Check that an exception is raised instead + // of incorrectly skipping the single row group and producing incorrect results. + val exception = intercept[SparkException] { + spark.read + .schema("a LONG") + .parquet(path.toString) + .where(s"a < ${Long.MaxValue}") + .collect() + } + assert(exception.getCause.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]) + } + } + + test("test pre-fetching multiple files") { + def makeRawParquetFile( + path: Path, + dictionaryEnabled: Boolean, + n: Int, + pageSize: Int): Seq[Option[Int]] = { + val schemaStr = + """ + |message root { + | optional boolean _1; + | optional int32 _2(INT_8); + | optional int32 _3(INT_16); + | optional int32 _4; + | optional int64 _5; + | optional float _6; + | optional double _7; + | optional binary _8(UTF8); + | optional int32 _9(UINT_8); + | optional int32 _10(UINT_16); + | optional int32 _11(UINT_32); + | optional int64 _12(UINT_64); + | optional binary _13(ENUM); + |} + """.stripMargin + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter( + schema, + path, + dictionaryEnabled = dictionaryEnabled, + pageSize = pageSize, + dictionaryPageSize = pageSize) + + val rand = scala.util.Random + val expected = (0 until n).map { i => + if (rand.nextBoolean()) { + None + } else { + Some(i) + } + } + expected.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i % 2 == 0) + record.add(1, i.toByte) + record.add(2, i.toShort) + record.add(3, i) + record.add(4, i.toLong) + record.add(5, i.toFloat) + record.add(6, i.toDouble) + record.add(7, i.toString * 48) + record.add(8, (-i).toByte) + record.add(9, (-i).toShort) + record.add(10, -i) + record.add(11, (-i).toLong) + record.add(12, i.toString) + case _ => + } + writer.write(record) + } + + writer.close() + expected + } + + val conf = new Configuration() + conf.set("spark.comet.scan.preFetch.enabled", "true"); + conf.set("spark.comet.scan.preFetch.threadNum", "4"); + + withTempDir { dir => + val threadPool = CometPrefetchThreadPool.getOrCreateThreadPool(2) + + val readers = (0 to 10).map { idx => + val path = new Path(dir.toURI.toString, s"part-r-$idx.parquet") + makeRawParquetFile(path, dictionaryEnabled = false, 10000, 500) + + val reader = new BatchReader(conf, path.toString, 1000, null, null) + reader.submitPrefetchTask(threadPool) + + reader + } + + // Wait for all pre-fetch tasks + readers.foreach { reader => + val task = reader.getPrefetchTask() + task.get() + } + + val totolRows = readers.map { reader => + val queue = reader.getPrefetchQueue() + var rowCount = 0L + + while (!queue.isEmpty) { + val rowGroup = queue.take().getLeft + rowCount += rowGroup.getRowCount + } + + reader.close() + + rowCount + }.sum + + readParquetFile(dir.toString) { df => + assert(df.count() == totolRows) + } + } + } + def testScanner(cometEnabled: String, scanner: String, v1: Option[String] = None): Unit = { + withSQLConf( + CometConf.COMET_ENABLED.key -> cometEnabled, + CometConf.COMET_EXEC_ENABLED.key -> cometEnabled, + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> cometEnabled, + SQLConf.USE_V1_SOURCE_LIST.key -> v1.getOrElse("")) { + withParquetTable(Seq((Long.MaxValue, 1), (Long.MaxValue, 2)), "tbl") { + val df = spark.sql("select * from tbl") + assert( + stripAQEPlan(df.queryExecution.executedPlan) + .collectLeaves() + .head + .toString() + .startsWith(scanner)) + } + } + } + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + Seq(true, false).foreach { prefetch => + val cometTestName = if (prefetch) { + testName + " (prefetch enabled)" + } else { + testName + } + + super.test(cometTestName, testTags: _*) { + withSQLConf(CometConf.COMET_SCAN_PREFETCH_ENABLED.key -> prefetch.toString) { + testFun + } + } + } + } +} + +class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + super.test(testName, testTags: _*)(withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + testFun + })(pos) + } + + override def checkParquetScan[T <: Product: ClassTag: TypeTag]( + data: Seq[T], + f: Row => Boolean = _ => true): Unit = { + withParquetDataFrame(data) { r => + val scans = collect(r.filter(f).queryExecution.executedPlan) { case p: CometScanExec => + p + } + if (CometConf.COMET_ENABLED.get()) { + assert(scans.nonEmpty) + } else { + assert(scans.isEmpty) + } + } + } + + test("Test V1 parquet scan uses respective scanner") { + Seq(("false", "FileScan parquet"), ("true", "CometScan parquet")).foreach { + case (cometEnabled, expectedScanner) => + testScanner(cometEnabled, scanner = expectedScanner, v1 = Some("parquet")) + } + } +} + +class ParquetReadV2Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + super.test(testName, testTags: _*)(withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + testFun + })(pos) + } + + override def checkParquetScan[T <: Product: ClassTag: TypeTag]( + data: Seq[T], + f: Row => Boolean = _ => true): Unit = { + withParquetDataFrame(data) { r => + val scans = collect(r.filter(f).queryExecution.executedPlan) { case p: CometBatchScanExec => + p.scan + } + if (CometConf.COMET_ENABLED.get()) { + assert(scans.nonEmpty && scans.forall(_.isInstanceOf[CometParquetScan])) + } else { + assert(!scans.exists(_.isInstanceOf[CometParquetScan])) + } + } + } + + test("Test V2 parquet scan uses respective scanner") { + Seq(("false", "BatchScan"), ("true", "CometBatchScan")).foreach { + case (cometEnabled, expectedScanner) => + testScanner(cometEnabled, scanner = expectedScanner, v1 = None) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/CometPluginsSuite.scala b/spark/src/test/scala/org/apache/spark/CometPluginsSuite.scala new file mode 100644 index 0000000000..a95674118f --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/CometPluginsSuite.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import org.apache.spark.sql.CometTestBase + +class CometPluginsSuite extends CometTestBase { + override protected def sparkConf: SparkConf = { + val conf = new SparkConf() + conf.set("spark.driver.memory", "1G") + conf.set("spark.executor.memory", "1G") + conf.set("spark.executor.memoryOverhead", "2G") + conf.set("spark.plugins", "org.apache.spark.CometPlugin") + conf.set("spark.comet.enabled", "true") + conf.set("spark.comet.exec.enabled", "true") + conf + } + + test("Default Comet memory overhead") { + val execMemOverhead1 = spark.conf.get("spark.executor.memoryOverhead") + val execMemOverhead2 = spark.sessionState.conf.getConfString("spark.executor.memoryOverhead") + val execMemOverhead3 = spark.sparkContext.getConf.get("spark.executor.memoryOverhead") + val execMemOverhead4 = spark.sparkContext.conf.get("spark.executor.memoryOverhead") + + // 2GB + 384MB (default Comet memory overhead) + assert(execMemOverhead1 == "2432M") + assert(execMemOverhead2 == "2432M") + assert(execMemOverhead3 == "2432M") + assert(execMemOverhead4 == "2432M") + } +} + +class CometPluginsDefaultSuite extends CometTestBase { + override protected def sparkConf: SparkConf = { + val conf = new SparkConf() + conf.set("spark.driver.memory", "1G") + conf.set("spark.executor.memory", "1G") + conf.set("spark.executor.memoryOverheadFactor", "0.5") + conf.set("spark.plugins", "org.apache.spark.CometPlugin") + conf.set("spark.comet.enabled", "true") + conf.set("spark.comet.exec.enabled", "true") + conf + } + + test("Default executor memory overhead + Comet memory overhead") { + val execMemOverhead1 = spark.conf.get("spark.executor.memoryOverhead") + val execMemOverhead2 = spark.sessionState.conf.getConfString("spark.executor.memoryOverhead") + val execMemOverhead3 = spark.sparkContext.getConf.get("spark.executor.memoryOverhead") + val execMemOverhead4 = spark.sparkContext.conf.get("spark.executor.memoryOverhead") + + // Spark executor memory overhead = executor memory (1G) * memoryOverheadFactor (0.5) = 512MB + // 512MB + 384MB (default Comet memory overhead) + assert(execMemOverhead1 == "896M") + assert(execMemOverhead2 == "896M") + assert(execMemOverhead3 == "896M") + assert(execMemOverhead4 == "896M") + } +} + +class CometPluginsNonOverrideSuite extends CometTestBase { + override protected def sparkConf: SparkConf = { + val conf = new SparkConf() + conf.set("spark.driver.memory", "1G") + conf.set("spark.executor.memory", "1G") + conf.set("spark.executor.memoryOverhead", "2G") + conf.set("spark.executor.memoryOverheadFactor", "0.5") + conf.set("spark.plugins", "org.apache.spark.CometPlugin") + conf.set("spark.comet.enabled", "true") + conf + } + + test("executor memory overhead is not overridden") { + val execMemOverhead1 = spark.conf.get("spark.executor.memoryOverhead") + val execMemOverhead2 = spark.sessionState.conf.getConfString("spark.executor.memoryOverhead") + val execMemOverhead3 = spark.sparkContext.getConf.get("spark.executor.memoryOverhead") + val execMemOverhead4 = spark.sparkContext.conf.get("spark.executor.memoryOverhead") + + assert(execMemOverhead1 == "2G") + assert(execMemOverhead2 == "2G") + assert(execMemOverhead3 == "2G") + assert(execMemOverhead4 == "2G") + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueriesList.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueriesList.scala new file mode 100644 index 0000000000..c83113231d --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQueriesList.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.benchmark.CometTPCDSQueryBenchmark.{nameSuffixForQueriesV2_7, tpcdsQueries, tpcdsQueriesV2_7} +import org.apache.spark.sql.benchmark.TPCDSSchemaHelper +import org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark.tables +import org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmarkArguments + +/** + * Utility to list Comet execution enabling status for TPCDS queries. + * + * To run this benchmark: + * {{{ + * // Build [tpcds-kit](https://github.com/databricks/tpcds-kit) + * cd /tmp && git clone https://github.com/databricks/tpcds-kit.git + * cd tpcds-kit/tools && make OS=MACOS + * + * // GenTPCDSData + * cd $COMET_HOME && mkdir /tmp/tpcds + * make benchmark-org.apache.spark.sql.GenTPCDSData -- --dsdgenDir /tmp/tpcds-kit/tools --location /tmp/tpcds --scaleFactor 1 + * + * // CometTPCDSQueriesList + * make benchmark-org.apache.spark.sql.CometTPCDSQueriesList -- --data-location /tmp/tpcds + * }}} + * + * Results will be written to "spark/inspections/CometTPCDSQueriesList-results.txt". + */ +object CometTPCDSQueriesList extends CometTPCQueryListBase with CometTPCQueryBase with Logging { + override def runSuite(mainArgs: Array[String]): Unit = { + val benchmarkArgs = new TPCDSQueryBenchmarkArguments(mainArgs) + + // If `--query-filter` defined, filters the queries that this option selects + val queriesV1_4ToRun = filterQueries(tpcdsQueries, benchmarkArgs.queryFilter) + val queriesV2_7ToRun = filterQueries( + tpcdsQueriesV2_7, + benchmarkArgs.queryFilter, + nameSuffix = nameSuffixForQueriesV2_7) + + if ((queriesV1_4ToRun ++ queriesV2_7ToRun).isEmpty) { + throw new RuntimeException( + s"Empty queries to run. Bad query name filter: ${benchmarkArgs.queryFilter}") + } + + setupTables( + benchmarkArgs.dataLocation, + createTempView = false, + tables, + TPCDSSchemaHelper.getTableColumns) + + setupCBO(cometSpark, benchmarkArgs.cboEnabled, tables) + + runQueries("tpcds", queries = queriesV1_4ToRun) + runQueries("tpcds-v2.7.0", queries = queriesV2_7ToRun, nameSuffix = nameSuffixForQueriesV2_7) + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala new file mode 100644 index 0000000000..f8213734bd --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import org.apache.spark.SparkConf + +import org.apache.comet.CometConf + +class CometTPCDSQuerySuite + extends { + // This is private in `TPCDSBase`. + val excludedTpcdsQueries: Seq[String] = Seq("q34", "q41", "q64") + + // This is private in `TPCDSBase` and `excludedTpcdsQueries` is private too. + // So we cannot override `excludedTpcdsQueries` to exclude the queries. + val tpcdsAllQueries: Seq[String] = Seq( + "q1", + "q2", + "q3", + "q4", + "q5", + "q6", + "q7", + "q8", + "q9", + "q10", + "q11", + "q12", + "q13", + "q14a", + "q14b", + "q15", + "q16", + "q17", + "q18", + "q19", + "q20", + "q21", + "q22", + "q23a", + "q23b", + "q24a", + "q24b", + "q25", + "q26", + "q27", + "q28", + "q29", + "q30", + "q31", + "q32", + "q33", + "q34", + "q35", + "q36", + "q37", + "q38", + "q39a", + "q39b", + "q40", + "q41", + "q42", + "q43", + "q44", + "q45", + "q46", + "q47", + "q48", + "q49", + "q50", + "q51", + "q52", + "q53", + "q54", + "q55", + "q56", + "q57", + "q58", + "q59", + "q60", + "q61", + "q62", + "q63", + "q64", + "q65", + "q66", + "q67", + "q68", + "q69", + "q70", + "q71", + "q72", + "q73", + "q74", + "q75", + "q76", + "q77", + "q78", + "q79", + "q80", + "q81", + "q82", + "q83", + "q84", + "q85", + "q86", + "q87", + "q88", + "q89", + "q90", + "q91", + "q92", + "q93", + "q94", + "q95", + "q96", + "q97", + "q98", + "q99") + + // TODO: enable the 3 queries after fixing the issues #1358. + override val tpcdsQueries: Seq[String] = + tpcdsAllQueries.filterNot(excludedTpcdsQueries.contains) + + } + with TPCDSQueryTestSuite { + override def sparkConf: SparkConf = { + val conf = super.sparkConf + conf.set("spark.sql.extensions", "org.apache.comet.CometSparkSessionExtensions") + conf.set(CometConf.COMET_ENABLED.key, "true") + conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") + conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "2g") + conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") + conf.set(CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key, "true") + conf + } + + override protected val baseResourcePath: String = { + getWorkspaceFilePath( + "spark", + "src", + "test", + "resources", + "tpcds-query-results").toFile.getAbsolutePath + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCHQueriesList.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCHQueriesList.scala new file mode 100644 index 0000000000..3d8e1ed02b --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCHQueriesList.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmarkArguments + +/** + * Utility to list Comet execution enabling status for TPCH queries. + * + * To run this benchmark: + * {{{ + * // Set scale factor in GB + * scale_factor=1 + * + * // GenTPCHData to create the data set at /tmp/tpch/sf1_parquet + * cd $COMET_HOME + * make benchmark-org.apache.spark.sql.GenTPCHData -- --location /tmp --scaleFactor ${scale_factor} + * + * // CometTPCHQueriesList + * make benchmark-org.apache.spark.sql.CometTPCHQueriesList -- --data-location /tmp/tpch/sf${scale_factor}_parquet + * }}} + * + * Results will be written to "spark/inspections/CometTPCHQueriesList-results.txt". + */ +object CometTPCHQueriesList extends CometTPCQueryListBase with CometTPCQueryBase with Logging { + val tables: Seq[String] = + Seq("customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier") + + override def runSuite(mainArgs: Array[String]): Unit = { + val benchmarkArgs = new TPCDSQueryBenchmarkArguments(mainArgs) + + // List of all TPC-H queries + val tpchQueries = (1 to 22).map(n => s"q$n") + + // If `--query-filter` defined, filters the queries that this option selects + val queries = filterQueries(tpchQueries, benchmarkArgs.queryFilter) + + if (queries.isEmpty) { + throw new RuntimeException( + s"Empty queries to run. Bad query name filter: ${benchmarkArgs.queryFilter}") + } + + setupTables(benchmarkArgs.dataLocation, createTempView = !benchmarkArgs.cboEnabled, tables) + + setupCBO(cometSpark, benchmarkArgs.cboEnabled, tables) + + runQueries("tpch", queries, " TPCH Snappy") + runQueries("tpch-extended", queries, " TPCH Extended Snappy") + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala new file mode 100644 index 0000000000..5f1ba6296a --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala @@ -0,0 +1,297 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import java.io.File +import java.nio.file.{Files, Paths} + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.util.{fileToString, resourceToString, stringToFile} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.{SharedSparkSession, TestSparkSession} + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus + +/** + * End-to-end tests to check TPCH query results. + * + * To run this test suite: + * {{{ + * SPARK_TPCH_DATA=<path of TPCH SF=1 data> + * ./mvnw -Dsuites=org.apache.spark.sql.CometTPCHQuerySuite test + * }}} + * + * To re-generate golden files for this suite, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 SPARK_TPCH_DATA=<path of TPCH SF=1 data> + * ./mvnw -Dsuites=org.apache.spark.sql.CometTPCHQuerySuite test + * }}} + */ +class CometTPCHQuerySuite extends QueryTest with CometTPCBase with SQLQueryTestHelper { + + private val tpchDataPath = sys.env.get("SPARK_TPCH_DATA") + + val tpchQueries: Seq[String] = Seq( + "q1", + "q2", + "q3", + "q4", + "q5", + "q6", + "q7", + "q8", + "q9", + "q10", + "q11", + "q12", + "q13", + "q14", + "q15", + "q16", + "q17", + "q18", + "q19", + "q20", + "q21", + "q22") + val disabledTpchQueries: Seq[String] = Seq("q1", "q15", "q17", "q18") + + // To make output results deterministic + def testSparkConf: SparkConf = { + val conf = super.sparkConf + conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "1") + conf.set("spark.sql.extensions", "org.apache.comet.CometSparkSessionExtensions") + conf.set(CometConf.COMET_ENABLED.key, "true") + conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") + conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "2g") + conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") + conf.set(CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key, "true") + } + + protected override def createSparkSession: TestSparkSession = { + new TestSparkSession(new SparkContext("local[1]", this.getClass.getSimpleName, testSparkConf)) + } + + val tableNames: Seq[String] = + Seq("customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier") + + // We use SF=1 table data here, so we cannot use SF=100 stats + protected override val injectStats: Boolean = false + + if (tpchDataPath.nonEmpty) { + val nonExistentTables = tableNames.filterNot { tableName => + Files.exists(Paths.get(s"${tpchDataPath.get}/$tableName")) + } + if (nonExistentTables.nonEmpty) { + fail( + s"Non-existent TPCH table paths found in ${tpchDataPath.get}: " + + nonExistentTables.mkString(", ")) + } + } + + protected val baseResourcePath: String = { + // use the same way as `SQLQueryTestSuite` to get the resource path + getWorkspaceFilePath( + "spark", + "src", + "test", + "resources", + "tpch-query-results").toFile.getAbsolutePath + } + + override def createTables(): Unit = { + tableNames.foreach { tableName => + spark.catalog.createTable(tableName, s"${tpchDataPath.get}/$tableName", "parquet") + } + } + + override def dropTables(): Unit = { + tableNames.foreach { tableName => + spark.sessionState.catalog.dropTable(TableIdentifier(tableName), true, true) + } + } + + private def runQuery(query: String, goldenFile: File, conf: Map[String, String]): Unit = { + val shouldSortResults = sortMergeJoinConf != conf // Sort for other joins + withSQLConf(conf.toSeq: _*) { + try { + val (schema, output) = handleExceptions(getNormalizedResult(spark, query)) + val queryString = query.trim + val outputString = output.mkString("\n").replaceAll("\\s+$", "") + if (shouldRegenerateGoldenFiles) { + val goldenOutput = { + s"-- Automatically generated by ${getClass.getSimpleName}\n\n" + + "-- !query schema\n" + + schema + "\n" + + "-- !query output\n" + + outputString + + "\n" + } + val parent = goldenFile.getParentFile + if (!parent.exists()) { + assert(parent.mkdirs(), "Could not create directory: " + parent) + } + stringToFile(goldenFile, goldenOutput) + } + + // Read back the golden file. + val (expectedSchema, expectedOutput) = { + val goldenOutput = fileToString(goldenFile) + val segments = goldenOutput.split("-- !query.*\n") + + // query has 3 segments, plus the header + assert( + segments.size == 3, + s"Expected 3 blocks in result file but got ${segments.size}. " + + "Try regenerate the result files.") + + (segments(1).trim, segments(2).replaceAll("\\s+$", "")) + } + + assertResult(expectedSchema, s"Schema did not match\n$queryString") { + schema + } + if (shouldSortResults) { + val expectSorted = expectedOutput + .split("\n") + .sorted + .map(_.trim) + .mkString("\n") + .replaceAll("\\s+$", "") + val outputSorted = output.sorted.map(_.trim).mkString("\n").replaceAll("\\s+$", "") + assertResult(expectSorted, s"Result did not match\n$queryString") { + outputSorted + } + } else { + assertResult(expectedOutput, s"Result did not match\n$queryString") { + outputString + } + } + } catch { + case e: Throwable => + val configs = conf.map { case (k, v) => + s"$k=$v" + } + throw new Exception(s"${e.getMessage}\nError using configs:\n${configs.mkString("\n")}") + } + } + } + + val sortMergeJoinConf: Map[String, String] = Map( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.PREFER_SORTMERGEJOIN.key -> "true") + + val broadcastHashJoinConf: Map[String, String] = Map( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10485760") + + val shuffledHashJoinConf: Map[String, String] = Map( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + "spark.sql.join.forceApplyShuffledHashJoin" -> "true") + + val allJoinConfCombinations: Seq[Map[String, String]] = + Seq(sortMergeJoinConf, broadcastHashJoinConf, shuffledHashJoinConf) + + val joinConfs: Seq[Map[String, String]] = if (shouldRegenerateGoldenFiles) { + require( + !sys.env.contains("SPARK_TPCH_JOIN_CONF"), + "'SPARK_TPCH_JOIN_CONF' cannot be set together with 'SPARK_GENERATE_GOLDEN_FILES'") + Seq(sortMergeJoinConf) + } else { + sys.env + .get("SPARK_TPCH_JOIN_CONF") + .map { s => + val p = new java.util.Properties() + p.load(new java.io.StringReader(s)) + Seq(p.asScala.toMap) + } + .getOrElse(allJoinConfCombinations) + } + + assert(joinConfs.nonEmpty) + joinConfs.foreach(conf => + require( + allJoinConfCombinations.contains(conf), + s"Join configurations [$conf] should be one of $allJoinConfCombinations")) + + if (tpchDataPath.nonEmpty) { + tpchQueries.foreach { name => + if (disabledTpchQueries.contains(name)) { + ignore(s"skipped because $name is disabled") {} + } else { + val queryString = resourceToString( + s"tpch/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + test(name) { + // Only run the tests in Spark 3.4+ + assume(isSpark34Plus) + + val goldenFile = new File(s"$baseResourcePath", s"$name.sql.out") + joinConfs.foreach { conf => + System.gc() // Workaround for GitHub Actions memory limitation, see also SPARK-37368 + runQuery(queryString, goldenFile, conf) + } + } + } + } + } else { + ignore("skipped because env `SPARK_TPCH_DATA` is not set") {} + } + + // TODO: remove once Spark 3.2 & 3.3 is no longer supported + private val shouldRegenerateGoldenFiles: Boolean = + System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" +} + +/** + * `TPCBase` doesn't exist in Spark 3.2. TODO: remove once Spark 3.2 is no longer supported + */ +trait CometTPCBase extends SharedSparkSession { + protected def injectStats: Boolean = false + + override protected def sparkConf: SparkConf = { + if (injectStats) { + super.sparkConf + .set(SQLConf.MAX_TO_STRING_FIELDS, Int.MaxValue) + .set(SQLConf.CBO_ENABLED, true) + .set(SQLConf.PLAN_STATS_ENABLED, true) + .set(SQLConf.JOIN_REORDER_ENABLED, true) + } else { + super.sparkConf.set(SQLConf.MAX_TO_STRING_FIELDS, Int.MaxValue) + } + } + + override def beforeAll(): Unit = { + super.beforeAll() + createTables() + } + + override def afterAll(): Unit = { + dropTables() + super.afterAll() + } + + protected def createTables(): Unit + + protected def dropTables(): Unit +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryBase.scala new file mode 100644 index 0000000000..8f83ac04b3 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryBase.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import scala.util.Try + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +import org.apache.comet.{CometConf, CometSparkSessionExtensions} + +/** + * Base trait for TPC related query execution + */ +trait CometTPCQueryBase extends Logging { + protected val cometSpark: SparkSession = { + val conf = new SparkConf() + .setMaster(System.getProperty("spark.sql.test.master", "local[*]")) + .setAppName(this.getClass.getSimpleName.stripSuffix("$")) + .set("spark.sql.parquet.compression.codec", "snappy") + .set( + "spark.sql.shuffle.partitions", + System.getProperty("spark.sql.shuffle.partitions", "4")) + .set("spark.driver.memory", "3g") + .set("spark.executor.memory", "3g") + .set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString) + .set("spark.sql.crossJoin.enabled", "true") + .setIfMissing("parquet.enable.dictionary", "true") + + val sparkSession = SparkSession.builder + .config(conf) + .withExtensions(new CometSparkSessionExtensions) + .getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + sparkSession.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(CometConf.COMET_ENABLED.key, "false") + sparkSession.conf.set(CometConf.COMET_EXEC_ENABLED.key, "false") + + sparkSession + } + + protected def setupTables( + dataLocation: String, + createTempView: Boolean, + tables: Seq[String], + tableColumns: Map[String, StructType] = Map.empty): Map[String, Long] = { + tables.map { tableName => + if (createTempView) { + cometSpark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) + } else { + cometSpark.sql(s"DROP TABLE IF EXISTS $tableName") + if (tableColumns.isEmpty) { + cometSpark.catalog.createTable(tableName, s"$dataLocation/$tableName", "parquet") + } else { + // SPARK-39584: Fix TPCDSQueryBenchmark Measuring Performance of Wrong Query Results + val options = Map("path" -> s"$dataLocation/$tableName") + cometSpark.catalog.createTable(tableName, "parquet", tableColumns(tableName), options) + } + // Recover partitions but don't fail if a table is not partitioned. + Try { + cometSpark.sql(s"ALTER TABLE $tableName RECOVER PARTITIONS") + }.getOrElse { + logInfo(s"Recovering partitions of table $tableName failed") + } + } + tableName -> cometSpark.table(tableName).count() + }.toMap + } + + protected def setupCBO(spark: SparkSession, cboEnabled: Boolean, tables: Seq[String]): Unit = { + if (cboEnabled) { + spark.sql(s"SET ${SQLConf.CBO_ENABLED.key}=true") + spark.sql(s"SET ${SQLConf.PLAN_STATS_ENABLED.key}=true") + spark.sql(s"SET ${SQLConf.JOIN_REORDER_ENABLED.key}=true") + spark.sql(s"SET ${SQLConf.HISTOGRAM_ENABLED.key}=true") + + // Analyze all the tables before running queries + val startTime = System.nanoTime() + tables.foreach { tableName => + spark.sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR ALL COLUMNS") + } + logInfo( + "The elapsed time to analyze all the tables is " + + s"${(System.nanoTime() - startTime) / NANOS_PER_SECOND.toDouble} seconds") + } else { + spark.sql(s"SET ${SQLConf.CBO_ENABLED.key}=false") + } + } + + protected def filterQueries( + origQueries: Seq[String], + queryFilter: Set[String], + nameSuffix: String = ""): Seq[String] = { + if (queryFilter.nonEmpty) { + if (nameSuffix.nonEmpty) { + origQueries.filter(name => queryFilter.contains(s"$name$nameSuffix")) + } else { + origQueries.filter(queryFilter.contains) + } + } else { + origQueries + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryListBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryListBase.scala new file mode 100644 index 0000000000..1f28b76a1e --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCQueryListBase.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import java.io.{File, FileOutputStream, OutputStream, PrintStream} + +import scala.collection.mutable + +import org.apache.commons.io.output.TeeOutputStream +import org.apache.spark.benchmark.Benchmarks +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.resourceToString +import org.apache.spark.sql.comet.CometExec +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + +import org.apache.comet.CometConf + +trait CometTPCQueryListBase + extends CometTPCQueryBase + with AdaptiveSparkPlanHelper + with SQLHelper { + var output: Option[OutputStream] = None + + def main(args: Array[String]): Unit = { + val resultFileName = + s"${this.getClass.getSimpleName.replace("$", "")}-results.txt" + val prefix = Benchmarks.currentProjectRoot.map(_ + "/").getOrElse("") + val dir = new File(s"${prefix}inspections/") + if (!dir.exists()) { + // scalastyle:off println + println(s"Creating ${dir.getAbsolutePath} for query inspection results.") + // scalastyle:on println + dir.mkdirs() + } + val file = new File(dir, resultFileName) + if (!file.exists()) { + file.createNewFile() + } + output = Some(new FileOutputStream(file)) + + runSuite(args) + + output.foreach { o => + if (o != null) { + o.close() + } + } + } + + protected def runQueries( + queryLocation: String, + queries: Seq[String], + nameSuffix: String = ""): Unit = { + + val out = if (output.isDefined) { + new PrintStream(new TeeOutputStream(System.out, output.get)) + } else { + System.out + } + + queries.foreach { name => + val queryString = resourceToString( + s"$queryLocation/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + + val df = cometSpark.sql(queryString) + val cometPlans = mutable.HashSet.empty[String] + stripAQEPlan(df.queryExecution.executedPlan).foreach { case op: CometExec => + cometPlans += s"${op.nodeName}" + } + + if (cometPlans.nonEmpty) { + out.println( + s"Query: $name$nameSuffix. Comet Exec: Enabled (${cometPlans.mkString(", ")})") + } else { + out.println(s"Query: $name$nameSuffix. Comet Exec: Disabled") + } + } + } + } + + protected def checkCometExec(df: DataFrame, f: SparkPlan => Unit): Unit = { + if (CometConf.COMET_ENABLED.get() && CometConf.COMET_EXEC_ENABLED.get()) { + f(stripAQEPlan(df.queryExecution.executedPlan)) + } + } + + def runSuite(mainArgs: Array[String]): Unit +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala new file mode 100644 index 0000000000..08c6cf419b --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -0,0 +1,597 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import scala.concurrent.duration._ +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + +import org.scalactic.source.Position +import org.scalatest.BeforeAndAfterEach +import org.scalatest.Tag + +import org.apache.hadoop.fs.Path +import org.apache.parquet.column.ParquetProperties +import org.apache.parquet.example.data.Group +import org.apache.parquet.example.data.simple.SimpleGroup +import org.apache.parquet.hadoop.ParquetWriter +import org.apache.parquet.hadoop.example.ExampleParquetWriter +import org.apache.parquet.schema.{MessageType, MessageTypeParser} +import org.apache.spark._ +import org.apache.spark.sql.comet.{CometBatchScanExec, CometExec, CometScanExec, CometScanWrapper, CometSinkPlaceHolder} +import org.apache.spark.sql.execution.{ColumnarToRowExec, InputAdapter, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.internal._ +import org.apache.spark.sql.test._ +import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.types.StructType + +import org.apache.comet._ +import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus + +/** + * Base class for testing. This exists in `org.apache.spark.sql` since [[SQLTestUtils]] is + * package-private. + */ +abstract class CometTestBase + extends QueryTest + with SQLTestUtils + with BeforeAndAfterEach + with AdaptiveSparkPlanHelper { + import testImplicits._ + + protected def sparkConf: SparkConf = { + val conf = new SparkConf() + conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) + conf.set(SQLConf.SHUFFLE_PARTITIONS, 10) // reduce parallelism in tests + conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "2g") + conf + } + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + super.test(testName, testTags: _*) { + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key -> "true", + SQLConf.ANSI_ENABLED.key -> "false") { + testFun + } + } + } + + /** + * A helper function for comparing Comet DataFrame with Spark result using absolute tolerance. + */ + protected def checkAnswerWithTol( + dataFrame: DataFrame, + expectedAnswer: Seq[Row], + absTol: Double): Unit = { + val actualAnswer = dataFrame.collect() + require( + actualAnswer.length == expectedAnswer.length, + s"actual num rows ${actualAnswer.length} != expected num of rows ${expectedAnswer.length}") + + actualAnswer.zip(expectedAnswer).foreach { case (actualRow, expectedRow) => + checkAnswerWithTol(actualRow, expectedRow, absTol) + } + } + + /** + * Compares two answers and makes sure the answer is within absTol of the expected result. + */ + protected def checkAnswerWithTol( + actualAnswer: Row, + expectedAnswer: Row, + absTol: Double): Unit = { + require( + actualAnswer.length == expectedAnswer.length, + s"actual answer length ${actualAnswer.length} != " + + s"expected answer length ${expectedAnswer.length}") + require(absTol > 0 && absTol <= 1e-6, s"absTol $absTol is out of range (0, 1e-6]") + + actualAnswer.toSeq.zip(expectedAnswer.toSeq).foreach { + case (actual: Double, expected: Double) => + if (!actual.isNaN && !expected.isNaN) { + assert( + math.abs(actual - expected) < absTol, + s"actual answer $actual not within $absTol of correct answer $expected") + } + case (actual, expected) => + assert(actual == expected, s"$actualAnswer did not equal $expectedAnswer") + } + } + + protected def checkSparkAnswer(query: String): Unit = { + checkSparkAnswer(sql(query)) + } + + protected def checkSparkAnswer(df: => DataFrame): Unit = { + var expected: Array[Row] = Array.empty + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val dfSpark = Dataset.ofRows(spark, df.logicalPlan) + expected = dfSpark.collect() + } + val dfComet = Dataset.ofRows(spark, df.logicalPlan) + checkAnswer(dfComet, expected) + } + + protected def checkSparkAnswerAndOperator(query: String, excludedClasses: Class[_]*): Unit = { + checkSparkAnswerAndOperator(sql(query), excludedClasses: _*) + } + + protected def checkSparkAnswerAndOperator( + df: => DataFrame, + excludedClasses: Class[_]*): Unit = { + checkCometOperators(stripAQEPlan(df.queryExecution.executedPlan), excludedClasses: _*) + checkSparkAnswer(df) + } + + protected def checkCometOperators(plan: SparkPlan, excludedClasses: Class[_]*): Unit = { + plan.foreach { + case _: CometScanExec | _: CometBatchScanExec => true + case _: CometSinkPlaceHolder | _: CometScanWrapper => false + case _: CometExec => true + case _: WholeStageCodegenExec | _: ColumnarToRowExec | _: InputAdapter => true + case op => + if (excludedClasses.exists(c => c.isAssignableFrom(op.getClass))) { + true + } else { + assert( + false, + s"Expected only Comet native operators, but found ${op.nodeName}.\n" + + s"plan: $plan") + } + } + } + + /** + * Check the answer of a Comet SQL query with Spark result using absolute tolerance. + */ + protected def checkSparkAnswerWithTol(query: String, absTol: Double = 1e-6): Unit = { + checkSparkAnswerWithTol(sql(query), absTol) + } + + /** + * Check the answer of a Comet DataFrame with Spark result using absolute tolerance. + */ + protected def checkSparkAnswerWithTol(df: => DataFrame, absTol: Double): Unit = { + var expected: Array[Row] = Array.empty + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val dfSpark = Dataset.ofRows(spark, df.logicalPlan) + expected = dfSpark.collect() + } + val dfComet = Dataset.ofRows(spark, df.logicalPlan) + checkAnswerWithTol(dfComet, expected, absTol: Double) + } + + private var _spark: SparkSession = _ + protected implicit def spark: SparkSession = _spark + protected implicit def sqlContext: SQLContext = _spark.sqlContext + + override protected def sparkContext: SparkContext = { + SparkContext.clearActiveContext() + + val conf = sparkConf + + if (!conf.contains("spark.master")) { + conf.setMaster("local[5]") + } + + if (!conf.contains("spark.app.name")) { + conf.setAppName(java.util.UUID.randomUUID().toString) + } + + SparkContext.getOrCreate(conf) + } + + protected def createSparkSession: SparkSession = { + SparkSession.cleanupAnyExistingSession() + + SparkSession + .builder() + .config( + sparkContext.getConf + ) // Don't use `sparkConf` as we can have overridden it in plugin + .withExtensions(new CometSparkSessionExtensions) + .getOrCreate() + } + + protected def initializeSession(): Unit = { + if (_spark == null) _spark = createSparkSession + } + + protected override def beforeAll(): Unit = { + initializeSession() + super.beforeAll() + } + + protected override def afterAll(): Unit = { + try { + super.afterAll() + } finally { + if (_spark != null) { + try { + _spark.stop() + } finally { + _spark = null + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + } + } + } + } + + protected override def beforeEach(): Unit = { + super.beforeEach() + DebugFilesystem.clearOpenStreams() + } + + protected override def afterEach(): Unit = { + super.afterEach() + spark.sharedState.cacheManager.clearCache() + eventually(timeout(10.seconds), interval(2.seconds)) { + DebugFilesystem.assertNoOpenStreams() + } + } + + protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet(getResourceParquetFilePath(name)) + } + + protected def getResourceParquetFilePath(name: String): String = { + Thread.currentThread().getContextClassLoader.getResource(name).toString + } + + protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag]( + data: Seq[T], + withDictionary: Boolean = true, + schema: Option[StructType] = None)(f: DataFrame => Unit): Unit = { + withParquetFile(data, withDictionary)(path => readParquetFile(path, schema)(f)) + } + + protected def withParquetTable[T <: Product: ClassTag: TypeTag]( + data: Seq[T], + tableName: String, + withDictionary: Boolean = true)(f: => Unit): Unit = { + withParquetDataFrame(data, withDictionary) { df => + df.createOrReplaceTempView(tableName) + withTempView(tableName)(f) + } + } + + protected def withParquetTable(df: DataFrame, tableName: String)(f: => Unit): Unit = { + df.createOrReplaceTempView(tableName) + withTempView(tableName)(f) + } + + protected def withParquetTable(path: String, tableName: String)(f: => Unit): Unit = { + val df = spark.read.format("parquet").load(path) + withParquetTable(df, tableName)(f) + } + + protected def withParquetFile[T <: Product: ClassTag: TypeTag]( + data: Seq[T], + withDictionary: Boolean = true)(f: String => Unit): Unit = { + withTempPath { file => + spark + .createDataFrame(data) + .write + .option("parquet.enable.dictionary", withDictionary.toString) + .parquet(file.getCanonicalPath) + f(file.getCanonicalPath) + } + } + + protected def readParquetFile(path: String, schema: Option[StructType] = None)( + f: DataFrame => Unit): Unit = schema match { + case Some(s) => f(spark.read.format("parquet").schema(s).load(path)) + case None => f(spark.read.format("parquet").load(path)) + } + + protected def createParquetWriter( + schema: MessageType, + path: Path, + dictionaryEnabled: Boolean = false, + pageSize: Int = 1024, + dictionaryPageSize: Int = 1024, + pageRowCountLimit: Int = ParquetProperties.DEFAULT_PAGE_ROW_COUNT_LIMIT, + rowGroupSize: Long = 1024 * 1024L): ParquetWriter[Group] = { + val hadoopConf = spark.sessionState.newHadoopConf() + + ExampleParquetWriter + .builder(path) + .withDictionaryEncoding(dictionaryEnabled) + .withType(schema) + .withRowGroupSize(rowGroupSize.toInt) + .withPageSize(pageSize) + .withDictionaryPageSize(dictionaryPageSize) + .withPageRowCountLimit(pageRowCountLimit) + .withConf(hadoopConf) + .build() + } + + // Maps `i` to both positive and negative to test timestamp after and before the Unix epoch + protected def getValue(i: Long, div: Long): Long = { + val value = if (i % 2 == 0) i else -i + value % div + } + + def makeParquetFileAllTypes(path: Path, dictionaryEnabled: Boolean, n: Int): Unit = { + makeParquetFileAllTypes(path, dictionaryEnabled, 0, n) + } + + def makeParquetFileAllTypes( + path: Path, + dictionaryEnabled: Boolean, + begin: Int, + end: Int, + pageSize: Int = 128, + randomSize: Int = 0): Unit = { + val schemaStr = + if (isSpark34Plus) { + """ + |message root { + | optional boolean _1; + | optional int32 _2(INT_8); + | optional int32 _3(INT_16); + | optional int32 _4; + | optional int64 _5; + | optional float _6; + | optional double _7; + | optional binary _8(UTF8); + | optional int32 _9(UINT_8); + | optional int32 _10(UINT_16); + | optional int32 _11(UINT_32); + | optional int64 _12(UINT_64); + | optional binary _13(ENUM); + | optional FIXED_LEN_BYTE_ARRAY(3) _14; + | optional int32 _15(DECIMAL(5, 2)); + | optional int64 _16(DECIMAL(18, 10)); + | optional FIXED_LEN_BYTE_ARRAY(16) _17(DECIMAL(38, 37)); + | optional INT64 _18(TIMESTAMP(MILLIS,true)); + | optional INT64 _19(TIMESTAMP(MICROS,true)); + | optional INT32 _20(DATE); + |} + """.stripMargin + } else { + """ + |message root { + | optional boolean _1; + | optional int32 _2(INT_8); + | optional int32 _3(INT_16); + | optional int32 _4; + | optional int64 _5; + | optional float _6; + | optional double _7; + | optional binary _8(UTF8); + | optional int32 _9(UINT_8); + | optional int32 _10(UINT_16); + | optional int32 _11(UINT_32); + | optional int64 _12(UINT_64); + | optional binary _13(ENUM); + | optional binary _14(UTF8); + | optional int32 _15(DECIMAL(5, 2)); + | optional int64 _16(DECIMAL(18, 10)); + | optional FIXED_LEN_BYTE_ARRAY(16) _17(DECIMAL(38, 37)); + | optional INT64 _18(TIMESTAMP(MILLIS,true)); + | optional INT64 _19(TIMESTAMP(MICROS,true)); + | optional INT32 _20(DATE); + |} + """.stripMargin + } + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter( + schema, + path, + dictionaryEnabled = dictionaryEnabled, + pageSize = pageSize, + dictionaryPageSize = pageSize) + + val rand = scala.util.Random + val data = (begin until end).map { i => + if (rand.nextBoolean()) { + None + } else { + if (dictionaryEnabled) Some(i % 4) else Some(i) + } + } + data.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i % 2 == 0) + record.add(1, i.toByte) + record.add(2, i.toShort) + record.add(3, i) + record.add(4, i.toLong) + record.add(5, i.toFloat) + record.add(6, i.toDouble) + record.add(7, i.toString * 48) + record.add(8, (-i).toByte) + record.add(9, (-i).toShort) + record.add(10, -i) + record.add(11, (-i).toLong) + record.add(12, i.toString) + record.add(13, ((i % 10).toString * 3).take(3)) + record.add(14, i) + record.add(15, i.toLong) + record.add(16, ((i % 10).toString * 16).take(16)) + record.add(17, i.toLong) + record.add(18, i.toLong) + record.add(19, i) + case _ => + } + writer.write(record) + } + (0 until randomSize).foreach { _ => + val i = rand.nextLong() + val record = new SimpleGroup(schema) + record.add(0, i % 2 == 0) + record.add(1, i.toByte) + record.add(2, i.toShort) + record.add(3, i.toInt) + record.add(4, i) + record.add(5, java.lang.Float.intBitsToFloat(i.toInt)) + record.add(6, java.lang.Double.longBitsToDouble(i)) + record.add(7, i.toString * 24) + record.add(8, (-i).toByte) + record.add(9, (-i).toShort) + record.add(10, (-i).toInt) + record.add(11, -i) + record.add(12, i.toString) + record.add(13, i.toString.take(3).padTo(3, '0')) + record.add(14, i.toInt % 100000) + record.add(15, i % 1000000000000000000L) + record.add(16, i.toString.take(16).padTo(16, '0')) + record.add(17, i) + record.add(18, i) + record.add(19, i.toInt) + writer.write(record) + } + + writer.close() + } + + protected def makeRawTimeParquetFileColumns( + path: Path, + dictionaryEnabled: Boolean, + n: Int, + rowGroupSize: Long = 1024 * 1024L): Seq[Option[Long]] = { + val schemaStr = + """ + |message root { + | optional int64 _0(INT_64); + | optional int64 _1(INT_64); + | optional int64 _2(INT_64); + | optional int64 _3(INT_64); + | optional int64 _4(INT_64); + | optional int64 _5(INT_64); + |} + """.stripMargin + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter( + schema, + path, + dictionaryEnabled = dictionaryEnabled, + rowGroupSize = rowGroupSize) + val div = if (dictionaryEnabled) 10 else n // maps value to a small range for dict to kick in + + val rand = scala.util.Random + val expected = (0 until n).map { i => + if (rand.nextBoolean()) { + None + } else { + Some(getValue(i, div)) + } + } + expected.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i) + record.add(1, i * 1000) // convert millis to micros, same below + record.add(2, i) + record.add(3, i) + record.add(4, i * 1000) + record.add(5, i * 1000) + case _ => + } + writer.write(record) + } + + writer.close() + expected + } + + // Creates Parquet file of timestamp values + protected def makeRawTimeParquetFile( + path: Path, + dictionaryEnabled: Boolean, + n: Int, + rowGroupSize: Long = 1024 * 1024L): Seq[Option[Long]] = { + val schemaStr = + """ + |message root { + | optional int64 _0(TIMESTAMP_MILLIS); + | optional int64 _1(TIMESTAMP_MICROS); + | optional int64 _2(TIMESTAMP(MILLIS,true)); + | optional int64 _3(TIMESTAMP(MILLIS,false)); + | optional int64 _4(TIMESTAMP(MICROS,true)); + | optional int64 _5(TIMESTAMP(MICROS,false)); + | optional int64 _6(INT_64); + |} + """.stripMargin + + val schema = MessageTypeParser.parseMessageType(schemaStr) + val writer = createParquetWriter( + schema, + path, + dictionaryEnabled = dictionaryEnabled, + rowGroupSize = rowGroupSize) + val div = if (dictionaryEnabled) 10 else n // maps value to a small range for dict to kick in + + val rand = scala.util.Random + val expected = (0 until n).map { i => + if (rand.nextBoolean()) { + None + } else { + Some(getValue(i, div)) + } + } + expected.foreach { opt => + val record = new SimpleGroup(schema) + opt match { + case Some(i) => + record.add(0, i) + record.add(1, i * 1000) // convert millis to micros, same below + record.add(2, i) + record.add(3, i) + record.add(4, i * 1000) + record.add(5, i * 1000) + record.add(6, i * 1000) + case _ => + } + writer.write(record) + } + + writer.close() + expected + } + + def makeDecimalRDD(num: Int, decimal: DecimalType, useDictionary: Boolean): DataFrame = { + val div = if (useDictionary) 5 else num // narrow the space to make it dictionary encoded + spark + .range(num) + .map(_ % div) + // Parquet doesn't allow column names with spaces, have to add an alias here. + // Minus 500 here so that negative decimals are also tested. + .select((($"value" - 500) / 100.0) cast decimal as Symbol("dec")) + .coalesce(1) + } + + def stripRandomPlanParts(plan: String): String = { + plan.replaceFirst("file:.*,", "").replaceAll(raw"#\d+", "") + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/GenTPCHData.scala b/spark/src/test/scala/org/apache/spark/sql/GenTPCHData.scala new file mode 100644 index 0000000000..1dcbc743ba --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/GenTPCHData.scala @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +import java.io.{File, PrintWriter} +import java.nio.file.Files +import java.nio.file.Path + +import scala.sys.process._ +import scala.util.Try + +import org.apache.commons.io.FileUtils +import org.apache.spark.deploy.SparkHadoopUtil + +/** + * This class generates TPCH table data by using tpch-dbgen: + * - https://github.com/databricks/tpch-dbgen + * + * To run this: + * {{{ + * make benchmark-org.apache.spark.sql.GenTPCHData -- --location <path> --scaleFactor 1 + * }}} + */ +object GenTPCHData { + val TEMP_DBGEN_DIR: Path = new File("/tmp").toPath + val DBGEN_DIR_PREFIX = "tempTPCHGen" + + def main(args: Array[String]): Unit = { + val config = new GenTPCHDataConfig(args) + + val spark = SparkSession + .builder() + .appName(getClass.getName) + .master(config.master) + .getOrCreate() + + setScaleConfig(spark, config.scaleFactor) + + // Number of worker nodes + val workers = spark.sparkContext.getExecutorMemoryStatus.size + + var defaultDbgenDir: File = null + + val dbgenDir = if (config.dbgenDir == null) { + defaultDbgenDir = Files.createTempDirectory(TEMP_DBGEN_DIR, DBGEN_DIR_PREFIX).toFile + val baseDir = defaultDbgenDir.getAbsolutePath + defaultDbgenDir.delete() + // Install the data generators in all nodes + // TODO: think a better way to install on each worker node + // such as https://stackoverflow.com/a/40876671 + spark.range(0, workers, 1, workers).foreach(worker => installDBGEN(baseDir)(worker)) + s"${baseDir}/dbgen" + } else { + config.dbgenDir + } + + val tables = new TPCHTables(spark.sqlContext, dbgenDir, config.scaleFactor.toString) + + // Generate data + // Since dbgen may uses stdout to output the data, tables.genData needs to run table by table + val tableNames = + if (config.tableFilter.isBlank) tables.tables.map(_.name) else Seq(config.tableFilter) + tableNames.foreach { tableName => + tables.genData( + location = s"${config.location}/tpch/sf${config.scaleFactor}_${config.format}", + format = config.format, + overwrite = config.overwrite, + partitionTables = config.partitionTables, + clusterByPartitionColumns = config.clusterByPartitionColumns, + filterOutNullPartitionValues = config.filterOutNullPartitionValues, + tableFilter = tableName, + numPartitions = config.numPartitions) + } + + // Clean up + if (defaultDbgenDir != null) { + spark.range(0, workers, 1, workers).foreach { _ => + val _ = FileUtils.deleteQuietly(defaultDbgenDir) + } + } + + spark.stop() + } + + def setScaleConfig(spark: SparkSession, scaleFactor: Int): Unit = { + // Avoid OOM when shuffling large scale factors and errors like 2GB shuffle limit at 10TB like: + // org.apache.spark.shuffle.FetchFailedException: Too large frame: 9640891355 + // For 10TB 16x4core nodes were needed with the config below, 8x for 1TB and below. + // About 24hrs. for SF 1 to 10,000. + if (scaleFactor >= 10000) { + spark.conf.set("spark.sql.shuffle.partitions", "20000") + SparkHadoopUtil.get.conf.set("parquet.memory.pool.ratio", "0.1") + } else if (scaleFactor >= 1000) { + spark.conf.set( + "spark.sql.shuffle.partitions", + "2001" + ) // one above 2000 to use HighlyCompressedMapStatus + SparkHadoopUtil.get.conf.set("parquet.memory.pool.ratio", "0.3") + } else { + spark.conf.set("spark.sql.shuffle.partitions", "200") // default + SparkHadoopUtil.get.conf.set("parquet.memory.pool.ratio", "0.5") + } + } + + // Install tpch-dbgen (with the stdout patch) + def installDBGEN( + baseDir: String, + url: String = "https://github.com/databricks/tpch-dbgen.git", + useStdout: Boolean = true)(i: java.lang.Long): Unit = { + // Check if we want the revision which makes dbgen output to stdout + val checkoutRevision: String = + if (useStdout) "git checkout 0469309147b42abac8857fa61b4cf69a6d3128a8 -- bm_utils.c" else "" + + Seq("mkdir", "-p", baseDir).! + val pw = new PrintWriter(s"${baseDir}/dbgen_$i.sh") + pw.write(s""" + |rm -rf ${baseDir}/dbgen + |rm -rf ${baseDir}/dbgen_install_$i + |mkdir ${baseDir}/dbgen_install_$i + |cd ${baseDir}/dbgen_install_$i + |git clone '$url' + |cd tpch-dbgen + |$checkoutRevision + |sed -i'' -e 's/#include <malloc\\.h>/#ifndef __APPLE__\\n#include <malloc\\.h>\\n#endif/' bm_utils.c + |sed -i'' -e 's/#include <malloc\\.h>/#if defined(__MACH__)\\n#include <stdlib\\.h>\\n#else\\n#include <malloc\\.h>\\n#endif/' varsub.c + |make + |ln -sf ${baseDir}/dbgen_install_$i/tpch-dbgen ${baseDir}/dbgen || echo "ln -sf failed" + |test -e ${baseDir}/dbgen/dbgen + |echo "OK" + """.stripMargin) + pw.close + Seq("chmod", "+x", s"${baseDir}/dbgen_$i.sh").! + Seq(s"${baseDir}/dbgen_$i.sh").!! + } +} + +class GenTPCHDataConfig(args: Array[String]) { + var master: String = "local[*]" + var dbgenDir: String = null + var location: String = null + var scaleFactor: Int = 1 + var format: String = "parquet" + var overwrite: Boolean = false + var partitionTables: Boolean = false + var clusterByPartitionColumns: Boolean = false + var filterOutNullPartitionValues: Boolean = false + var tableFilter: String = "" + var numPartitions: Int = 100 + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + var args = inputArgs + + while (args.nonEmpty) { + args match { + case "--master" :: value :: tail => + master = value + args = tail + + case "--dbgenDir" :: value :: tail => + dbgenDir = value + args = tail + + case "--location" :: value :: tail => + location = value + args = tail + + case "--scaleFactor" :: value :: tail => + scaleFactor = toPositiveIntValue("Scale factor", value) + args = tail + + case "--format" :: value :: tail => + format = value + args = tail + + case "--overwrite" :: tail => + overwrite = true + args = tail + + case "--partitionTables" :: tail => + partitionTables = true + args = tail + + case "--clusterByPartitionColumns" :: tail => + clusterByPartitionColumns = true + args = tail + + case "--filterOutNullPartitionValues" :: tail => + filterOutNullPartitionValues = true + args = tail + + case "--tableFilter" :: value :: tail => + tableFilter = value + args = tail + + case "--numPartitions" :: value :: tail => + numPartitions = toPositiveIntValue("Number of partitions", value) + args = tail + + case "--help" :: _ => + printUsageAndExit(0) + + case _ => + // scalastyle:off println + System.err.println("Unknown/unsupported param " + args) + // scalastyle:on println + printUsageAndExit(1) + } + } + + checkRequiredArguments() + } + + private def printUsageAndExit(exitCode: Int): Unit = { + // scalastyle:off + System.err.println(""" + |make benchmark-org.apache.spark.sql.GenTPCHData -- [Options] + |Options: + | --master the Spark master to use, default to local[*] + | --dbgenDir location of dbgen on worker nodes, if not provided, installs default dbgen + | --location root directory of location to generate data in + | --scaleFactor size of the dataset to generate (in GB) + | --format generated data format, Parquet, ORC ... + | --overwrite whether to overwrite the data that is already there + | --partitionTables whether to create the partitioned fact tables + | --clusterByPartitionColumns whether to shuffle to get partitions coalesced into single files + | --filterOutNullPartitionValues whether to filter out the partition with NULL key value + | --tableFilter comma-separated list of table names to generate (e.g., store_sales,store_returns), + | all the tables are generated by default + | --numPartitions how many dbgen partitions to run - number of input tasks + """.stripMargin) + // scalastyle:on + System.exit(exitCode) + } + + private def toPositiveIntValue(name: String, v: String): Int = { + if (Try(v.toInt).getOrElse(-1) <= 0) { + // scalastyle:off println + System.err.println(s"$name must be a positive number") + // scalastyle:on println + printUsageAndExit(-1) + } + v.toInt + } + + private def checkRequiredArguments(): Unit = { + if (location == null) { + // scalastyle:off println + System.err.println("Must specify an output location") + // scalastyle:on println + printUsageAndExit(-1) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/TPCDSQueries.scala b/spark/src/test/scala/org/apache/spark/sql/TPCDSQueries.scala new file mode 100644 index 0000000000..7db65669d1 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/TPCDSQueries.scala @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +trait TPCDSQueries { + // List of all TPC-DS v1.4 queries + val tpcdsQueries: Seq[String] = Seq( + "q1", + "q2", + "q3", + "q4", + "q5", + "q6", + "q7", + "q8", + "q9", + "q10", + "q11", + "q12", + "q13", + "q14a", + "q14b", + "q15", + "q16", + "q17", + "q18", + "q19", + "q20", + "q21", + "q22", + "q23a", + "q23b", + "q24a", + "q24b", + "q25", + "q26", + "q27", + "q28", + "q29", + "q30", + "q31", + "q32", + "q33", + "q34", + "q35", + "q36", + "q37", + "q38", + "q39a", + "q39b", + "q40", + "q41", + "q42", + "q43", + "q44", + "q45", + "q46", + "q47", + "q48", + "q49", + "q50", + "q51", + "q52", + "q53", + "q54", + "q55", + "q56", + "q57", + "q58", + "q59", + "q60", + "q61", + "q62", + "q63", + "q64", + "q65", + "q66", + "q67", + "q68", + "q69", + "q70", + "q71", + "q72", + "q73", + "q74", + "q75", + "q76", + "q77", + "q78", + "q79", + "q80", + "q81", + "q82", + "q83", + "q84", + "q85", + "q86", + "q87", + "q88", + "q89", + "q90", + "q91", + "q92", + "q93", + "q94", + "q95", + "q96", + "q97", + "q98", + "q99") + + // This list only includes TPC-DS v2.7 queries that are different from v1.4 ones + val nameSuffixForQueriesV2_7 = "-v2.7" + val tpcdsQueriesV2_7: Seq[String] = Seq( + "q5a", + "q6", + "q10a", + "q11", + "q12", + "q14", + "q14a", + "q18a", + "q20", + "q22", + "q22a", + "q24", + "q27a", + "q34", + "q35", + "q35a", + "q36a", + "q47", + "q49", + "q51a", + "q57", + "q64", + "q67a", + "q70a", + "q72", + "q74", + "q75", + "q77a", + "q78", + "q80a", + "q86a", + "q98") +} diff --git a/spark/src/test/scala/org/apache/spark/sql/TPCH.scala b/spark/src/test/scala/org/apache/spark/sql/TPCH.scala new file mode 100644 index 0000000000..31e5cf7980 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/TPCH.scala @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +/** + * Mostly copied from + * https://github.com/databricks/spark-sql-perf/blob/master/src/main/scala/com/databricks/spark/sql/perf/tpch/TPCH.scala + * + * TODO: port this back to the upstream Spark similar to TPCDS benchmark + */ + +package org.apache.spark.sql + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD + +class Dbgen(dbgenDir: String, params: Seq[String]) extends DataGenerator { + val dbgen: String = s"$dbgenDir/dbgen" + + def generate( + sparkContext: SparkContext, + name: String, + partitions: Int, + scaleFactor: String): RDD[String] = { + val smallTables = Seq("nation", "region") + val numPartitions = if (partitions > 1 && !smallTables.contains(name)) partitions else 1 + val generatedData = { + sparkContext + .parallelize(1 to numPartitions, numPartitions) + .flatMap { i => + val localToolsDir = if (new java.io.File(dbgen).exists) { + dbgenDir + } else if (new java.io.File(s"/$dbgenDir").exists) { + s"/$dbgenDir" + } else { + sys.error(s"Could not find dbgen at $dbgen or /$dbgenDir. Run install") + } + val parallel = if (numPartitions > 1) s"-C $partitions -S $i" else "" + val shortTableNames = Map( + "customer" -> "c", + "lineitem" -> "L", + "nation" -> "n", + "orders" -> "O", + "part" -> "P", + "region" -> "r", + "supplier" -> "s", + "partsupp" -> "S") + val paramsString = params.mkString(" ") + val commands = Seq( + "bash", + "-c", + s"cd $localToolsDir && ./dbgen -q $paramsString -T ${shortTableNames(name)} " + + s"-s $scaleFactor $parallel") + println(commands) + BlockingLineStream(commands) + } + .repartition(numPartitions) + } + + generatedData.setName(s"$name, sf=$scaleFactor, strings") + generatedData + } +} + +class TPCHTables( + sqlContext: SQLContext, + dbgenDir: String, + scaleFactor: String, + useDoubleForDecimal: Boolean = false, + useStringForDate: Boolean = false, + generatorParams: Seq[String] = Nil) + extends Tables(sqlContext, scaleFactor, useDoubleForDecimal, useStringForDate) { + import sqlContext.implicits._ + + val dataGenerator = new Dbgen(dbgenDir, generatorParams) + + val tables: Seq[Table] = Seq( + Table( + "part", + partitionColumns = "p_brand" :: Nil, + 'p_partkey.long, + 'p_name.string, + 'p_mfgr.string, + 'p_brand.string, + 'p_type.string, + 'p_size.int, + 'p_container.string, + 'p_retailprice.decimal(12, 2), + 'p_comment.string), + Table( + "supplier", + partitionColumns = Nil, + 's_suppkey.long, + 's_name.string, + 's_address.string, + 's_nationkey.long, + 's_phone.string, + 's_acctbal.decimal(12, 2), + 's_comment.string), + Table( + "partsupp", + partitionColumns = Nil, + 'ps_partkey.long, + 'ps_suppkey.long, + 'ps_availqty.int, + 'ps_supplycost.decimal(12, 2), + 'ps_comment.string), + Table( + "customer", + partitionColumns = "c_mktsegment" :: Nil, + 'c_custkey.long, + 'c_name.string, + 'c_address.string, + 'c_nationkey.long, + 'c_phone.string, + 'c_acctbal.decimal(12, 2), + 'c_mktsegment.string, + 'c_comment.string), + Table( + "orders", + partitionColumns = "o_orderdate" :: Nil, + 'o_orderkey.long, + 'o_custkey.long, + 'o_orderstatus.string, + 'o_totalprice.decimal(12, 2), + 'o_orderdate.date, + 'o_orderpriority.string, + 'o_clerk.string, + 'o_shippriority.int, + 'o_comment.string), + Table( + "lineitem", + partitionColumns = "l_shipdate" :: Nil, + 'l_orderkey.long, + 'l_partkey.long, + 'l_suppkey.long, + 'l_linenumber.int, + 'l_quantity.decimal(12, 2), + 'l_extendedprice.decimal(12, 2), + 'l_discount.decimal(12, 2), + 'l_tax.decimal(12, 2), + 'l_returnflag.string, + 'l_linestatus.string, + 'l_shipdate.date, + 'l_commitdate.date, + 'l_receiptdate.date, + 'l_shipinstruct.string, + 'l_shipmode.string, + 'l_comment.string), + Table( + "nation", + partitionColumns = Nil, + 'n_nationkey.long, + 'n_name.string, + 'n_regionkey.long, + 'n_comment.string), + Table("region", partitionColumns = Nil, 'r_regionkey.long, 'r_name.string, 'r_comment.string)) + .map(_.convertTypes()) +} diff --git a/spark/src/test/scala/org/apache/spark/sql/Tables.scala b/spark/src/test/scala/org/apache/spark/sql/Tables.scala new file mode 100644 index 0000000000..138238da80 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/Tables.scala @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +/** + * Mostly copied from + * https://github.com/databricks/spark-sql-perf/blob/master/src/main/scala/com/databricks/spark/sql/perf/Tables.scala + * + * TODO: port this back to the upstream Spark similar to TPCDS benchmark + */ + +package org.apache.spark.sql + +import scala.util.Try + +import org.slf4j.LoggerFactory + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +trait DataGenerator extends Serializable { + def generate( + sparkContext: SparkContext, + name: String, + partitions: Int, + scaleFactor: String): RDD[String] +} + +abstract class Tables( + sqlContext: SQLContext, + scaleFactor: String, + useDoubleForDecimal: Boolean = false, + useStringForDate: Boolean = false) + extends Serializable { + + def dataGenerator: DataGenerator + def tables: Seq[Table] + + private val log = LoggerFactory.getLogger(getClass) + + def sparkContext = sqlContext.sparkContext + + case class Table(name: String, partitionColumns: Seq[String], fields: StructField*) { + val schema: StructType = StructType(fields) + + def nonPartitioned: Table = { + Table(name, Nil, fields: _*) + } + + /** + * If convertToSchema is true, the data from generator will be parsed into columns and + * converted to `schema`. Otherwise, it just outputs the raw data (as a single STRING column). + */ + def df(convertToSchema: Boolean, numPartition: Int): DataFrame = { + val generatedData = dataGenerator.generate(sparkContext, name, numPartition, scaleFactor) + val rows = generatedData.mapPartitions { iter => + iter.map { l => + if (convertToSchema) { + val values = l.split("\\|", -1).dropRight(1).map { v => + if (v.equals("")) { + // If the string value is an empty string, we turn it to a null + null + } else { + v + } + } + Row.fromSeq(values) + } else { + Row.fromSeq(Seq(l)) + } + } + } + + if (convertToSchema) { + val stringData = + sqlContext.createDataFrame( + rows, + StructType(schema.fields.map(f => StructField(f.name, StringType)))) + + val convertedData = { + val columns = schema.fields.map { f => + col(f.name).cast(f.dataType).as(f.name) + } + stringData.select(columns: _*) + } + + convertedData + } else { + sqlContext.createDataFrame(rows, StructType(Seq(StructField("value", StringType)))) + } + } + + def convertTypes(): Table = { + val newFields = fields.map { field => + val newDataType = field.dataType match { + case _: DecimalType if useDoubleForDecimal => DoubleType + case _: DateType if useStringForDate => StringType + case other => other + } + field.copy(dataType = newDataType) + } + + Table(name, partitionColumns, newFields: _*) + } + + def genData( + location: String, + format: String, + overwrite: Boolean, + clusterByPartitionColumns: Boolean, + filterOutNullPartitionValues: Boolean, + numPartitions: Int): Unit = { + val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Ignore + + val data = df(format != "text", numPartitions) + val tempTableName = s"${name}_text" + data.createOrReplaceTempView(tempTableName) + + val writer = if (partitionColumns.nonEmpty) { + if (clusterByPartitionColumns) { + val columnString = data.schema.fields + .map { field => + field.name + } + .mkString(",") + val partitionColumnString = partitionColumns.mkString(",") + val predicates = if (filterOutNullPartitionValues) { + partitionColumns.map(col => s"$col IS NOT NULL").mkString("WHERE ", " AND ", "") + } else { + "" + } + + val query = + s""" + |SELECT + | $columnString + |FROM + | $tempTableName + |$predicates + |DISTRIBUTE BY + | $partitionColumnString + """.stripMargin + val grouped = sqlContext.sql(query) + println(s"Pre-clustering with partitioning columns with query $query.") + log.info(s"Pre-clustering with partitioning columns with query $query.") + grouped.write + } else { + data.write + } + } else { + // treat non-partitioned tables as "one partition" that we want to coalesce + if (clusterByPartitionColumns) { + // in case data has more than maxRecordsPerFile, split into multiple writers to improve + // datagen speed files will be truncated to maxRecordsPerFile value, so the final result + // will be the same + val numRows = data.count + val maxRecordPerFile = + Try(sqlContext.getConf("spark.sql.files.maxRecordsPerFile").toInt).getOrElse(0) + + println( + s"Data has $numRows rows clustered $clusterByPartitionColumns for $maxRecordPerFile") + log.info( + s"Data has $numRows rows clustered $clusterByPartitionColumns for $maxRecordPerFile") + + if (maxRecordPerFile > 0 && numRows > maxRecordPerFile) { + val numFiles = (numRows.toDouble / maxRecordPerFile).ceil.toInt + println(s"Coalescing into $numFiles files") + log.info(s"Coalescing into $numFiles files") + data.coalesce(numFiles).write + } else { + data.coalesce(1).write + } + } else { + data.write + } + } + writer.format(format).mode(mode) + if (partitionColumns.nonEmpty) { + writer.partitionBy(partitionColumns: _*) + } + println(s"Generating table $name in database to $location with save mode $mode.") + log.info(s"Generating table $name in database to $location with save mode $mode.") + writer.save(location) + sqlContext.dropTempTable(tempTableName) + } + } + + def genData( + location: String, + format: String, + overwrite: Boolean, + partitionTables: Boolean, + clusterByPartitionColumns: Boolean, + filterOutNullPartitionValues: Boolean, + tableFilter: String = "", + numPartitions: Int = 100): Unit = { + var tablesToBeGenerated = if (partitionTables) { + tables + } else { + tables.map(_.nonPartitioned) + } + + if (tableFilter.nonEmpty) { + tablesToBeGenerated = tablesToBeGenerated.filter(_.name == tableFilter) + if (tablesToBeGenerated.isEmpty) { + throw new RuntimeException("Bad table name filter: " + tableFilter) + } + } + + tablesToBeGenerated.foreach { table => + val tableLocation = s"$location/${table.name}" + table.genData( + tableLocation, + format, + overwrite, + clusterByPartitionColumns, + filterOutNullPartitionValues, + numPartitions) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometAggregateBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometAggregateBenchmark.scala new file mode 100644 index 0000000000..190fb2304b --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometAggregateBenchmark.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.DecimalType + +import org.apache.comet.CometConf + +/** + * Benchmark to measure Comet execution performance. To run this benchmark: + * {{{ + * SPARK_GENERATE_BENCHMARK_FILES=1 make benchmark-org.apache.spark.sql.benchmark.CometAggregateBenchmark + * }}} + * + * Results will be written to "spark/benchmarks/CometAggregateBenchmark-**results.txt". + */ +object CometAggregateBenchmark extends CometBenchmarkBase { + override def getSparkSession: SparkSession = { + val session = super.getSparkSession + session.conf.set("parquet.enable.dictionary", "false") + session.conf.set("spark.sql.shuffle.partitions", "2") + session + } + + def singleGroupAndAggregate(values: Int, groupingKeyCardinality: Int): Unit = { + val benchmark = + new Benchmark( + s"Grouped HashAgg Exec: single group key (cardinality $groupingKeyCardinality), single aggregate", + values, + output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql(s"SELECT value, floor(rand() * $groupingKeyCardinality) as key FROM $tbl")) + + val query = "SELECT key, SUM(value) FROM parquetV1Table GROUP BY key" + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(query).noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.run() + } + } + } + + def singleGroupAndAggregateDecimal( + values: Int, + dataType: DecimalType, + groupingKeyCardinality: Int): Unit = { + val benchmark = + new Benchmark( + s"Grouped HashAgg Exec: single group key (cardinality $groupingKeyCardinality), single aggregate on decimal", + values, + output = output) + + val df = makeDecimalDataFrame(values, dataType, false); + + withTempPath { dir => + withTempTable("parquetV1Table") { + df.createOrReplaceTempView(tbl) + prepareTable( + dir, + spark.sql( + s"SELECT dec as value, floor(rand() * $groupingKeyCardinality) as key FROM $tbl")) + + val query = "SELECT key, SUM(value) FROM parquetV1Table GROUP BY key" + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(query).noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.run() + } + } + } + + def multiGroupKeys(values: Int, groupingKeyCard: Int): Unit = { + val benchmark = + new Benchmark( + s"Grouped HashAgg Exec: multiple group keys (cardinality $groupingKeyCard), single aggregate", + values, + output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"SELECT value, floor(rand() * $groupingKeyCard) as key1, " + + s"floor(rand() * $groupingKeyCard) as key2 FROM $tbl")) + + val query = "SELECT key1, key2, SUM(value) FROM parquetV1Table GROUP BY key1, key2" + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(query).noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_MEMORY_OVERHEAD.key -> "1G") { + spark.sql(query).noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", + CometConf.COMET_MEMORY_OVERHEAD.key -> "1G") { + spark.sql(query).noop() + } + } + + benchmark.run() + } + } + } + + def multiAggregates(values: Int, groupingKeyCard: Int): Unit = { + val benchmark = + new Benchmark( + s"Grouped HashAgg Exec: single group key (cardinality $groupingKeyCard), multiple aggregates", + values, + output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"SELECT value as value1, value as value2, floor(rand() * $groupingKeyCard) as key " + + s"FROM $tbl")) + + val query = "SELECT key, SUM(value1), SUM(value2) FROM parquetV1Table GROUP BY key" + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(query).noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + val total = 1024 * 1024 * 10 + val combinations = List(100, 1024, 1024 * 1024) // number of distinct groups + + runBenchmarkWithTable("Grouped Aggregate (single group key + single aggregate)", total) { v => + for (card <- combinations) { + singleGroupAndAggregate(v, card) + } + } + + runBenchmarkWithTable("Grouped Aggregate (multiple group keys + single aggregate)", total) { + v => + for (card <- combinations) { + multiGroupKeys(v, card) + } + } + + runBenchmarkWithTable("Grouped Aggregate (single group key + multiple aggregates)", total) { + v => + for (card <- combinations) { + multiAggregates(v, card) + } + } + + runBenchmarkWithTable( + "Grouped Aggregate (single group key + single aggregate on decimal)", + total) { v => + for (card <- combinations) { + singleGroupAndAggregateDecimal(v, DecimalType(18, 10), card) + } + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometArithmeticBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometArithmeticBenchmark.scala new file mode 100644 index 0000000000..cd3fcadbd6 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometArithmeticBenchmark.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.types._ + +import org.apache.comet.CometConf + +/** + * Benchmark to measure Comet expression evaluation performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometArithmeticBenchmark` Results will be written to + * "spark/benchmarks/CometArithmeticBenchmark-**results.txt". + */ +object CometArithmeticBenchmark extends CometBenchmarkBase { + private val table = "parquetV1Table" + + def integerArithmeticBenchmark(values: Int, op: BinaryOp, useDictionary: Boolean): Unit = { + val dataType = IntegerType + val benchmark = new Benchmark( + s"Binary op ${dataType.sql}, dictionary = $useDictionary", + values, + output = output) + + withTempPath { dir => + withTempTable(table) { + prepareTable( + dir, + spark.sql( + s"SELECT CAST(value AS ${dataType.sql}) AS c1, " + + s"CAST(value AS ${dataType.sql}) c2 FROM $tbl")) + + benchmark.addCase(s"$op ($dataType) - Spark") { _ => + spark.sql(s"SELECT c1 ${op.sig} c2 FROM $table").noop() + } + + benchmark.addCase(s"$op ($dataType) - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(s"SELECT c1 ${op.sig} c2 FROM $table").noop() + } + } + + benchmark.addCase(s"$op ($dataType) - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql(s"SELECT c1 ${op.sig} c2 FROM $table").noop() + } + } + + benchmark.run() + } + } + } + + def decimalArithmeticBenchmark( + values: Int, + dataType: DecimalType, + op: BinaryOp, + useDictionary: Boolean): Unit = { + val benchmark = new Benchmark( + s"Binary op ${dataType.sql}, dictionary = $useDictionary", + values, + output = output) + val df = makeDecimalDataFrame(values, dataType, useDictionary) + + withTempPath { dir => + withTempTable(table) { + df.createOrReplaceTempView(tbl) + prepareTable(dir, spark.sql(s"SELECT dec AS c1, dec AS c2 FROM $tbl")) + + benchmark.addCase(s"$op ($dataType) - Spark") { _ => + spark.sql(s"SELECT c1 ${op.sig} c2 FROM $table").noop() + } + + benchmark.addCase(s"$op ($dataType) - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(s"SELECT c1 ${op.sig} c2 FROM $table").noop() + } + } + + benchmark.addCase(s"$op ($dataType) - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql(s"SELECT c1 ${op.sig} c2 FROM $table").noop() + } + } + + benchmark.run() + } + } + } + + private val TOTAL: Int = 1024 * 1024 * 10 + + override def runCometBenchmark(args: Array[String]): Unit = { + Seq(true, false).foreach { useDictionary => + Seq(Plus, Mul, Div).foreach { op => + for ((precision, scale) <- Seq((5, 2), (18, 10), (38, 37))) { + runBenchmark(op.name) { + decimalArithmeticBenchmark(TOTAL, DecimalType(precision, scale), op, useDictionary) + } + } + } + } + + Seq(true, false).foreach { useDictionary => + Seq(Minus, Mul).foreach { op => + runBenchmarkWithTable(op.name, TOTAL, useDictionary) { v => + integerArithmeticBenchmark(v, op, useDictionary) + } + } + } + } + + private val Plus: BinaryOp = BinaryOp("plus", "+") + private val Minus: BinaryOp = BinaryOp("minus", "-") + private val Mul: BinaryOp = BinaryOp("mul", "*") + private val Div: BinaryOp = BinaryOp("div", "/") + + case class BinaryOp(name: String, sig: String) { + override def toString: String = name + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala new file mode 100644 index 0000000000..f370400201 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import java.io.File + +import scala.util.Random + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession} +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.DecimalType + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions + +trait CometBenchmarkBase extends SqlBasedBenchmark { + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + .setAppName("CometReadBenchmark") + // Since `spark.master` always exists, overrides this value + .set("spark.master", "local[1]") + .setIfMissing("spark.driver.memory", "3g") + .setIfMissing("spark.executor.memory", "3g") + + val sparkSession = SparkSession.builder + .config(conf) + .withExtensions(new CometSparkSessionExtensions) + .getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + sparkSession.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(CometConf.COMET_ENABLED.key, "false") + sparkSession.conf.set(CometConf.COMET_EXEC_ENABLED.key, "false") + + sparkSession + } + + def runCometBenchmark(args: Array[String]): Unit + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runCometBenchmark(mainArgs) + } + + protected val tbl = "comet_table" + + protected def withTempTable(tableNames: String*)(f: => Unit): Unit = { + try f + finally tableNames.foreach(spark.catalog.dropTempView) + } + + protected def runBenchmarkWithTable( + benchmarkName: String, + values: Int, + useDictionary: Boolean = false)(f: Int => Any): Unit = { + withTempTable(tbl) { + import spark.implicits._ + spark + .range(values) + .map(_ => if (useDictionary) Random.nextLong % 5 else Random.nextLong) + .createOrReplaceTempView(tbl) + runBenchmark(benchmarkName)(f(values)) + } + } + + /** Runs function `f` with Comet on and off. */ + final def runWithComet(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality, output = output) + + benchmark.addCase(s"$name - Spark ") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + f + } + } + + benchmark.addCase(s"$name - Comet") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", + SQLConf.ANSI_ENABLED.key -> "false") { + f + } + } + + benchmark.run() + } + + protected def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = { + val testDf = if (partition.isDefined) { + df.write.partitionBy(partition.get) + } else { + df.write + } + + saveAsParquetV1Table(testDf, dir.getCanonicalPath + "/parquetV1") + } + + protected def saveAsParquetV1Table(df: DataFrameWriter[Row], dir: String): Unit = { + df.mode("overwrite").option("compression", "snappy").parquet(dir) + spark.read.parquet(dir).createOrReplaceTempView("parquetV1Table") + } + + protected def makeDecimalDataFrame( + values: Int, + decimal: DecimalType, + useDictionary: Boolean): DataFrame = { + import spark.implicits._ + + val div = if (useDictionary) 5 else values + spark + .range(values) + .map(_ % div) + .select((($"value" - 500) / 100.0) cast decimal as Symbol("dec")) + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometConditionalExpressionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometConditionalExpressionBenchmark.scala new file mode 100644 index 0000000000..032b9469ae --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometConditionalExpressionBenchmark.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.benchmark.Benchmark + +import org.apache.comet.CometConf + +/** + * Benchmark to measure Comet execution performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometConditionalExpressionBenchmark` Results will be + * written to "spark/benchmarks/CometConditionalExpressionBenchmark-**results.txt". + */ +object CometConditionalExpressionBenchmark extends CometBenchmarkBase { + + def caseWhenExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Case When Expr", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT value AS c1 FROM $tbl")) + + val query = + "select CASE WHEN c1 < 0 THEN '<0' WHEN c1 = 0 THEN '=0' ELSE '>0' END from parquetV1Table" + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(query).noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.run() + } + } + } + + def ifExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("If Expr", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT value AS c1 FROM $tbl")) + val query = "select IF (c1 < 0, '<0', '>=0') from parquetV1Table" + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(query).noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + val values = 1024 * 1024; + + runBenchmarkWithTable("caseWhenExpr", values) { v => + caseWhenExprBenchmark(v) + } + + runBenchmarkWithTable("ifExpr", values) { v => + ifExprBenchmark(v) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometDatetimeExpressionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometDatetimeExpressionBenchmark.scala new file mode 100644 index 0000000000..0af1ecade5 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometDatetimeExpressionBenchmark.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, LA} +import org.apache.spark.sql.internal.SQLConf + +/** + * Benchmark to measure Comet execution performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometDatetimeExpressionBenchmark` Results will be + * written to "spark/benchmarks/CometDatetimeExpressionBenchmark-**results.txt". + */ +object CometDatetimeExpressionBenchmark extends CometBenchmarkBase { + + def dateTruncExprBenchmark(values: Int, useDictionary: Boolean): Unit = { + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"select cast(timestamp_micros(cast(value/100000 as integer)) as date) as dt FROM $tbl")) + Seq("YEAR", "YYYY", "YY", "MON", "MONTH", "MM").foreach { level => + val isDictionary = if (useDictionary) "(Dictionary)" else "" + runWithComet(s"Date Truncate $isDictionary - $level", values) { + spark.sql(s"select trunc(dt, '$level') from parquetV1Table").noop() + } + } + } + } + } + + def timestampTruncExprBenchmark(values: Int, useDictionary: Boolean): Unit = { + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql(s"select timestamp_micros(cast(value/100000 as integer)) as ts FROM $tbl")) + Seq( + "YEAR", + "YYYY", + "YY", + "MON", + "MONTH", + "MM", + "DAY", + "DD", + "HOUR", + "MINUTE", + "SECOND", + "WEEK", + "QUARTER").foreach { level => + val isDictionary = if (useDictionary) "(Dictionary)" else "" + runWithComet(s"Timestamp Truncate $isDictionary - $level", values) { + spark.sql(s"select date_trunc('$level', ts) from parquetV1Table").noop() + } + } + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + val values = 1024 * 1024; + + withDefaultTimeZone(LA) { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> LA.getId, + "spark.sql.parquet.datetimeRebaseModeInWrite" -> "CORRECTED") { + + runBenchmarkWithTable("DateTrunc", values) { v => + dateTruncExprBenchmark(v, useDictionary = false) + } + runBenchmarkWithTable("DateTrunc (Dictionary)", values, useDictionary = true) { v => + dateTruncExprBenchmark(v, useDictionary = true) + } + runBenchmarkWithTable("TimestampTrunc", values) { v => + timestampTruncExprBenchmark(v, useDictionary = false) + } + runBenchmarkWithTable("TimestampTrunc (Dictionary)", values, useDictionary = true) { v => + timestampTruncExprBenchmark(v, useDictionary = true) + } + } + } + } + +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala new file mode 100644 index 0000000000..8f3f2438df --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometExecBenchmark.scala @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.SQLConf + +import org.apache.comet.{CometConf, CometSparkSessionExtensions} + +/** + * Benchmark to measure Comet execution performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometExecBenchmark` Results will be written to + * "spark/benchmarks/CometExecBenchmark-**results.txt". + */ +object CometExecBenchmark extends CometBenchmarkBase { + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + .setAppName("CometExecBenchmark") + .set("spark.master", "local[5]") + .setIfMissing("spark.driver.memory", "3g") + .setIfMissing("spark.executor.memory", "3g") + .set("spark.executor.memoryOverhead", "10g") + + val sparkSession = SparkSession.builder + .config(conf) + .withExtensions(new CometSparkSessionExtensions) + .getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + sparkSession.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(CometConf.COMET_ENABLED.key, "false") + sparkSession.conf.set(CometConf.COMET_EXEC_ENABLED.key, "false") + sparkSession.conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "10g") + // TODO: support dictionary encoding in vectorized execution + sparkSession.conf.set("parquet.enable.dictionary", "false") + sparkSession.conf.set("spark.sql.shuffle.partitions", "2") + + sparkSession + } + + def numericFilterExecBenchmark(values: Int, fractionOfZeros: Double): Unit = { + val percentageOfZeros = fractionOfZeros * 100 + val benchmark = + new Benchmark(s"Project + Filter Exec ($percentageOfZeros% zeros)", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"SELECT IF(RAND(1) < $fractionOfZeros, -1, value) AS c1, value AS c2 FROM " + + s"$tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select c2 + 1, c1 + 2 from parquetV1Table where c1 + 1 > 0").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select c2 + 1, c1 + 2 from parquetV1Table where c1 + 1 > 0").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select c2 + 1, c1 + 2 from parquetV1Table where c1 + 1 > 0").noop() + } + } + + benchmark.run() + } + } + } + + def subqueryExecBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Subquery", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql(s"SELECT value as col1, value + 100 as col2, value + 10 as col3 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql( + "SELECT (SELECT max(col1) AS parquetV1Table FROM parquetV1Table) AS a, " + + "col2, col3 FROM parquetV1Table") + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql( + "SELECT (SELECT max(col1) AS parquetV1Table FROM parquetV1Table) AS a, " + + "col2, col3 FROM parquetV1Table") + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql( + "SELECT (SELECT max(col1) AS parquetV1Table FROM parquetV1Table) AS a, " + + "col2, col3 FROM parquetV1Table") + } + } + + benchmark.run() + } + } + } + + def sortExecBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Sort Exec", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT * FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select * from parquetV1Table").sortWithinPartitions("value").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select * from parquetV1Table").sortWithinPartitions("value").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select * from parquetV1Table").sortWithinPartitions("value").noop() + } + } + + benchmark.run() + } + } + } + + def expandExecBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expand Exec", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql(s"SELECT value as col1, value + 100 as col2, value + 10 as col3 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark + .sql("SELECT col1, col2, SUM(col3) FROM parquetV1Table " + + "GROUP BY col1, col2 GROUPING SETS ((col1), (col2))") + .noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark + .sql("SELECT col1, col2, SUM(col3) FROM parquetV1Table " + + "GROUP BY col1, col2 GROUPING SETS ((col1), (col2))") + .noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark + .sql("SELECT col1, col2, SUM(col3) FROM parquetV1Table " + + "GROUP BY col1, col2 GROUPING SETS ((col1), (col2))") + .noop() + } + } + + benchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + runBenchmarkWithTable("Subquery", 1024 * 1024 * 10) { v => + subqueryExecBenchmark(v) + } + + runBenchmarkWithTable("Expand", 1024 * 1024 * 10) { v => + expandExecBenchmark(v) + } + + runBenchmarkWithTable("Project + Filter", 1024 * 1024 * 10) { v => + for (fractionOfZeros <- List(0.0, 0.50, 0.95)) { + numericFilterExecBenchmark(v, fractionOfZeros) + } + } + + runBenchmarkWithTable("Sort", 1024 * 1024 * 10) { v => + sortExecBenchmark(v) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometPredicateExpressionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometPredicateExpressionBenchmark.scala new file mode 100644 index 0000000000..7f1f4b44ee --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometPredicateExpressionBenchmark.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.benchmark.Benchmark + +import org.apache.comet.CometConf + +/** + * Benchmark to measure Comet execution performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometPredicateExpressionBenchmark` Results will be + * written to "spark/benchmarks/CometPredicateExpressionBenchmark -**results.txt". + */ +object CometPredicateExpressionBenchmark extends CometBenchmarkBase { + + def inExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("in Expr", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + "select CASE WHEN value < 0 THEN 'negative'" + + s" WHEN value = 0 THEN 'zero' ELSE 'positive' END c1 from $tbl")) + val query = "select * from parquetV1Table where c1 in ('positive', 'zero')" + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(query).noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql(query).noop() + } + } + + benchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + val values = 1024 * 1024; + + runBenchmarkWithTable("inExpr", values) { v => + inExprBenchmark(v) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala new file mode 100644 index 0000000000..4c2f832af9 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import java.io.File + +import scala.collection.JavaConverters._ +import scala.util.Random + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnVector + +import org.apache.comet.{CometConf, TestUtils} +import org.apache.comet.parquet.BatchReader + +/** + * Benchmark to measure Comet read performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometReadBenchmark` Results will be written to + * "spark/benchmarks/CometReadBenchmark-**results.txt". + */ +object CometReadBenchmark extends CometBenchmarkBase { + + def numericScanBenchmark(values: Int, dataType: DataType): Unit = { + // Benchmarks running through spark sql. + val sqlBenchmark = + new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM $tbl")) + + val query = dataType match { + case BooleanType => "sum(cast(id as bigint))" + case _ => "sum(id)" + } + + sqlBenchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(s"select $query from parquetV1Table").noop() + } + + sqlBenchmark.addCase("SQL Parquet - Comet") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(s"select $query from parquetV1Table").noop() + } + } + + sqlBenchmark.run() + } + } + } + + def decimalScanBenchmark(values: Int, precision: Int, scale: Int): Unit = { + val sqlBenchmark = new Benchmark( + s"SQL Single Decimal(precision: $precision, scale: $scale) Column Scan", + values, + output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"SELECT CAST(value / 10000000.0 as DECIMAL($precision, $scale)) " + + s"id FROM $tbl")) + + sqlBenchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select sum(id) from parquetV1Table").noop() + } + + sqlBenchmark.addCase("SQL Parquet - Comet") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select sum(id) from parquetV1Table").noop() + } + } + + sqlBenchmark.run() + } + } + } + + def readerBenchmark(values: Int, dataType: DataType): Unit = { + val sqlBenchmark = + new Benchmark(s"Parquet reader benchmark for $dataType", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM $tbl")) + + val enableOffHeapColumnVector = spark.sessionState.conf.offHeapColumnVectorEnabled + val vectorizedReaderBatchSize = CometConf.COMET_BATCH_SIZE.get(spark.sessionState.conf) + + var longSum = 0L + var doubleSum = 0.0 + val aggregateValue: (ColumnVector, Int) => Unit = dataType match { + case BooleanType => (col: ColumnVector, i: Int) => if (col.getBoolean(i)) longSum += 1 + case ByteType => (col: ColumnVector, i: Int) => longSum += col.getByte(i) + case ShortType => (col: ColumnVector, i: Int) => longSum += col.getShort(i) + case IntegerType => (col: ColumnVector, i: Int) => longSum += col.getInt(i) + case LongType => (col: ColumnVector, i: Int) => longSum += col.getLong(i) + case FloatType => (col: ColumnVector, i: Int) => doubleSum += col.getFloat(i) + case DoubleType => (col: ColumnVector, i: Int) => doubleSum += col.getDouble(i) + case StringType => + (col: ColumnVector, i: Int) => longSum += col.getUTF8String(i).toLongExact + } + + val files = TestUtils.listDirectory(new File(dir, "parquetV1")) + + sqlBenchmark.addCase("ParquetReader Spark") { _ => + files.map(_.asInstanceOf[String]).foreach { p => + val reader = new VectorizedParquetRecordReader( + enableOffHeapColumnVector, + vectorizedReaderBatchSize) + try { + reader.initialize(p, ("id" :: Nil).asJava) + val batch = reader.resultBatch() + val column = batch.column(0) + var totalNumRows = 0 + while (reader.nextBatch()) { + val numRows = batch.numRows() + var i = 0 + while (i < numRows) { + if (!column.isNullAt(i)) aggregateValue(column, i) + i += 1 + } + totalNumRows += batch.numRows() + } + } finally { + reader.close() + } + } + } + + sqlBenchmark.addCase("ParquetReader Comet") { _ => + files.map(_.asInstanceOf[String]).foreach { p => + val reader = new BatchReader(p, vectorizedReaderBatchSize) + reader.init() + try { + var totalNumRows = 0 + while (reader.nextBatch()) { + val batch = reader.currentBatch() + val column = batch.column(0) + val numRows = batch.numRows() + var i = 0 + while (i < numRows) { + if (!column.isNullAt(i)) aggregateValue(column, i) + i += 1 + } + totalNumRows += batch.numRows() + } + } finally { + reader.close() + } + } + } + + sqlBenchmark.run() + } + } + } + + def numericFilterScanBenchmark(values: Int, fractionOfZeros: Double): Unit = { + val percentageOfZeros = fractionOfZeros * 100 + val benchmark = + new Benchmark(s"Numeric Filter Scan ($percentageOfZeros% zeros)", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table", "parquetV2Table") { + prepareTable( + dir, + spark.sql( + s"SELECT IF(RAND(1) < $fractionOfZeros, -1, value) AS c1, value AS c2 FROM " + + s"$tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select sum(c2) from parquetV1Table where c1 + 1 > 0").noop() + } + + benchmark.addCase("SQL Parquet - Comet") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select sum(c2) from parquetV1Table where c1 + 1 > 0").noop() + } + } + + benchmark.run() + } + } + } + + def stringWithDictionaryScanBenchmark(values: Int): Unit = { + val sqlBenchmark = + new Benchmark("String Scan with Dictionary Encoding", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table", "parquetV2Table") { + prepareTable( + dir, + spark.sql(s""" + |WITH tmp + | AS (SELECT RAND() r FROM $tbl) + |SELECT + | CASE + | WHEN r < 0.2 THEN 'aaa' + | WHEN r < 0.4 THEN 'bbb' + | WHEN r < 0.6 THEN 'ccc' + | WHEN r < 0.8 THEN 'ddd' + | ELSE 'eee' + | END + |AS id + |FROM tmp + |""".stripMargin)) + + sqlBenchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select sum(length(id)) from parquetV1Table").noop() + } + + sqlBenchmark.addCase("SQL Parquet - Comet") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select sum(length(id)) from parquetV1Table").noop() + } + } + + sqlBenchmark.run() + } + } + } + + def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { + val percentageOfNulls = fractionOfNulls * 100 + val benchmark = + new Benchmark(s"String with Nulls Scan ($percentageOfNulls%)", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(value as STRING)) AS c1, " + + s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(value as STRING)) AS c2 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark + .sql("select sum(length(c2)) from parquetV1Table where c1 is " + + "not NULL and c2 is not NULL") + .noop() + } + + benchmark.addCase("SQL Parquet - Comet") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark + .sql("select sum(length(c2)) from parquetV1Table where c1 is " + + "not NULL and c2 is not NULL") + .noop() + } + } + + benchmark.run() + } + } + } + + def columnsBenchmark(values: Int, width: Int): Unit = { + val benchmark = + new Benchmark(s"Single Column Scan from $width columns", values, output = output) + + withTempPath { dir => + withTempTable("t1", "parquetV1Table") { + val middle = width / 2 + val selectExpr = (1 to width).map(i => s"value as c$i") + spark.table(tbl).selectExpr(selectExpr: _*).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql("SELECT * FROM t1")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql(s"SELECT sum(c$middle) FROM parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql(s"SELECT sum(c$middle) FROM parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def largeStringFilterScanBenchmark(values: Int, fractionOfZeros: Double): Unit = { + val percentageOfZeros = fractionOfZeros * 100 + val benchmark = + new Benchmark( + s"Large String Filter Scan ($percentageOfZeros% zeros)", + values, + output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"SELECT IF(RAND(1) < $fractionOfZeros, -1, value) AS c1, " + + s"REPEAT(CAST(value AS STRING), 100) AS c2 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() + } + + benchmark.addCase("SQL Parquet - Comet") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() + } + } + + benchmark.run() + } + } + } + + def sortedLgStrFilterScanBenchmark(values: Int, fractionOfZeros: Double): Unit = { + val percentageOfZeros = fractionOfZeros * 100 + val benchmark = + new Benchmark( + s"Sorted Lg Str Filter Scan ($percentageOfZeros% zeros)", + values, + output = output) + + withTempPath { dir => + withTempTable("parquetV1Table", "parquetV2Table") { + prepareTable( + dir, + spark.sql( + s"SELECT IF(RAND(1) < $fractionOfZeros, -1, value) AS c1, " + + s"REPEAT(CAST(value AS STRING), 100) AS c2 FROM $tbl ORDER BY c1, c2")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() + } + + benchmark.addCase("SQL Parquet - Comet") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() + } + } + + benchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + runBenchmarkWithTable("Parquet Reader", 1024 * 1024 * 15) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType).foreach { dataType => + readerBenchmark(v, dataType) + } + } + + runBenchmarkWithTable("SQL Single Numeric Column Scan", 1024 * 1024 * 15) { v => + Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) + .foreach { dataType => + numericScanBenchmark(v, dataType) + } + } + + runBenchmark("SQL Decimal Column Scan") { + withTempTable(tbl) { + import spark.implicits._ + spark.range(1024 * 1024 * 15).map(_ => Random.nextInt).createOrReplaceTempView(tbl) + + Seq((5, 2), (18, 4), (20, 8)).foreach { case (precision, scale) => + decimalScanBenchmark(1024 * 1024 * 15, precision, scale) + } + } + } + + runBenchmarkWithTable("String Scan with Dictionary", 1024 * 1024 * 15) { v => + stringWithDictionaryScanBenchmark(v) + } + + runBenchmarkWithTable("Numeric Filter Scan", 1024 * 1024 * 10) { v => + for (fractionOfZeros <- List(0.0, 0.50, 0.95)) { + numericFilterScanBenchmark(v, fractionOfZeros) + } + } + + runBenchmarkWithTable("String with Nulls Scan", 1024 * 1024 * 10) { v => + for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { + stringWithNullsScanBenchmark(v, fractionOfNulls) + } + } + + runBenchmarkWithTable("Single Column Scan From Wide Columns", 1024 * 1024 * 1) { v => + for (columnWidth <- List(10, 50, 100)) { + columnsBenchmark(v, columnWidth) + } + } + + runBenchmarkWithTable("Large String Filter Scan", 1024 * 1024) { v => + for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { + largeStringFilterScanBenchmark(v, fractionOfZeros) + } + } + + runBenchmarkWithTable("Sorted Lg Str Filter Scan", 1024 * 1024) { v => + for (fractionOfZeros <- List(0.0, 0.50, 0.999)) { + sortedLgStrFilterScanBenchmark(v, fractionOfZeros) + } + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometStringExpressionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometStringExpressionBenchmark.scala new file mode 100644 index 0000000000..eac68b93ce --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometStringExpressionBenchmark.scala @@ -0,0 +1,632 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.benchmark.Benchmark + +import org.apache.comet.CometConf + +/** + * Benchmark to measure Comet execution performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometStringExpressionBenchmark` Results will be + * written to "spark/benchmarks/CometStringExpressionBenchmark-**results.txt". + */ +object CometStringExpressionBenchmark extends CometBenchmarkBase { + + def subStringExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Substring Expr", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select substring(c1, 1, 100) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select substring(c1, 1, 100) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select substring(c1, 1, 100) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def stringSpaceExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("StringSpace Expr", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT CAST(RAND(1) * 100 AS INTEGER) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select space(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select space(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select space(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def asciiExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr ascii", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select ascii(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select ascii(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select ascii(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def bitLengthExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr bit_length", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select bit_length(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select bit_length(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select bit_length(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def octetLengthExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr octet_length", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select octet_length(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select octet_length(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select octet_length(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def upperExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr upper", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select upper(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select upper(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select upper(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def lowerExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr lower", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select lower(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select lower(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select lower(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def chrExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr chr", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select chr(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select chr(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select chr(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def initCapExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr initCap", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select initCap(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select initCap(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select initCap(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def trimExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr trim", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select trim(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select trim(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select trim(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def concatwsExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr concatws", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select concat_ws(' ', c1, c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select concat_ws(' ', c1, c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select concat_ws(' ', c1, c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def lengthExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr length", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select length(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select length(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select length(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def repeatExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr repeat", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select repeat(c1, 3) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select repeat(c1, 3) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select repeat(c1, 3) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def reverseExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr reverse", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select reverse(c1) from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select reverse(c1) from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select reverse(c1) from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def instrExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr instr", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select instr(c1, '123') from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select instr(c1, '123') from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select instr(c1, '123') from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def replaceExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr replace", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select replace(c1, '123', 'abc') from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select replace(c1, '123', 'abc') from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select replace(c1, '123', 'abc') from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + def translateExprBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Expr translate", values, output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT REPEAT(CAST(value AS STRING), 100) AS c1 FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark.sql("select translate(c1, '123456', 'aBcDeF') from parquetV1Table").noop() + } + + benchmark.addCase("SQL Parquet - Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + spark.sql("select translate(c1, '123456', 'aBcDeF') from parquetV1Table").noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + spark.sql("select translate(c1, '123456', 'aBcDeF') from parquetV1Table").noop() + } + } + + benchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + val values = 1024 * 1024; + + runBenchmarkWithTable("Substring", values) { v => + subStringExprBenchmark(v) + } + + runBenchmarkWithTable("StringSpace", values) { v => + stringSpaceExprBenchmark(v) + } + + runBenchmarkWithTable("ascii", values) { v => + asciiExprBenchmark(v) + } + + runBenchmarkWithTable("bitLength", values) { v => + bitLengthExprBenchmark(v) + } + + runBenchmarkWithTable("octet_length", values) { v => + octetLengthExprBenchmark(v) + } + + runBenchmarkWithTable("upper", values) { v => + upperExprBenchmark(v) + } + + runBenchmarkWithTable("lower", values) { v => + lowerExprBenchmark(v) + } + + runBenchmarkWithTable("chr", values) { v => + chrExprBenchmark(v) + } + + runBenchmarkWithTable("initCap", values) { v => + initCapExprBenchmark(v) + } + + runBenchmarkWithTable("trim", values) { v => + trimExprBenchmark(v) + } + + runBenchmarkWithTable("concatws", values) { v => + concatwsExprBenchmark(v) + } + + runBenchmarkWithTable("repeat", values) { v => + repeatExprBenchmark(v) + } + + runBenchmarkWithTable("length", values) { v => + lengthExprBenchmark(v) + } + + runBenchmarkWithTable("reverse", values) { v => + reverseExprBenchmark(v) + } + + runBenchmarkWithTable("instr", values) { v => + instrExprBenchmark(v) + } + + runBenchmarkWithTable("replace", values) { v => + replaceExprBenchmark(v) + } + + runBenchmarkWithTable("translate", values) { v => + translateExprBenchmark(v) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSQueryBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSQueryBenchmark.scala new file mode 100644 index 0000000000..1be5d685d9 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSQueryBenchmark.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.sql.{TPCDSQueries, TPCDSSchema} +import org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmark.tables +import org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmarkArguments +import org.apache.spark.sql.types.StructType + +/** + * Benchmark to measure Comet TPCDS query performance. + * + * To run this benchmark: + * {{{ + * // Build [tpcds-kit](https://github.com/databricks/tpcds-kit) + * cd /tmp && git clone https://github.com/databricks/tpcds-kit.git + * cd tpcds-kit/tools && make OS=MACOS + * + * // GenTPCDSData + * cd $COMET_HOME && mkdir /tmp/tpcds + * make benchmark-org.apache.spark.sql.GenTPCDSData -- --dsdgenDir /tmp/tpcds-kit/tools --location /tmp/tpcds --scaleFactor 1 + * + * // CometTPCDSQueryBenchmark + * SPARK_GENERATE_BENCHMARK_FILES=1 make benchmark-org.apache.spark.sql.benchmark.CometTPCDSQueryBenchmark -- --data-location /tmp/tpcds + * }}} + * + * Results will be written to "spark/benchmarks/CometTPCDSQueryBenchmark-**results.txt". + */ +object CometTPCDSQueryBenchmark extends CometTPCQueryBenchmarkBase with TPCDSQueries { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val benchmarkArgs = new TPCDSQueryBenchmarkArguments(mainArgs) + + // If `--query-filter` defined, filters the queries that this option selects + val queriesV1_4ToRun = filterQueries(tpcdsQueries, benchmarkArgs.queryFilter) + val queriesV2_7ToRun = filterQueries( + tpcdsQueriesV2_7, + benchmarkArgs.queryFilter, + nameSuffix = nameSuffixForQueriesV2_7) + + if ((queriesV1_4ToRun ++ queriesV2_7ToRun).isEmpty) { + throw new RuntimeException( + s"Empty queries to run. Bad query name filter: ${benchmarkArgs.queryFilter}") + } + + val tableSizes = setupTables( + benchmarkArgs.dataLocation, + createTempView = false, + tables, + TPCDSSchemaHelper.getTableColumns) + + setupCBO(cometSpark, benchmarkArgs.cboEnabled, tables) + + runQueries("tpcds", queries = queriesV1_4ToRun, tableSizes, "TPCDS Snappy") + runQueries( + "tpcds-v2.7.0", + queries = queriesV2_7ToRun, + tableSizes, + "TPCDS Snappy", + nameSuffix = nameSuffixForQueriesV2_7) + } +} + +object TPCDSSchemaHelper extends TPCDSSchema { + def getTableColumns: Map[String, StructType] = + tableColumns.map(kv => kv._1 -> StructType.fromDDL(kv._2)) +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCHQueryBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCHQueryBenchmark.scala new file mode 100644 index 0000000000..af1ee3a497 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCHQueryBenchmark.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import java.util.Locale + +/** + * Benchmark to measure Comet TPCH query performance. + * + * To run this benchmark: + * {{{ + * // Set scale factor in GB + * scale_factor=1 + * + * // GenTPCHData to create the data set at /tmp/tpch/sf1_parquet + * cd $COMET_HOME + * make benchmark-org.apache.spark.sql.GenTPCHData -- --location /tmp --scaleFactor ${scale_factor} + * + * // CometTPCHQueryBenchmark + * SPARK_GENERATE_BENCHMARK_FILES=1 make benchmark-org.apache.spark.sql.benchmark.CometTPCHQueryBenchmark -- --data-location /tmp/tpch/sf${scale_factor}_parquet + * }}} + * + * Results will be written to "spark/benchmarks/CometTPCHQueryBenchmark-**results.txt". + */ +object CometTPCHQueryBenchmark extends CometTPCQueryBenchmarkBase { + val tables: Seq[String] = + Seq("customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier") + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val benchmarkArgs = new TPCHQueryBenchmarkArguments(mainArgs) + + // List of all TPC-H queries + val tpchQueries = (1 to 22).map(n => s"q$n") + + // If `--query-filter` defined, filters the queries that this option selects + val queries = filterQueries(tpchQueries, benchmarkArgs.queryFilter) + + if (queries.isEmpty) { + throw new RuntimeException( + s"Empty queries to run. Bad query name filter: ${benchmarkArgs.queryFilter}") + } + + val tableSizes = + setupTables(benchmarkArgs.dataLocation, createTempView = !benchmarkArgs.cboEnabled, tables) + + setupCBO(cometSpark, benchmarkArgs.cboEnabled, tables) + + runQueries("tpch", queries, tableSizes, "TPCH Snappy") + runQueries("tpch-extended", queries, tableSizes, " TPCH Extended Snappy") + } +} + +/** + * Mostly copied from TPCDSQueryBenchmarkArguments. Only the help message is different TODO: make + * TPCDSQueryBenchmarkArguments extensible to avoid copies + */ +class TPCHQueryBenchmarkArguments(val args: Array[String]) { + var dataLocation: String = null + var queryFilter: Set[String] = Set.empty + var cboEnabled: Boolean = false + + parseArgs(args.toList) + validateArguments() + + private def optionMatch(optionName: String, s: String): Boolean = { + optionName == s.toLowerCase(Locale.ROOT) + } + + private def parseArgs(inputArgs: List[String]): Unit = { + var args = inputArgs + + while (args.nonEmpty) { + args match { + case optName :: value :: tail if optionMatch("--data-location", optName) => + dataLocation = value + args = tail + + case optName :: value :: tail if optionMatch("--query-filter", optName) => + queryFilter = value.toLowerCase(Locale.ROOT).split(",").map(_.trim).toSet + args = tail + + case optName :: tail if optionMatch("--cbo", optName) => + cboEnabled = true + args = tail + + case _ => + // scalastyle:off println + System.err.println("Unknown/unsupported param " + args) + // scalastyle:on println + printUsageAndExit(1) + } + } + } + + private def printUsageAndExit(exitCode: Int): Unit = { + // scalastyle:off + System.err.println(""" + |Usage: spark-submit --class <this class> <spark sql test jar> [Options] + |Options: + | --data-location Path to TPCH data + | --query-filter Queries to filter, e.g., q3,q5,q13 + | --cbo Whether to enable cost-based optimization + | + |------------------------------------------------------------------------------------------------------------------ + |In order to run this benchmark, please follow the instructions of + |org.apache.spark.sql.GenTPCHData to generate the TPCH data. + |Thereafter, the value of <TPCH data location> needs to be set to the location where the generated data is stored. + """.stripMargin) + // scalastyle:on + System.exit(exitCode) + } + + private def validateArguments(): Unit = { + if (dataLocation == null) { + // scalastyle:off println + System.err.println("Must specify a data location") + // scalastyle:on println + printUsageAndExit(-1) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCQueryBenchmarkBase.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCQueryBenchmarkBase.scala new file mode 100644 index 0000000000..d23d76fe17 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCQueryBenchmarkBase.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{CometTPCQueryBase, SparkSession} +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.catalyst.util.resourceToString +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark +import org.apache.spark.sql.execution.datasources.LogicalRelation + +import org.apache.comet.CometConf + +/** + * Base class for CometTPCDSQueryBenchmark and CometTPCHQueryBenchmark Mostly copied from + * TPCDSQueryBenchmark. TODO: make TPCDSQueryBenchmark extensible to avoid copies + */ +trait CometTPCQueryBenchmarkBase extends SqlBasedBenchmark with CometTPCQueryBase with Logging { + override def getSparkSession: SparkSession = cometSpark + + protected def runQueries( + queryLocation: String, + queries: Seq[String], + tableSizes: Map[String, Long], + benchmarkName: String, + nameSuffix: String = ""): Unit = { + queries.foreach { name => + val queryString = resourceToString( + s"$queryLocation/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + + // This is an indirect hack to estimate the size of each query's input by traversing the + // logical plan and adding up the sizes of all tables that appear in the plan. + val queryRelations = scala.collection.mutable.HashSet[String]() + cometSpark.sql(queryString).queryExecution.analyzed.foreach { + case SubqueryAlias(alias, _: LogicalRelation) => + queryRelations.add(alias.name) + case LogicalRelation(_, _, Some(catalogTable), _) => + queryRelations.add(catalogTable.identifier.table) + case HiveTableRelation(tableMeta, _, _, _, _) => + queryRelations.add(tableMeta.identifier.table) + case _ => + } + val numRows = queryRelations.map(tableSizes.getOrElse(_, 0L)).sum + val benchmark = new Benchmark(benchmarkName, numRows, 2, output = output) + benchmark.addCase(s"$name$nameSuffix") { _ => + cometSpark.sql(queryString).noop() + } + benchmark.addCase(s"$name$nameSuffix") { _ => + cometSpark.sql(queryString).noop() + } + benchmark.addCase(s"$name$nameSuffix: Comet (Scan)") { _ => + withSQLConf(CometConf.COMET_ENABLED.key -> "true") { + cometSpark.sql(queryString).noop() + } + } + benchmark.addCase(s"$name$nameSuffix: Comet (Scan, Exec)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true") { + cometSpark.sql(queryString).noop() + } + } + benchmark.run() + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala new file mode 100644 index 0000000000..55ed064425 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -0,0 +1,315 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.io.File +import java.nio.charset.StandardCharsets + +import scala.collection.mutable + +import org.apache.commons.io.FileUtils +import org.apache.spark.SparkContext +import org.apache.spark.sql.TPCDSBase +import org.apache.spark.sql.catalyst.expressions.AttributeSet +import org.apache.spark.sql.catalyst.util.resourceToString +import org.apache.spark.sql.execution.{FormattedMode, ReusedSubqueryExec, SparkPlan, SubqueryBroadcastExec, SubqueryExec} +import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite +import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec, ValidateRequirements} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.TestSparkSession + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus + +/** + * Similar to [[org.apache.spark.sql.PlanStabilitySuite]], checks that TPC-DS Comet plans don't + * change. + * + * If there are plan differences, the error message looks like this: Plans did not match: last + * approved simplified plan: /path/to/tpcds-plan-stability/approved-plans-xxx/q1/simplified.txt + * last approved explain plan: /path/to/tpcds-plan-stability/approved-plans-xxx/q1/explain.txt + * [last approved simplified plan] + * + * actual simplified plan: /path/to/tmp/q1.actual.simplified.txt actual explain plan: + * /path/to/tmp/q1.actual.explain.txt [actual simplified plan] + * + * To run the entire test suite, for instance `CometTPCDSV2_7_PlanStabilitySuite`: + * {{{ + * mvn -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" test + * }}} + * + * To re-generate golden files for entire suite, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 mvn -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" test + * }}} + */ +trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBase { + protected val baseResourcePath: File = { + getWorkspaceFilePath("spark", "src", "test", "resources", "tpcds-plan-stability").toFile + } + + private val referenceRegex = "#\\d+".r + private val normalizeRegex = "#\\d+L?".r + private val planIdRegex = "plan_id=\\d+".r + + private val clsName = this.getClass.getCanonicalName + + def goldenFilePath: String + + private val approvedAnsiPlans: Seq[String] = Seq("q83", "q83.sf100") + + private def getDirForTest(name: String): File = { + val goldenFileName = if (SQLConf.get.ansiEnabled && approvedAnsiPlans.contains(name)) { + name + ".ansi" + } else { + name + } + new File(goldenFilePath, goldenFileName) + } + + private def isApproved( + dir: File, + actualSimplifiedPlan: String, + actualExplain: String): Boolean = { + val simplifiedFile = new File(dir, "simplified.txt") + val expectedSimplified = FileUtils.readFileToString(simplifiedFile, StandardCharsets.UTF_8) + lazy val explainFile = new File(dir, "explain.txt") + lazy val expectedExplain = FileUtils.readFileToString(explainFile, StandardCharsets.UTF_8) + expectedSimplified == actualSimplifiedPlan && expectedExplain == actualExplain + } + + /** + * Serialize and save this SparkPlan. The resulting file is used by [[checkWithApproved]] to + * check stability. + * + * @param plan + * the SparkPlan + * @param name + * the name of the query + * @param explain + * the full explain output; this is saved to help debug later as the simplified plan is not + * too useful for debugging + */ + private def generateGoldenFile(plan: SparkPlan, name: String, explain: String): Unit = { + val dir = getDirForTest(name) + val simplified = getSimplifiedPlan(plan) + val foundMatch = dir.exists() && isApproved(dir, simplified, explain) + + if (!foundMatch) { + FileUtils.deleteDirectory(dir) + assert(dir.mkdirs()) + + val file = new File(dir, "simplified.txt") + FileUtils.writeStringToFile(file, simplified, StandardCharsets.UTF_8) + val fileOriginalPlan = new File(dir, "explain.txt") + FileUtils.writeStringToFile(fileOriginalPlan, explain, StandardCharsets.UTF_8) + logDebug(s"APPROVED: $file $fileOriginalPlan") + } + } + + private def checkWithApproved(plan: SparkPlan, name: String, explain: String): Unit = { + val dir = getDirForTest(name) + val tempDir = FileUtils.getTempDirectory + val actualSimplified = getSimplifiedPlan(plan) + val foundMatch = isApproved(dir, actualSimplified, explain) + + if (!foundMatch) { + // show diff with last approved + val approvedSimplifiedFile = new File(dir, "simplified.txt") + val approvedExplainFile = new File(dir, "explain.txt") + + val actualSimplifiedFile = new File(tempDir, s"$name.actual.simplified.txt") + val actualExplainFile = new File(tempDir, s"$name.actual.explain.txt") + + val approvedSimplified = + FileUtils.readFileToString(approvedSimplifiedFile, StandardCharsets.UTF_8) + // write out for debugging + FileUtils.writeStringToFile(actualSimplifiedFile, actualSimplified, StandardCharsets.UTF_8) + FileUtils.writeStringToFile(actualExplainFile, explain, StandardCharsets.UTF_8) + + fail(s""" + |Plans did not match: + |last approved simplified plan: ${approvedSimplifiedFile.getAbsolutePath} + |last approved explain plan: ${approvedExplainFile.getAbsolutePath} + | + |$approvedSimplified + | + |actual simplified plan: ${actualSimplifiedFile.getAbsolutePath} + |actual explain plan: ${actualExplainFile.getAbsolutePath} + | + |$actualSimplified + """.stripMargin) + } + } + + /** + * Get the simplified plan for a specific SparkPlan. In the simplified plan, the node only has + * its name and all the sorted reference and produced attributes names(without ExprId) and its + * simplified children as well. And we'll only identify the performance sensitive nodes, e.g., + * Exchange, Subquery, in the simplified plan. Given such a identical but simplified plan, we'd + * expect to avoid frequent plan changing and catch the possible meaningful regression. + */ + private def getSimplifiedPlan(plan: SparkPlan): String = { + val exchangeIdMap = new mutable.HashMap[Int, Int]() + val subqueriesMap = new mutable.HashMap[Int, Int]() + + def getId(plan: SparkPlan): Int = plan match { + case exchange: Exchange => + exchangeIdMap.getOrElseUpdate(exchange.id, exchangeIdMap.size + 1) + case ReusedExchangeExec(_, exchange) => + exchangeIdMap.getOrElseUpdate(exchange.id, exchangeIdMap.size + 1) + case subquery: SubqueryExec => + subqueriesMap.getOrElseUpdate(subquery.id, subqueriesMap.size + 1) + case subquery: SubqueryBroadcastExec => + subqueriesMap.getOrElseUpdate(subquery.id, subqueriesMap.size + 1) + case ReusedSubqueryExec(subquery) => + subqueriesMap.getOrElseUpdate(subquery.id, subqueriesMap.size + 1) + case _ => -1 + } + + /** + * Some expression names have ExprId in them due to using things such as + * "sum(sr_return_amt#14)", so we remove all of these using regex + */ + def cleanUpReferences(references: AttributeSet): String = { + referenceRegex.replaceAllIn(references.map(_.name).mkString(","), "") + } + + /** + * Generate a simplified plan as a string Example output: TakeOrderedAndProject + * [c_customer_id] WholeStageCodegen Project [c_customer_id] + */ + def simplifyNode(node: SparkPlan, depth: Int): String = { + val padding = " " * depth + var thisNode = node.nodeName + if (node.references.nonEmpty) { + thisNode += s" [${cleanUpReferences(node.references)}]" + } + if (node.producedAttributes.nonEmpty) { + thisNode += s" [${cleanUpReferences(node.producedAttributes)}]" + } + val id = getId(node) + if (id > 0) { + thisNode += s" #$id" + } + val childrenSimplified = node.children.map(simplifyNode(_, depth + 1)) + val subqueriesSimplified = node.subqueries.map(simplifyNode(_, depth + 1)) + s"$padding$thisNode\n${subqueriesSimplified.mkString("")}${childrenSimplified.mkString("")}" + } + + simplifyNode(plan, 0) + } + + private def normalizeIds(plan: String): String = { + val map = new mutable.HashMap[String, String]() + normalizeRegex + .findAllMatchIn(plan) + .map(_.toString) + .foreach(map.getOrElseUpdate(_, (map.size + 1).toString)) + val exprIdNormalized = + normalizeRegex.replaceAllIn(plan, regexMatch => s"#${map(regexMatch.toString)}") + + // Normalize the plan id in Exchange nodes. See `Exchange.stringArgs`. + val planIdMap = new mutable.HashMap[String, String]() + planIdRegex + .findAllMatchIn(exprIdNormalized) + .map(_.toString) + .foreach(planIdMap.getOrElseUpdate(_, (planIdMap.size + 1).toString)) + planIdRegex.replaceAllIn( + exprIdNormalized, + regexMatch => s"plan_id=${planIdMap(regexMatch.toString)}") + } + + private def normalizeLocation(plan: String): String = { + plan.replaceAll( + s"Location.*$clsName/", + "Location [not included in comparison]/{warehouse_dir}/") + } + + /** + * Test a TPC-DS query. Depending on the settings this test will either check if the plan + * matches a golden file or it will create a new golden file. + */ + protected def testQuery(tpcdsGroup: String, query: String, suffix: String = ""): Unit = { + // Only run the tests in Spark 3.4+ + assume(isSpark34Plus) + + val queryString = resourceToString( + s"$tpcdsGroup/$query.sql", + classLoader = Thread.currentThread().getContextClassLoader) + // Disable char/varchar read-side handling for better performance. + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key -> "true", + "spark.sql.readSideCharPadding" -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10MB") { + val qe = sql(queryString).queryExecution + val plan = qe.executedPlan + val explain = normalizeLocation(normalizeIds(qe.explainString(FormattedMode))) + + assert(ValidateRequirements.validate(plan)) + + if (shouldRegenerateGoldenFiles) { + generateGoldenFile(plan, query + suffix, explain) + } else { + checkWithApproved(plan, query + suffix, explain) + } + } + } + + protected override def createSparkSession: TestSparkSession = { + val conf = super.sparkConf + conf.set("spark.sql.extensions", "org.apache.comet.CometSparkSessionExtensions") + conf.set(CometConf.COMET_ENABLED.key, "true") + conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") + conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "1g") + conf.set(CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key, "true") + conf.set(CometConf.COMET_EXEC_ALL_EXPR_ENABLED.key, "true") + + new TestSparkSession(new SparkContext("local[1]", this.getClass.getCanonicalName, conf)) + } + + // TODO: remove once Spark 3.2 & 3.3 is no longer supported + private val shouldRegenerateGoldenFiles: Boolean = + System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" +} + +class CometTPCDSV1_4_PlanStabilitySuite extends CometPlanStabilitySuite { + override val goldenFilePath: String = + new File(baseResourcePath, "approved-plans-v1_4").getAbsolutePath + + tpcdsQueries.foreach { q => + test(s"check simplified (tpcds-v1.4/$q)") { + testQuery("tpcds", q) + } + } +} + +class CometTPCDSV2_7_PlanStabilitySuite extends CometPlanStabilitySuite { + override val goldenFilePath: String = + new File(baseResourcePath, "approved-plans-v2_7").getAbsolutePath + + tpcdsQueriesV2_7_0.foreach { q => + test(s"check simplified (tpcds-v2.7.0/$q)") { + testQuery("tpcds-v2.7.0", q) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala new file mode 100644 index 0000000000..5bb7e8f706 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import org.scalactic.source.Position +import org.scalatest.Tag + +import org.apache.spark.SparkException +import org.apache.spark.sql.{CometTestBase, DataFrame, Dataset, Row} +import org.apache.spark.sql.internal.SQLConf + +import org.apache.comet.CometConf + +// This test checks if Comet reads ancient dates & timestamps that are before 1582, as if they are +// read according to the `LegacyBehaviorPolicy.CORRECTED` mode (i.e., no rebase) in Spark. +abstract class ParquetDatetimeRebaseSuite extends CometTestBase { + + // This is a flag defined in Spark's `org.apache.spark.internal.config.Tests` but is only + // visible under package `spark`. + val SPARK_TESTING: String = "spark.testing" + + test("reading ancient dates before 1582") { + Seq(true, false).foreach { exceptionOnRebase => + withSQLConf( + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key -> + exceptionOnRebase.toString) { + Seq("2_4_5", "2_4_6", "3_2_0").foreach { sparkVersion => + val file = + getResourceParquetFilePath( + s"test-data/before_1582_date_v$sparkVersion.snappy.parquet") + val df = spark.read.parquet(file) + + // Parquet file written by 2.4.5 should throw exception for both Spark and Comet + if (exceptionOnRebase || sparkVersion == "2_4_5") { + intercept[SparkException](df.collect()) + } else { + checkSparkNoRebaseAnswer(df) + } + } + } + } + } + + test("reading ancient timestamps before 1582") { + Seq(true, false).foreach { exceptionOnRebase => + withSQLConf( + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key -> + exceptionOnRebase.toString) { + Seq("2_4_5", "2_4_6", "3_2_0").foreach { sparkVersion => + Seq("micros", "millis").foreach { timestampUnit => + val file = getResourceParquetFilePath( + s"test-data/before_1582_timestamp_${timestampUnit}_v${sparkVersion}.snappy.parquet") + val df = spark.read.parquet(file) + + // Parquet file written by 2.4.5 should throw exception for both Spark and Comet + if (exceptionOnRebase || sparkVersion == "2_4_5") { + intercept[SparkException](df.collect()) + } else { + checkSparkNoRebaseAnswer(df) + } + } + } + } + } + } + + test("reading ancient int96 timestamps before 1582") { + Seq(true, false).foreach { exceptionOnRebase => + withSQLConf( + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key -> + exceptionOnRebase.toString) { + Seq("2_4_5", "2_4_6", "3_2_0").foreach { sparkVersion => + Seq("dict", "plain").foreach { parquetEncoding => + val file = getResourceParquetFilePath( + s"test-data/before_1582_timestamp_int96_${parquetEncoding}_v${sparkVersion}.snappy.parquet") + val df = spark.read.parquet(file) + + // Parquet file written by 2.4.5 should throw exception for both Spark and Comet + if (exceptionOnRebase || sparkVersion == "2_4_5") { + intercept[SparkException](df.collect()) + } else { + checkSparkNoRebaseAnswer(df) + } + } + } + } + } + } + + private def checkSparkNoRebaseAnswer(df: => DataFrame): Unit = { + var expected: Array[Row] = Array.empty + + withSQLConf(CometConf.COMET_ENABLED.key -> "false", "spark.test.forceNoRebase" -> "true") { + + val previousPropertyValue = Option.apply(System.getProperty(SPARK_TESTING)) + System.setProperty(SPARK_TESTING, "true") + + val dfSpark = Dataset.ofRows(spark, df.logicalPlan) + expected = dfSpark.collect() + + previousPropertyValue match { + case Some(v) => System.setProperty(SPARK_TESTING, v) + case None => System.clearProperty(SPARK_TESTING) + } + } + + val dfComet = Dataset.ofRows(spark, df.logicalPlan) + checkAnswer(dfComet, expected) + } +} + +class ParquetDatetimeRebaseV1Suite extends ParquetDatetimeRebaseSuite { + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + super.test(testName, testTags: _*)(withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + testFun + })(pos) + } +} + +class ParquetDatetimeRebaseV2Suite extends ParquetDatetimeRebaseSuite { + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + super.test(testName, testTags: _*)(withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + testFun + })(pos) + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala new file mode 100644 index 0000000000..0d1a105ff5 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.io.File +import java.io.RandomAccessFile +import java.nio.charset.StandardCharsets +import java.util.Base64 + +import org.junit.runner.RunWith +import org.scalactic.source.Position +import org.scalatest.Tag +import org.scalatestplus.junit.JUnitRunner + +import org.apache.spark.{DebugFilesystem, SparkConf} +import org.apache.spark.sql.{QueryTest, SparkSession, SQLContext} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils + +import org.apache.comet.{CometConf, CometSparkSessionExtensions, IntegrationTestSuite} + +/** + * A integration test suite that tests parquet modular encryption usage. + */ +@RunWith(classOf[JUnitRunner]) +@IntegrationTestSuite +class ParquetEncryptionITCase extends QueryTest with SQLTestUtils { + private val encoder = Base64.getEncoder + private val footerKey = + encoder.encodeToString("0123456789012345".getBytes(StandardCharsets.UTF_8)) + private val key1 = encoder.encodeToString("1234567890123450".getBytes(StandardCharsets.UTF_8)) + private val key2 = encoder.encodeToString("1234567890123451".getBytes(StandardCharsets.UTF_8)) + + test("SPARK-34990: Write and read an encrypted parquet") { + import testImplicits._ + + Seq("org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory").foreach { + factoryClass => + withTempDir { dir => + withSQLConf( + "parquet.crypto.factory.class" -> factoryClass, + "parquet.encryption.kms.client.class" -> + "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS", + "parquet.encryption.key.list" -> + s"footerKey: ${footerKey}, key1: ${key1}, key2: ${key2}") { + + // Make sure encryption works with multiple Parquet files + val inputDF = spark + .range(0, 2000) + .map(i => (i, i.toString, i.toFloat)) + .repartition(10) + .toDF("a", "b", "c") + val parquetDir = new File(dir, "parquet").getCanonicalPath + inputDF.write + .option("parquet.encryption.column.keys", "key1: a, b; key2: c") + .option("parquet.encryption.footer.key", "footerKey") + .parquet(parquetDir) + + verifyParquetEncrypted(parquetDir) + + val parquetDF = spark.read.parquet(parquetDir) + assert(parquetDF.inputFiles.nonEmpty) + val readDataset = parquetDF.select("a", "b", "c") + checkAnswer(readDataset, inputDF) + } + } + } + } + + test("SPARK-37117: Can't read files in Parquet encryption external key material mode") { + import testImplicits._ + + Seq("org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory").foreach { + factoryClass => + withTempDir { dir => + withSQLConf( + "parquet.crypto.factory.class" -> factoryClass, + "parquet.encryption.kms.client.class" -> + "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS", + "parquet.encryption.key.material.store.internally" -> "false", + "parquet.encryption.key.list" -> + s"footerKey: ${footerKey}, key1: ${key1}, key2: ${key2}") { + + val inputDF = spark + .range(0, 2000) + .map(i => (i, i.toString, i.toFloat)) + .repartition(10) + .toDF("a", "b", "c") + val parquetDir = new File(dir, "parquet").getCanonicalPath + inputDF.write + .option("parquet.encryption.column.keys", "key1: a, b; key2: c") + .option("parquet.encryption.footer.key", "footerKey") + .parquet(parquetDir) + + val parquetDF = spark.read.parquet(parquetDir) + assert(parquetDF.inputFiles.nonEmpty) + val readDataset = parquetDF.select("a", "b", "c") + checkAnswer(readDataset, inputDF) + } + } + } + } + + protected def sparkConf: SparkConf = { + val conf = new SparkConf() + conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) + conf + } + + protected def createSparkSession: SparkSession = { + SparkSession + .builder() + .config(sparkConf) + .master("local[1]") + .withExtensions(new CometSparkSessionExtensions) + .getOrCreate() + } + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + Seq("true", "false").foreach { cometEnabled => + super.test(testName + s" Comet($cometEnabled)", testTags: _*) { + withSQLConf( + CometConf.COMET_ENABLED.key -> cometEnabled, + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_ALL_OPERATOR_ENABLED.key -> "true", + SQLConf.ANSI_ENABLED.key -> "true") { + testFun + } + } + } + } + + protected override def beforeAll(): Unit = { + if (_spark == null) _spark = createSparkSession + super.beforeAll() + } + + private var _spark: SparkSession = _ + protected implicit def spark: SparkSession = _spark + protected implicit def sqlContext: SQLContext = _spark.sqlContext + + /** + * Verify that the directory contains an encrypted parquet in encrypted footer mode by means of + * checking for all the parquet part files in the parquet directory that their magic string is + * PARE, as defined in the spec: + * https://github.com/apache/parquet-format/blob/master/Encryption.md#54-encrypted-footer-mode + */ + private def verifyParquetEncrypted(parquetDir: String): Unit = { + val parquetPartitionFiles = getListOfParquetFiles(new File(parquetDir)) + assert(parquetPartitionFiles.size >= 1) + parquetPartitionFiles.foreach { parquetFile => + val magicString = "PARE" + val magicStringLength = magicString.length() + val byteArray = new Array[Byte](magicStringLength) + val randomAccessFile = new RandomAccessFile(parquetFile, "r") + try { + randomAccessFile.read(byteArray, 0, magicStringLength) + } finally { + randomAccessFile.close() + } + val stringRead = new String(byteArray, StandardCharsets.UTF_8) + assert(magicString == stringRead) + } + } + + private def getListOfParquetFiles(dir: File): List[File] = { + dir.listFiles.filter(_.isFile).toList.filter { file => + file.getName.endsWith("parquet") + } + } +} From 5553fb7c9217558d2fde5d4652f843d855ca1db0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh <viirya@gmail.com> Date: Sat, 27 Jan 2024 15:33:30 -0800 Subject: [PATCH 2/4] Add license header to Makefile. Remove unncessary file core/.lldbinit. --- Makefile | 17 +++++++++++++++++ core/.lldbinit | 37 ------------------------------------- 2 files changed, 17 insertions(+), 37 deletions(-) delete mode 100644 core/.lldbinit diff --git a/Makefile b/Makefile index 10cb989724..475c887367 100644 --- a/Makefile +++ b/Makefile @@ -1,3 +1,20 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + .PHONY: all core jvm test clean release-linux release bench all: core jvm diff --git a/core/.lldbinit b/core/.lldbinit deleted file mode 100644 index e7242277d4..0000000000 --- a/core/.lldbinit +++ /dev/null @@ -1,37 +0,0 @@ -## -## From https://github.com/bsmt/lldbinit/blob/master/lldbinit -## - -# wish lldb supported colors :/ -settings set prompt [lldb-rust-comet]$ - -# breakpoint shortcuts -# break on function/method/selector: b -n name -# break on C/C++ method: b -M method -# break on selector: b -S selector:here: -# break on address: b -a 0xfeedface -command alias b breakpoint set -command alias bd breakpoint disable -command alias be breakpoint enable -command alias bdel breakpoint delete -command alias bcommand breakpoint command add -command alias commands breakpoint command list - -# jump aliases -# jump 0xfeedface -command alias jump register write pc -command alias jmp register write pc -command alias j register write pc - -# fix p/s -# p/s rsi -command alias p/s register read - -# fscript (cbf to fix fscript anywhere) -command alias f_init p (char)[[NSBundle bundleWithPath:@"/Library/Frameworks/FScript.framework"] load] -command alias f_start p (void)[FScriptMenuItem insertInMainMenu] - -command alias return thread return - -# For Comet Debugging -# settings set platform.plugin.darwin.ignored-exceptions EXC_BAD_INSTRUCTION|EXC_BAD_ACCESS From ce7852e825bfa934300820b71ce6440ee40de5c4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh <viirya@gmail.com> Date: Sat, 27 Jan 2024 16:52:25 -0800 Subject: [PATCH 3/4] Update DEBUGGING.md --- DEBUGGING.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/DEBUGGING.md b/DEBUGGING.md index 25a6e17d3c..e348b72159 100644 --- a/DEBUGGING.md +++ b/DEBUGGING.md @@ -33,7 +33,7 @@ _Caveat: The steps here have only been tested with JDK 11_ on Mac (M1) ## Debugging for Advanced Developers Add a `.lldbinit` to comet/core. This is not strictly necessary but will be useful if you want to -use advanced `lldb` debugging. A sample `.lldbinit` is provided in the comet/core directory +use advanced `lldb` debugging. ### In Intellij From 408dbe3542ff99d323db5f0ff0407b6cea726f81 Mon Sep 17 00:00:00 2001 From: Chao Sun <sunchao@apache.org> Date: Sat, 3 Feb 2024 12:14:09 -0800 Subject: [PATCH 4/4] add license and address comments --- LICENSE.txt | 212 ++++++++++++++++++++++++++++++++++++++++++++++++++++ README.md | 2 +- 2 files changed, 213 insertions(+), 1 deletion(-) create mode 100644 LICENSE.txt diff --git a/LICENSE.txt b/LICENSE.txt new file mode 100644 index 0000000000..d74c6b599d --- /dev/null +++ b/LICENSE.txt @@ -0,0 +1,212 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + + +This project includes code from Apache Aurora. + +* dev/release/{release,changelog,release-candidate} are based on the scripts from + Apache Aurora + +Copyright: 2016 The Apache Software Foundation. +Home page: https://aurora.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 diff --git a/README.md b/README.md index cab744874c..ba50f673a0 100644 --- a/README.md +++ b/README.md @@ -20,7 +20,7 @@ under the License. # Apache Arrow DataFusion Comet Comet is an Apache Spark plugin that uses [Apache Arrow DataFusion](https://arrow.apache.org/datafusion/) -as native runtime to achieve dramatic improvement in terms of query efficiency and query runtime. +as native runtime to achieve improvement in terms of query efficiency and query runtime. On a high level, Comet aims to support: - a native Parquet implementation, including both reader and writer